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/06/09 23:39:57 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #5011: Python: Use Pydantic for (de)serialization

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

   https://github.com/samuelcolvin/pydantic provides data validation and settings management using Python type-hints.
   
   Fast and extensible, pydantic plays nicely with your linters/IDE/brain. Define how data should be in pure, canonical Python 3.7+; validate it with pydantic.
   
   I took the basis from @samredai's PR in https://github.com/apache/iceberg/pull/3677
   
   It allows us to easily (de)serialize the TableMetadata including all the nested fields (schema, fields, and types).


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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -16,18 +16,22 @@
 # under the License.
 # pylint: disable=W0511
 
-from __future__ import annotations

Review Comment:
   There is a bug in Pydantic in combination with Python 3.8. Somewhere in a type reference of NestedField something is off. I want to dig into this, but I first wanted to get this PR ready to get feedback on 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]

Review Comment:
   Any spec in `partition_specs` will have a spec ID assigned. We only need to assign a default ID to the `partition-spec` field, and then only if `partition-specs` is undefined.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,363 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+class ValidationError(Exception):
+    ...
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:

Review Comment:
   Should this handle the case where default_sort_order_id is None?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,363 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+class ValidationError(Exception):
+    ...

Review Comment:
   Ah, missed that one. Just 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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,328 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+import os
+import tempfile
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream, FromInputFile, ToOutputFile
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet"},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+EXAMPLE_TABLE_METADATA_V2 = {
+    "format-version": 2,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "last-sequence-number": 1,
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        }
+    ],
+    "current-schema-id": 0,
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet", "read.split.target.size": 134217728},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+
+
+@pytest.mark.parametrize(
+    "metadata",
+    [
+        EXAMPLE_TABLE_METADATA_V1,
+        EXAMPLE_TABLE_METADATA_V2,
+    ],
+)
+def test_from_dict(metadata: dict):
+    """Test initialization of a TableMetadata instance from a dictionary"""
+    TableMetadata.parse_obj(metadata)

Review Comment:
   This is just checking the logic. This will be obsolete once https://github.com/samuelcolvin/pydantic/pull/3847 has been merged :) 



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/catalog/test_base.py:
##########
@@ -157,7 +158,7 @@ def update_namespace_properties(
 TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
 TEST_TABLE_NAMESPACE = ("com", "organization", "department")
 TEST_TABLE_NAME = "my_table"
-TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_SCHEMA = Schema(NestedField(1, "foo", StringType(), True), schema_id=1)

Review Comment:
   This was during testing. This wasn't possible for some time because of all the changes I made to the code, but since this is initialized before the test, it would crash right away before hitting any breakpoints. I'll revert 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   schema is a reserved keyword in Pydantic:
   
   ```python
   >>> table_metadata.schema()
   {
   	'title': 'TableMetadataV2',
   	'description': 'Metadata for an Iceberg table as specified in the Apache Iceberg\nspec (https://iceberg.apache.org/spec/#iceberg-table-spec)',
   	'type': 'object',
   	'properties': {
   		'table-uuid': {
   			'title': 'Table-Uuid',
   			'type': 'string',
   			'format': 'uuid'
   		},
   		'location': {
   			'title': 'Location',
   			'type': 'string'
   		},
   		'last-updated-ms': {
   			'title': 'Last-Updated-Ms',
   			'type': 'integer'
   		},
   		'last-column-id': {
   			'title': 'Last-Column-Id',
   			'type': 'integer'
   		},
   		'schemas': {
   			'title': 'Schemas',
   			'type': 'array',
   			'items': {
   				'$ref': '#/definitions/Schema'
   			}
   		},
   		'current-schema-id': {
   			'title': 'Current-Schema-Id',
   			'type': 'integer'
   		},
   		'partition-specs': {
   			'title': 'Partition-Specs',
   			'type': 'array',
   			'items': {}
   		},
   		'default-spec-id': {
   			'title': 'Default-Spec-Id',
   			'type': 'integer'
   		},
   		'last-partition-id': {
   			'title': 'Last-Partition-Id',
   			'type': 'integer'
   		},
   		'properties': {
   			'title': 'Properties',
   			'type': 'object'
   		},
   		'current-snapshot-id': {
   			'title': 'Current-Snapshot-Id',
   			'type': 'integer'
   		},
   		'snapshots': {
   			'title': 'Snapshots',
   			'type': 'array',
   			'items': {}
   		},
   		'snapshot-log': {
   			'title': 'Snapshot-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'metadata-log': {
   			'title': 'Metadata-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'sort-orders': {
   			'title': 'Sort-Orders',
   			'type': 'array',
   			'items': {}
   		},
   		'default-sort-order-id': {
   			'title': 'Default-Sort-Order-Id',
   			'type': 'integer'
   		},
   		'format-version': {
   			'title': 'Format-Version',
   			'enum': [2],
   			'type': 'integer'
   		},
   		'last-sequence-number': {
   			'title': 'Last-Sequence-Number',
   			'type': 'integer'
   		}
   	},
   	'required': ['table-uuid', 'location', 'last-updated-ms', 'last-column-id', 'schemas', 'current-schema-id', 'partition-specs', 'default-spec-id', 'last-partition-id', 'properties', 'current-snapshot-id', 'snapshots', 'snapshot-log', 'metadata-log', 'sort-orders', 'default-sort-order-id', 'format-version', 'last-sequence-number'],
   	'definitions': {
   		'IcebergType': {
   			'title': 'IcebergType',
   			'description': "Base type for all Iceberg Types\n\nExample:\n    >>> str(IcebergType())\n    'IcebergType()'\n    >>> repr(IcebergType())\n    'IcebergType()'",
   			'type': 'object',
   			'properties': {}
   		},
   		'NestedField': {
   			'title': 'NestedField',
   			'description': 'Represents a field of a struct, a map key, a map value, or a list element.\n\nThis is where field IDs, names, docs, and nullability are tracked.\nExample:\n    >>> str(NestedField(\n    ...     field_id=1,\n    ...     name=\'foo\',\n    ...     field_type=FixedType(22),\n    ...     required=False,\n    ... ))\n    \'1: foo: optional fixed[22]\'\n    >>> str(NestedField(\n    ...     field_id=2,\n    ...     name=\'bar\',\n    ...     field_type=LongType(),\n    ...     is_optional=False,\n    ...     doc="Just a long"\n    ... ))\n    \'2: bar: required long (Just a long)\'',
   			'type': 'object',
   			'properties': {
   				'id': {
   					'title': 'Id',
   					'type': 'integer'
   				},
   				'name': {
   					'title': 'Name',
   					'type': 'string'
   				},
   				'type': {
   					'$ref': '#/definitions/IcebergType'
   				},
   				'required': {
   					'title': 'Required',
   					'default': True,
   					'type': 'boolean'
   				},
   				'doc': {
   					'title': 'Doc',
   					'type': 'string'
   				}
   			},
   			'required': ['id', 'name', 'type']
   		},
   		'Schema': {
   			'title': 'Schema',
   			'description': 'A table Schema\n\nExample:\n    >>> from iceberg import schema\n    >>> from iceberg import types',
   			'type': 'object',
   			'properties': {
   				'fields': {
   					'title': 'Fields',
   					'type': 'array',
   					'items': {
   						'$ref': '#/definitions/NestedField'
   					}
   				},
   				'schema-id': {
   					'title': 'Schema-Id',
   					'type': 'integer'
   				},
   				'identifier-field-ids': {
   					'title': 'Identifier-Field-Ids',
   					'type': 'array',
   					'items': {
   						'type': 'integer'
   					}
   				}
   			},
   			'required': ['schema-id']
   		}
   	}
   }
   ```
   
   Therefore we need to set the alias to properly map it.
   
   I've added code to convert the v1 into v2, including a test. In v1 the schema's are optional, if it is set, I'll check if the schema is in there, otherwise, I'll add 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):
+    class Config:
+        allow_population_by_field_name = True
+
+    def dict(self, exclude_none=True, **kwargs):
+        return super().dict(exclude_none=exclude_none, **kwargs)
+
+    def json(self, exclude_none=True, by_alias=True, **kwargs):
+        return super().json(exclude_none=exclude_none, by_alias=True, **kwargs)

Review Comment:
   Looks like this still needs a comment and a fix for `by_alias`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):

Review Comment:
   Based on the comment above, I'd prefer to use a Pydantic validator:
   
   ```python
   @root_validator(pre=True)
   def populate_schemas(cls, data: Dict[str, Any]):
       # When we read a V1 format-version, we'll make sure to populate the fields
       # for V2 as well. This makes it easier downstream because we can just
       # assume that everything is a TableMetadataV2.
       # When writing, we should stick to the same version that it was,
       # because bumping the version should be an explicit operation that is up
       # to the owner of the table.
       schema = data["schema"]
       if "schemas" not in data:
           if all([schema != other_schema for other_schema in data["schemas"]]):
               data["schemas"].append(schema)
       else:
           data["schemas"] = [schema]
       data["current-schema-id"] = schema["schema-id"]
       data["last-sequence-number"] = _INITIAL_SEQUENCE_NUMBER
       return data
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()
+    _length: int = PrivateAttr()
 
-    _instances: ClassVar[Dict[int, "FixedType"]] = {}
-
-    def __new__(cls, length: int):
-        cls._instances[length] = cls._instances.get(length) or object.__new__(cls)
-        return cls._instances[length]
+    def __init__(self, length: int):
+        super().__init__(__root__=f"fixed[{length}]")
+        self._length = length
 
     @property
-    def string_type(self) -> str:
-        return f"fixed[{self.length}]"
+    def length(self) -> int:
+        return self._length
+
+    def __repr__(self) -> str:
+        return f"FixedType(length={self._length})"
 
 
-@dataclass(frozen=True, eq=True)
 class DecimalType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> DecimalType(32, 3)
         DecimalType(precision=32, scale=3)
         >>> DecimalType(8, 3) == DecimalType(8, 3)
         True
     """
 
-    precision: int = field()
-    scale: int = field()
+    __root__: str = Field()
 
-    _instances: ClassVar[Dict[Tuple[int, int], "DecimalType"]] = {}
+    _precision: int = PrivateAttr()
+    _scale: int = PrivateAttr()
 
-    def __new__(cls, precision: int, scale: int):
-        key = (precision, scale)
-        cls._instances[key] = cls._instances.get(key) or object.__new__(cls)
-        return cls._instances[key]
+    def __init__(self, precision: int, scale: int):
+        super().__init__(
+            __root__=f"decimal({precision}, {scale})",
+        )
+        self._precision = precision
+        self._scale = scale
+
+    @property
+    def precision(self) -> int:
+        return self._precision
 
     @property
-    def string_type(self) -> str:
-        return f"decimal({self.precision}, {self.scale})"
+    def scale(self) -> int:
+        return self._scale
+
+    def __repr__(self) -> str:
+        return f"DecimalType(precision={self._precision}, scale={self._scale})"
 
 
-@dataclass(frozen=True)
 class NestedField(IcebergType):
     """Represents a field of a struct, a map key, a map value, or a list element.
-

Review Comment:
   Ah, that should have been preserved. I've moved it back.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Looks good, except for the comment about sort order.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]
+            data["default_spec_id"] = INITIAL_SPEC_ID
+            data["last_partition_id"] = max(spec["spec-id"] for spec in data["partition_specs"])
+        return data
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(skip_on_failure=True)
+    def check_if_schema_is_found(cls, data: Dict[str, Any]):
+        current_schema_id = data["current_schema_id"]
+
+        for schema in data["schemas"]:
+            if schema.schema_id == current_schema_id:
+                return data
+
+        raise ValueError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+    @root_validator
+    def check_partition_spec(cls, data: Dict[str, Any]):
+        default_spec_id = data["default_spec_id"]
+
+        for spec in data["partition_specs"]:
+            if spec["spec-id"] == default_spec_id:
+                return data
+
+        raise ValueError(f"default-spec-id {default_spec_id} can't be found")
+
+    @root_validator(skip_on_failure=True)

Review Comment:
   What this means is that if there are missing fields, then we don't want to run the validation of the `sort_order`, because we have other problems (maybe missing fields). Therefore I figured to only run the validation if all prior validations have succeeded.
   
   From: https://pydantic-docs.helpmanual.io/usage/validators/#root-validators
   
   As with field validators, root validators can have `pre=True`, in which case they're called before field validation occurs (and are provided with the raw input data), or `pre=False` (the default), in which case they're called after field validation.
   
   Field validation will not occur if `pre=True` root validators raise an error. As with field validators, "post" (i.e. pre=False) root validators by default will be called even if prior validators fail; this behaviour can be changed by setting the `skip_on_failure=True` keyword argument to the validator. The `values` argument will be a dict containing the values which passed field validation and field defaults where applicable.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,363 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+class ValidationError(Exception):
+    ...
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:

Review Comment:
   The `default-sort-order-id` can't be null since it is a required field in v2, and for v1 we set it to 0 if not set:
   ```python
       @root_validator(pre=True)
       def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
           """Sets default values to be compatible with the format v2
   
           Set some sensible defaults for V1, so we comply with the schema
           this is in pre=True, meaning that this will be done before validation.
           We don't want to make the fields optional, since they are required for V2
   
           Args:
               data: The raw arguments when initializing a V1 TableMetadata
   
           Returns:
               The TableMetadata with the defaults applied
           """
   ...
           if "default-sort-order-id" not in data:
               data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
           return data
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]

Review Comment:
   So, this root_validator is after the aliases have applied (otherwise it would be `@root_validator(pre=True)`. This means that the reserved Pydantic `.schema` is already translated to `.schema_`. When we serilaize it, we go back to `schema`:
   ```json
   {
   	"location": "s3://bucket/test/location",
   	"last-updated-ms": 1602638573874,
   	"last-column-id": 3,
   	"schemas": [{
   		"fields": [{
   			"id": 1,
   			"name": "x",
   			"type": "long",
   			"required": true
   		}, {
   			"id": 2,
   			"name": "y",
   			"type": "long",
   			"required": true,
   			"doc": "comment"
   		}, {
   			"id": 3,
   			"name": "z",
   			"type": "long",
   			"required": true
   		}],
   		"schema-id": 0,
   		"identifier-field-ids": []
   	}],
   	"current-schema-id": 0,
   	"partition-specs": [{
   		"name": "x",
   		"transform": "identity",
   		"source-id": 1,
   		"field-id": 1000,
   		"spec-id": 0
   	}],
   	"default-spec-id": 0,
   	"last-partition-id": 1000,
   	"properties": {},
   	"current-snapshot-id": -1,
   	"snapshots": [{
   		"snapshot-id": 1925,
   		"timestamp-ms": 1602638573822
   	}],
   	"snapshot-log": [],
   	"metadata-log": [],
   	"sort-orders": [],
   	"default-sort-order-id": 0,
   	"table-uuid": "d20125c8-7284-442c-9aea-15fee620737c",
   	"format-version": 1,
   	"schema": {
   		"fields": [{
   			"id": 1,
   			"name": "x",
   			"type": "long",
   			"required": true
   		}, {
   			"id": 2,
   			"name": "y",
   			"type": "long",
   			"required": true,
   			"doc": "comment"
   		}, {
   			"id": 3,
   			"name": "z",
   			"type": "long",
   			"required": true
   		}],
   		"schema-id": 0,
   		"identifier-field-ids": []
   	},
   	"partition-spec": [{
   		"name": "x",
   		"transform": "identity",
   		"source-id": 1,
   		"field-id": 1000
   	}]
   }
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]

Review Comment:
   I've removed the `default_factory` since it is a bit cleaner.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)

Review Comment:
   I've added a pre-validator. This way we can also keep it cleaner down the line.



##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:

Review Comment:
   Great one, also removed the 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0
+        else:
+            check_sort_orders(data["sort_orders"])
+        return data
+
+    def to_v2(self) -> "TableMetadataV2":
+        metadata = copy(self.dict())
+        metadata["format_version"] = 2
+        return TableMetadataV2(**metadata)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   (Not urgent) One thing we will want to watch out for here is ensuring this is set when `current_schema_id` changes.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]

Review Comment:
   Should this also set `default_spec_id` here rather than in `set_v2_compatible_defaults`? That would match what `construct_schema` is doing.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    )
+
+
+@pytest.fixture(scope="session")
+def simple_list():
+    return ListType(element_id=22, element=StringType(), element_required=True)
+
+
+@pytest.fixture(scope="session")
+def simple_map():
+    return MapType(key_id=19, key_type=StringType(), value_id=25, value_type=DoubleType(), value_required=False)
+
+
+class LocalOutputFile(OutputFile):
+    """An OutputFile implementation for local files (for test use only)"""
+
+    def __init__(self, location: str):
+
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if parsed_location.scheme and parsed_location.scheme != "file":  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("LocalOutputFile location must have a scheme of `file`")
+        elif parsed_location.netloc:
+            raise ValueError(f"Network location is not allowed for LocalOutputFile: {parsed_location.netloc}")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    @property
+    def parsed_location(self) -> ParseResult:

Review Comment:
   I shamelessly copied this from the PR of @samredai, just updated it! 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(pre=True)
+    def populate_schemas(cls, data: Dict[str, Any]):
+        # When we read a V1 format-version, we'll make sure to populate the fields
+        # for V2 as well. This makes it easier downstream because we can just
+        # assume that everything is a TableMetadataV2.
+        # When writing, we should stick to the same version that it was,
+        # because bumping the version should be an explicit operation that is up
+        # to the owner of the table.
+        schema = data["schema"]
+        if "schemas" in data:

Review Comment:
   Hey @rdblue I've added tests for handling the sort-order, based on this input:
   
   - https://github.com/apache/iceberg/blob/master/core/src/test/resources/TableMetadataV1Valid.json
   - https://github.com/apache/iceberg/blob/master/core/src/test/resources/TableMetadataV2Valid.json
   
   This also shows that my previous test data was incorrect. For example, the partition spec doesn't have a spec id. I now set those to default for V1. I do believe that sort orders should work now, there are no fields to be migrated in V1/V2 if I understand correctly, it is just that it was optional before. We can also handle this as soon as we start adding the sort order classes. Let me know if I'm overlooking something 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [

Review Comment:
   I would recommend testing v1 without these newer structures to ensure that it is correctly defaulting them. Then when you test writing v1 metadata, make sure they are added.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",

Review Comment:
   This is the table location, not the metadata file location. The metadata file location isn't written into the metadata itself.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -16,18 +16,22 @@
 # under the License.
 # pylint: disable=W0511
 
-from __future__ import annotations

Review Comment:
   Is this incompatible with Pydantic?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   I'm not sure why I missed this comment. So, this is supposed to be still in, but I moved the actual parsing (regex) to the type itself. Because the schema is a `List[IcebergType]` it will try to validate that one first. As an example fixed[22]` It will not be passed to the Fixed validate method (because the `__root__` is dynamic).
   
   Before I had everything in the validate method, now I call a static method on the class itself:
   
   ```python
               if v.startswith("fixed"):
                   return FixedType.parse(v)
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0
+        else:
+            check_sort_orders(data["sort_orders"])
+        return data
+
+    def to_v2(self) -> "TableMetadataV2":
+        metadata = copy(self.dict())
+        metadata["format_version"] = 2
+        return TableMetadataV2(**metadata)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")

Review Comment:
   Yeah, we kept it nullable in Java because in the process of adding a UUID could cause weird cases where UUIDs were concurrently assigned. That would cause commits to fail because it appears to be a different table. Since UUIDs have been supported for a long time, that's no longer likely so I don't think we need to care about it for the Python library.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/refs.py:
##########
@@ -0,0 +1,37 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import Optional
+
+from pydantic import Field
+
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+MAIN_BRANCH = "main"
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"

Review Comment:
   Minor: Most of the [examples in the stdlib docs](https://docs.python.org/3/library/enum.html#enum.Enum) use upper case, which would match other constants and Java. Why not BRANCH and TAG?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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

   Nice work, @Fokko! Thanks for getting this done!


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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/refs.py:
##########
@@ -0,0 +1,37 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import Optional
+
+from pydantic import Field
+
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+MAIN_BRANCH = "main"
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"

Review Comment:
   Good one. I see this being used both ways, since they are static, I prefer uppercase. Updated!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,328 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+import os
+import tempfile
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream, FromInputFile, ToOutputFile
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet"},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+EXAMPLE_TABLE_METADATA_V2 = {
+    "format-version": 2,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "last-sequence-number": 1,
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        }
+    ],
+    "current-schema-id": 0,
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet", "read.split.target.size": 134217728},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+
+
+@pytest.mark.parametrize(
+    "metadata",
+    [
+        EXAMPLE_TABLE_METADATA_V1,
+        EXAMPLE_TABLE_METADATA_V2,
+    ],
+)
+def test_from_dict(metadata: dict):
+    """Test initialization of a TableMetadata instance from a dictionary"""
+    TableMetadata.parse_obj(metadata)

Review Comment:
   This is just checking the logic. This will be obsolete once https://github.com/samuelcolvin/pydantic/pull/3847 has been merged :) We check the contents in different 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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()

Review Comment:
   This is something special to Pydantic: https://pydantic-docs.helpmanual.io/usage/models/#custom-root-types
   
   In most cases, a class converts into a JSON dict. Using the `__root__` we can set it to a string (or a int, etc)



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'

Review Comment:
   Why remove this example? Is it no longer accurate?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    )
+
+
+@pytest.fixture(scope="session")
+def simple_list():
+    return ListType(element_id=22, element=StringType(), element_required=True)
+
+
+@pytest.fixture(scope="session")
+def simple_map():
+    return MapType(key_id=19, key_type=StringType(), value_id=25, value_type=DoubleType(), value_required=False)
+
+
+class LocalOutputFile(OutputFile):
+    """An OutputFile implementation for local files (for test use only)"""
+
+    def __init__(self, location: str):
+
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if parsed_location.scheme and parsed_location.scheme != "file":  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("LocalOutputFile location must have a scheme of `file`")
+        elif parsed_location.netloc:
+            raise ValueError(f"Network location is not allowed for LocalOutputFile: {parsed_location.netloc}")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    @property
+    def parsed_location(self) -> ParseResult:

Review Comment:
   Does this need to be exposed? It doesn't look like it is used anywhere outside of this class.
   
   Also, only `path` is called on it. Can we store just `path` instead?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:

Review Comment:
   Nit: use constant `DEFAULT_SORT_ORDER_UNSORTED`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]

Review Comment:
   (Not urgent) One thing I find slightly hard to follow, especially now that these validation methods are shared, is when the `name_with_underscores` vs `name-with-dashes` is used. I know the latter is when running a pre-validator, but it would be nice to standardize on either pre-validation or normal validation so we don't have to think about when to translate field strings by hand.
   
   It's also a bit odd that this uses `partition_specs`, but `spec-id` at the inner level... Seems like following up to use pre-validation everywhere would help readability. What do you think?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [

Review Comment:
   Hey Ryan, today I went through the spec a bit closer. I'm not an expert on how the spec grew over time, but as a good starting point I took the following examples:
   
   - https://github.com/apache/iceberg/blob/master/core/src/test/resources/TableMetadataV1Valid.json
   - https://github.com/apache/iceberg/blob/master/core/src/test/resources/TableMetadataV2Valid.json
   
   This also shows that my previous test data was incorrect. For example, the partition spec doesn't have a spec id. I now set those to default for V1.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID

Review Comment:
   These only need to be added if the incoming metadata doesn't have them. The current schema may already have an ID that is not `DEFAULT_SCHEMA_ID` and if we set this without checking, it could corrupt the table metadata.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Since refs are added as optional in V2:
   ![image](https://user-images.githubusercontent.com/1134248/175399608-383f2ff4-3b36-481d-b317-fbb7a2bf177f.png)
   
   I've added them as a property of the V2. Since they are optional, I set the `default_factory` to `dict`:
   ```
   class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
   ...
   
       refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
       """A map of snapshot references.
       The map keys are the unique snapshot reference names in the table,
       and the map values are snapshot reference objects.
       There is always a main branch reference pointing to the
       current-snapshot-id even if the refs map is null."""
   ```
   If it isn't supplied, an empty `dict` will be set.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Since refs are added as optional in V2:
   ![image](https://user-images.githubusercontent.com/1134248/175399608-383f2ff4-3b36-481d-b317-fbb7a2bf177f.png)
   
   I've added them as a property of the V2. Since they are optional, I set the `default_factory` to `dict`:
   ```python
   class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
   ...
   
       refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
       """A map of snapshot references.
       The map keys are the unique snapshot reference names in the table,
       and the map values are snapshot reference objects.
       There is always a main branch reference pointing to the
       current-snapshot-id even if the refs map is null."""
   ```
   If it isn't supplied, an empty `dict` will be set.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID

Review Comment:
   Good one, updated!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -134,7 +138,7 @@ def _lazy_id_to_accessor(self) -> dict[int, Accessor]:
 
     def as_struct(self) -> StructType:
         """Returns the underlying struct"""
-        return self._struct
+        return StructType(*self.fields)

Review Comment:
   I think I would opt to keep an inner struct like this did before, and then define `fields` to return `this.struct.fields`



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'
+
+
+def test_deserialization_decimal():
+    decimal = TestType.parse_raw('"decimal(19, 25)"')
+    assert decimal == DecimalType(19, 25)
+
+    inner = decimal.__root__
+    assert isinstance(inner, DecimalType)
+    assert inner.precision == 19
+    assert inner.scale == 25
+
+
+def test_str_decimal():
+    assert str(DecimalType(19, 25)) == "decimal(19, 25)"
+
+
+def test_repr_decimal():
+    assert repr(DecimalType(19, 25)) == "DecimalType(precision=19, scale=25)"
+
+
+def test_serialization_nestedfield():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    actual = NestedField(1, "required_field", StringType(), True, "this is a doc").json()
+    assert expected == actual
+
+
+def test_serialization_nestedfield_no_doc():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true}'
+    actual = NestedField(1, "required_field", StringType(), True).json()
+    assert expected == actual
+
+
+def test_str_nestedfield():
+    assert str(NestedField(1, "required_field", StringType(), True)) == "1: required_field: required string"
+
+
+def test_repr_nestedfield():
+    assert (
+        repr(NestedField(1, "required_field", StringType(), True))
+        == "NestedField(field_id=1, name='required_field', field_type=StringType(), required=True)"
+    )
+
+
+def test_nestedfield_by_alias():
+    # We should be able to initialize a NestedField by alias
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField(**{"id": 1, "name": "required_field", "type": "string", "required": True, "doc": "this is a doc"})
+    assert expected == actual
+
+
+def test_deserialization_nestedfield():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField.parse_raw(
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    )
+    assert expected == actual
+
+
+def test_deserialization_nestedfield_inner():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = TestType.parse_raw('{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}')
+    assert expected == actual.__root__
+
+
+def test_serialization_struct():
+    actual = StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    ).json()
+    expected = (
+        '{"type": "struct", "fields": ['
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}, '
+        '{"id": 2, "name": "optional_field", "type": "int", "required": true}'
+        "]}"
+    )
+    assert actual == expected
+
+
+def test_deserialization_struct():
+    actual = StructType.parse_raw(
+        """
+    {
+        "type": "struct",
+        "fields": [{
+                "id": 1,
+                "name": "required_field",
+                "type": "string",
+                "required": true,
+                "doc": "this is a doc"
+            },
+            {
+                "id": 2,
+                "name": "optional_field",
+                "type": "int",
+                "required": true,
+                "doc": null
+            }
+        ]
+    }
+    """
+    )
+
+    expected = StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    )
+
+    assert actual == expected
+
+
+def test_str_struct(simple_struct: StructType):
+    assert str(simple_struct) == "struct<1: required_field: required string (this is a doc), 2: optional_field: required int>"
+
+
+def test_repr_struct(simple_struct: StructType):
+    assert (
+        repr(simple_struct)
+        == "StructType(fields=[NestedField(field_id=1, name='required_field', field_type=StringType(), required=True), NestedField(field_id=2, name='optional_field', field_type=IntegerType(), required=True)])"
+    )
+
+
+def test_serialization_list(simple_list: ListType):
+    actual = simple_list.json()
+    expected = '{"type": "list", "element-id": 22, "element": "string", "element-required": true}'
+    assert actual == expected
+
+
+def test_deserialization_list(simple_list: ListType):
+    actual = ListType.parse_raw('{"type": "list", "element-id": 22, "element": "string", "element-required": true}')
+    assert actual == simple_list
+
+
+def test_str_list(simple_list: ListType):
+    assert str(simple_list) == "list<string>"
+
+
+def test_repr_list(simple_list: ListType):
+    assert repr(simple_list) == "ListType(type='list', element_id=22, element_type=StringType(), element_required=True)"
+
+
+def test_serialization_map(simple_map: MapType):
+    actual = simple_map.json()
+    expected = """{"type": "map", "key-id": 19, "key": "string", "value-id": 25, "value": "double", "value-required": false}"""
+
+    assert actual == expected
+
+
+def test_deserialization_map(simple_map: MapType):
+    actual = MapType.parse_raw(
+        """{"type": "map", "key-id": 19, "key": "string", "value-id": 25, "value": "double", "value-required": false}"""
+    )
+    assert actual == simple_map
+
+
+def test_str_map(simple_map: MapType):
+    assert str(simple_map) == "map<string, double>"
+
+
+def test_repr_map(simple_map: MapType):

Review Comment:
   It would be nice to have a more complex type that mixes maps, lists, and structs within one another, but I think this is good overall.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_schema.py:
##########
@@ -399,4 +399,20 @@ def get(self, pos: int) -> Any:
 
     accessors = build_position_accessors(table_schema_nested)
     container = TestStruct({6: TestStruct({0: "name"})})
-    assert accessors.get(16).get(container) == "name"
+    inner_accessor = accessors.get(16)
+    assert inner_accessor
+    assert inner_accessor.get(container) == "name"
+
+
+def test_serialize_schema(table_schema_simple: Schema):

Review Comment:
   Can you add some tests for nested types (structs, maps, and arrays) as well?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -181,29 +275,33 @@ class ListType(IcebergType):
         ListType(element_id=3, element_type=StringType(), element_required=True)
     """
 
-    element_id: int = field()
-    element_type: IcebergType = field()
-    element_required: bool = field(default=True)
-    element: NestedField = field(init=False, repr=False)
-
-    def __post_init__(self):
-        object.__setattr__(
-            self,
-            "element",
-            NestedField(
-                name="element",
-                required=self.element_required,
-                field_id=self.element_id,
-                field_type=self.element_type,
-            ),
+    class Config:
+        fields = {"element_field": {"exclude": True}}

Review Comment:
   Sounds reasonable to me.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):

Review Comment:
   I'm also okay with `IcebergObject`, but that might also be confusing since it is much more than the plain Python object. I don't think this is a Python convention, but I do like calling it a model because it connects it to Pydantic. It has much more than an object, for example, it has a schema: https://pydantic-docs.helpmanual.io/usage/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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'

Review Comment:
   I've converted into an abstract class that doesn't have a `__root__` defined, and therefore no representation. I'm not sure when we would want to instantiate the PrimitiveType, therefore I removed the example.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.

Review Comment:
   We can pass it to the field, but that doesn't add any value right now, until we start exporting the models (which I don't see happening any time soon :)



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]

Review Comment:
   This needs to be defaulted, 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Should this have a pre-validation that adds the `refs` structure?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]

Review Comment:
   Nice thanks! Beware that the code snippet will throw an error when `partition_spec` is not set. I've changed it to:
   ```python
   if "partition_specs" not in data:
       fields = data["partition_spec"] # Since it is required, another error will be thrown if the field is not there
       data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
   ```



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0

Review Comment:
   Use the constant instead of `0`?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:

Review Comment:
   Good one!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Doesn't look like the `parse_obj`/`parse_raw` gets invoked. The `IcebergType` is a bit of a special case, since it cannot be invoked, I think it would be okay to (mis)use the validator there. It gets called because it is part of the schema:
   ```
   class Schema:
       fields: Tuple[IcebergType. ...]
   ```
   When deserializing, it will first visit the IcebergType for validation. We can then handle the special cases for the decimal and fixed.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()
+    _length: int = PrivateAttr()
 
-    _instances: ClassVar[Dict[int, "FixedType"]] = {}
-
-    def __new__(cls, length: int):
-        cls._instances[length] = cls._instances.get(length) or object.__new__(cls)
-        return cls._instances[length]
+    def __init__(self, length: int):
+        super().__init__(__root__=f"fixed[{length}]")
+        self._length = length
 
     @property
-    def string_type(self) -> str:
-        return f"fixed[{self.length}]"
+    def length(self) -> int:
+        return self._length
+
+    def __repr__(self) -> str:
+        return f"FixedType(length={self._length})"
 
 
-@dataclass(frozen=True, eq=True)
 class DecimalType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> DecimalType(32, 3)
         DecimalType(precision=32, scale=3)
         >>> DecimalType(8, 3) == DecimalType(8, 3)
         True
     """
 
-    precision: int = field()
-    scale: int = field()
+    __root__: str = Field()
 
-    _instances: ClassVar[Dict[Tuple[int, int], "DecimalType"]] = {}
+    _precision: int = PrivateAttr()
+    _scale: int = PrivateAttr()
 
-    def __new__(cls, precision: int, scale: int):
-        key = (precision, scale)
-        cls._instances[key] = cls._instances.get(key) or object.__new__(cls)
-        return cls._instances[key]
+    def __init__(self, precision: int, scale: int):
+        super().__init__(
+            __root__=f"decimal({precision}, {scale})",
+        )
+        self._precision = precision
+        self._scale = scale
+
+    @property
+    def precision(self) -> int:
+        return self._precision
 
     @property
-    def string_type(self) -> str:
-        return f"decimal({self.precision}, {self.scale})"
+    def scale(self) -> int:
+        return self._scale
+
+    def __repr__(self) -> str:
+        return f"DecimalType(precision={self._precision}, scale={self._scale})"
 
 
-@dataclass(frozen=True)
 class NestedField(IcebergType):
     """Represents a field of a struct, a map key, a map value, or a list element.
-

Review Comment:
   Can you add the whitespace back? Or is this required by formatting?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)

Review Comment:
   Done!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet"},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+EXAMPLE_TABLE_METADATA_V2 = {
+    "format-version": 2,
+    "table-uuid": "aefee669-d568-4f9c-b732-3c0cfd3bc7b0",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "last-sequence-number": 1,
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        }
+    ],
+    "current-schema-id": 0,
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet", "read.split.target.size": 134217728},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",

Review Comment:
   `s3`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -181,29 +275,33 @@ class ListType(IcebergType):
         ListType(element_id=3, element_type=StringType(), element_required=True)
     """
 
-    element_id: int = field()
-    element_type: IcebergType = field()
-    element_required: bool = field(default=True)
-    element: NestedField = field(init=False, repr=False)
-
-    def __post_init__(self):
-        object.__setattr__(
-            self,
-            "element",
-            NestedField(
-                name="element",
-                required=self.element_required,
-                field_id=self.element_id,
-                field_type=self.element_type,
-            ),
+    class Config:
+        fields = {"element_field": {"exclude": True}}

Review Comment:
   Good question! This is because `PrivateField` requires the variable name to start with an underscore, and these variables shouldn't be used outside of the class itself. Because we use the `element_field` in the visitor as well, it makes more sense to make it an actual field, and omit it when we serialize the `ListType`.



##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())

Review Comment:
   Thanks!



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):

Review Comment:
   Looks reasonable to me.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):
+        # When we read a V1 format-version, we'll bump it to a V2 table right
+        # away by populating the required fields, and setting the version
+        data["format-version"] = 2
+        schema = data["schema"]
+        if "schemas" not in data:
+            if all([schema != other_schema for other_schema in data["schemas"]]):
+                data["schemas"].append(schema)
+        else:
+            data["schemas"] = [schema]
+        data["current-schema-id"] = schema["schema-id"]
+        data["last-sequence-number"] = _INITIAL_SEQUENCE_NUMBER
+        return TableMetadataV2(**data)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: dict = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):

Review Comment:
   Pydantic will check which version it is, and serialize the fields that are bound to the version. Of course, for V1 the schemas field is optional, and those will be set because we populate them when we initialize a class (the root_validator a comment above).
   
   The following will produce the json with last-sequence-number:
   ```python
   >>> table_metadata = TableMetadataV2(**EXAMPLE_TABLE_METADATA_V2)
   >>> table_metadata.json()
   {
   	"table-uuid": "aefee669-d568-4f9c-b732-3c0cfd3bc7b0",
   	"location": "s3://foo/bar/baz.metadata.json",
   	"last-updated-ms": 1600000000000,
   	"last-column-id": 4,
   	"schemas": [{
   		"fields": [{
   			"id": 1,
   			"name": "foo",
   			"type": "string",
   			"required": true
   		}, {
   			"id": 2,
   			"name": "bar",
   			"type": "string",
   			"required": true
   		}, {
   			"id": 3,
   			"name": "baz",
   			"type": "string",
   			"required": true
   		}, {
   			"id": 4,
   			"name": "qux",
   			"type": "string",
   			"required": true
   		}],
   		"schema-id": 0,
   		"identifier-field-ids": []
   	}],
   	"current-schema-id": 0,
   	"partition-specs": [{
   		"spec-id": 0,
   		"fields": []
   	}],
   	"default-spec-id": 0,
   	"last-partition-id": 999,
   	"properties": {
   		"owner": "root",
   		"write.format.default": "parquet",
   		"read.split.target.size": 134217728
   	},
   	"current-snapshot-id": 7681945274687743099,
   	"snapshots": [{
   		"snapshot-id": 7681945274687743099,
   		"timestamp-ms": 1637943123188,
   		"summary": {
   			"operation": "append",
   			"added-data-files": "6",
   			"added-records": "237993",
   			"added-files-size": "3386901",
   			"changed-partition-count": "1",
   			"total-records": "237993",
   			"total-files-size": "3386901",
   			"total-data-files": "6",
   			"total-delete-files": "0",
   			"total-position-deletes": "0",
   			"total-equality-deletes": "0"
   		},
   		"manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
   		"schema-id": 0
   	}],
   	"snapshot-log": [{
   		"timestamp-ms": 1637943123188,
   		"snapshot-id": 7681945274687743099
   	}],
   	"metadata-log": [{
   		"timestamp-ms": 1637943123331,
   		"metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json"
   	}],
   	"sort-orders": [{
   		"order-id": 0,
   		"fields": []
   	}],
   	"default-sort-order-id": 0,
   	"format-version": 2,
   	"last-sequence-number": 1
   }
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):

Review Comment:
   I like using `__new__` to translate at read time. I think this will work well.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):
+        # When we read a V1 format-version, we'll bump it to a V2 table right
+        # away by populating the required fields, and setting the version

Review Comment:
   Got it! I just updated the code, and the 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},

Review Comment:
   I wasn't aware of that, 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(pre=True)
+    def populate_schemas(cls, data: Dict[str, Any]):
+        # When we read a V1 format-version, we'll make sure to populate the fields
+        # for V2 as well. This makes it easier downstream because we can just
+        # assume that everything is a TableMetadataV2.
+        # When writing, we should stick to the same version that it was,
+        # because bumping the version should be an explicit operation that is up
+        # to the owner of the table.
+        schema = data["schema"]
+        if "schemas" in data:

Review Comment:
   I think you need the same handling for `partition-spec` and `partition-specs` / `default-spec-id` and some handling to initialize `sort-orders` / `default-sort-order-id`. You might want to check the spec and make sure this is correctly handling all of the optional v1 fields.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(pre=True)
+    def populate_schemas(cls, data: Dict[str, Any]):
+        # When we read a V1 format-version, we'll make sure to populate the fields

Review Comment:
   Style (minor): avoid using personal pronouns ("I" or "we") in docs or comments because it ends up making them longer and less 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):

Review Comment:
   Not something we need to do now, but eventually we will want to add `refs` to these classes.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):
+    class Config:
+        allow_population_by_field_name = True
+
+    def dict(self, exclude_none=True, **kwargs):
+        return super().dict(exclude_none=exclude_none, **kwargs)
+
+    def json(self, exclude_none=True, by_alias=True, **kwargs):
+        return super().json(exclude_none=exclude_none, by_alias=True, **kwargs)

Review Comment:
   In the superclass, the `by_alias` is set to False by default. But for Iceberg it is better to set this to `True` all the time because we rely on aliases when we convert JSON we always want to use the aliases (`schema-id` instead of `schema_id`, since `-` is an operator in Python). The same goes for excluding non-values in the case of the doc element of the NestedField that we don't want to include if it is `None`.
   
   By overriding this function, we set these values by default to True.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   schema is a reserved keyword in Pydantic:
   
   ```python
   >>> table_metadata.schema()
   {
   	'title': 'TableMetadataV2',
   	'description': 'Metadata for an Iceberg table as specified in the Apache Iceberg\nspec (https://iceberg.apache.org/spec/#iceberg-table-spec)',
   	'type': 'object',
   	'properties': {
   		'table-uuid': {
   			'title': 'Table-Uuid',
   			'type': 'string',
   			'format': 'uuid'
   		},
   		'location': {
   			'title': 'Location',
   			'type': 'string'
   		},
   		'last-updated-ms': {
   			'title': 'Last-Updated-Ms',
   			'type': 'integer'
   		},
   		'last-column-id': {
   			'title': 'Last-Column-Id',
   			'type': 'integer'
   		},
   		'schemas': {
   			'title': 'Schemas',
   			'type': 'array',
   			'items': {
   				'$ref': '#/definitions/Schema'
   			}
   		},
   		'current-schema-id': {
   			'title': 'Current-Schema-Id',
   			'type': 'integer'
   		},
   		'partition-specs': {
   			'title': 'Partition-Specs',
   			'type': 'array',
   			'items': {}
   		},
   		'default-spec-id': {
   			'title': 'Default-Spec-Id',
   			'type': 'integer'
   		},
   		'last-partition-id': {
   			'title': 'Last-Partition-Id',
   			'type': 'integer'
   		},
   		'properties': {
   			'title': 'Properties',
   			'type': 'object'
   		},
   		'current-snapshot-id': {
   			'title': 'Current-Snapshot-Id',
   			'type': 'integer'
   		},
   		'snapshots': {
   			'title': 'Snapshots',
   			'type': 'array',
   			'items': {}
   		},
   		'snapshot-log': {
   			'title': 'Snapshot-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'metadata-log': {
   			'title': 'Metadata-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'sort-orders': {
   			'title': 'Sort-Orders',
   			'type': 'array',
   			'items': {}
   		},
   		'default-sort-order-id': {
   			'title': 'Default-Sort-Order-Id',
   			'type': 'integer'
   		},
   		'format-version': {
   			'title': 'Format-Version',
   			'enum': [2],
   			'type': 'integer'
   		},
   		'last-sequence-number': {
   			'title': 'Last-Sequence-Number',
   			'type': 'integer'
   		}
   	},
   	'required': ['table-uuid', 'location', 'last-updated-ms', 'last-column-id', 'schemas', 'current-schema-id', 'partition-specs', 'default-spec-id', 'last-partition-id', 'properties', 'current-snapshot-id', 'snapshots', 'snapshot-log', 'metadata-log', 'sort-orders', 'default-sort-order-id', 'format-version', 'last-sequence-number'],
   	'definitions': {
   		'IcebergType': {
   			'title': 'IcebergType',
   			'description': "Base type for all Iceberg Types\n\nExample:\n    >>> str(IcebergType())\n    'IcebergType()'\n    >>> repr(IcebergType())\n    'IcebergType()'",
   			'type': 'object',
   			'properties': {}
   		},
   		'NestedField': {
   			'title': 'NestedField',
   			'description': 'Represents a field of a struct, a map key, a map value, or a list element.\n\nThis is where field IDs, names, docs, and nullability are tracked.\nExample:\n    >>> str(NestedField(\n    ...     field_id=1,\n    ...     name=\'foo\',\n    ...     field_type=FixedType(22),\n    ...     required=False,\n    ... ))\n    \'1: foo: optional fixed[22]\'\n    >>> str(NestedField(\n    ...     field_id=2,\n    ...     name=\'bar\',\n    ...     field_type=LongType(),\n    ...     is_optional=False,\n    ...     doc="Just a long"\n    ... ))\n    \'2: bar: required long (Just a long)\'',
   			'type': 'object',
   			'properties': {
   				'id': {
   					'title': 'Id',
   					'type': 'integer'
   				},
   				'name': {
   					'title': 'Name',
   					'type': 'string'
   				},
   				'type': {
   					'$ref': '#/definitions/IcebergType'
   				},
   				'required': {
   					'title': 'Required',
   					'default': True,
   					'type': 'boolean'
   				},
   				'doc': {
   					'title': 'Doc',
   					'type': 'string'
   				}
   			},
   			'required': ['id', 'name', 'type']
   		},
   		'Schema': {
   			'title': 'Schema',
   			'description': 'A table Schema\n\nExample:\n    >>> from iceberg import schema\n    >>> from iceberg import types',
   			'type': 'object',
   			'properties': {
   				'fields': {
   					'title': 'Fields',
   					'type': 'array',
   					'items': {
   						'$ref': '#/definitions/NestedField'
   					}
   				},
   				'schema-id': {
   					'title': 'Schema-Id',
   					'type': 'integer'
   				},
   				'identifier-field-ids': {
   					'title': 'Identifier-Field-Ids',
   					'type': 'array',
   					'items': {
   						'type': 'integer'
   					}
   				}
   			},
   			'required': ['schema-id']
   		}
   	}
   }
   """
   
   Therefore we need to set the alias to properly map it.
   
   I've added code to convert the v1 into v2, including a test. In v1 the schema's are optional, if it is set, I'll check if the schema is in there, otherwise, I'll add 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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: dict = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[2] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    last_sequence_number: int = Field(alias="last-sequence-number")
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+
+class TableMetadata:

Review Comment:
   Once https://github.com/samuelcolvin/pydantic/issues/3846 has been resolved, we can replace the TableMetadata with:
   
   ```python
   TableMetadata = Annotated[Union[TableMetadataV1, TableMetadataV2], Field(alias="format-version", discriminator="format-version")]
   ```
   
   And this is very nice because then we can just remove the boilerplate, and everything is done by pydantic
   ```python
   TableMetadata.parse_raw("{'file-format': 1, ...}")
   TableMetadata. parse_obj({'file-format': 1, ...})
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):
+    class Config:
+        allow_population_by_field_name = True
+
+    def dict(self, exclude_none=True, **kwargs):
+        return super().dict(exclude_none=exclude_none, **kwargs)
+
+    def json(self, exclude_none=True, by_alias=True, **kwargs):
+        return super().json(exclude_none=exclude_none, by_alias=True, **kwargs)

Review Comment:
   What is this implementation changing? Wouldn't the superclass's `json` method be called with these args? Is this excluding allowed arguments?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.

Review Comment:
   Minor: Why are these doc strings after the fields? Should this be passed to `Field`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id

Review Comment:
   If `current-schema-id` is already set, we should use it. If you want to check against the `schema` field's ID, then I think we should make that a check that throws `ValidationError` (or `ValueError`).



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [

Review Comment:
   Yeah, we originally had no place to put a spec ID, and didn't track multiple specs or schemas. To track multiple, we added the lists and added IDs to the objects. Partition specs were previously a list of partition fields, but they were updated to be JSON objects with `spec-id` and `fields` fields.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]

Review Comment:
   So, this root_validator is after the aliases have applied (otherwise it would be `@root_validator(pre=True)`. This means that the reserved Pydantic `.schema` is already translated to `.schema_`. When we serialize it, we go back to `schema`:
   ```json
   {
   	"location": "s3://bucket/test/location",
   	"last-updated-ms": 1602638573874,
   	"last-column-id": 3,
   	"schemas": [{
   		"fields": [{
   			"id": 1,
   			"name": "x",
   			"type": "long",
   			"required": true
   		}, {
   			"id": 2,
   			"name": "y",
   			"type": "long",
   			"required": true,
   			"doc": "comment"
   		}, {
   			"id": 3,
   			"name": "z",
   			"type": "long",
   			"required": true
   		}],
   		"schema-id": 0,
   		"identifier-field-ids": []
   	}],
   	"current-schema-id": 0,
   	"partition-specs": [{
   		"name": "x",
   		"transform": "identity",
   		"source-id": 1,
   		"field-id": 1000,
   		"spec-id": 0
   	}],
   	"default-spec-id": 0,
   	"last-partition-id": 1000,
   	"properties": {},
   	"current-snapshot-id": -1,
   	"snapshots": [{
   		"snapshot-id": 1925,
   		"timestamp-ms": 1602638573822
   	}],
   	"snapshot-log": [],
   	"metadata-log": [],
   	"sort-orders": [],
   	"default-sort-order-id": 0,
   	"table-uuid": "d20125c8-7284-442c-9aea-15fee620737c",
   	"format-version": 1,
   	"schema": {
   		"fields": [{
   			"id": 1,
   			"name": "x",
   			"type": "long",
   			"required": true
   		}, {
   			"id": 2,
   			"name": "y",
   			"type": "long",
   			"required": true,
   			"doc": "comment"
   		}, {
   			"id": 3,
   			"name": "z",
   			"type": "long",
   			"required": true
   		}],
   		"schema-id": 0,
   		"identifier-field-ids": []
   	},
   	"partition-spec": [{
   		"name": "x",
   		"transform": "identity",
   		"source-id": 1,
   		"field-id": 1000
   	}]
   }
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   I'm not sure why I missed this comment. So, this is supposed to be still in, but I moved the actual parsing (regex) to the type itself. Because the schema is a `List[IcebergType]` it will try to validate that one first. As an example `fixed[22]` It will not be passed to the Fixed validate method (because the `__root__` is dynamic).
   
   Before I had everything in the validate method, now I call a static method on the class itself:
   
   ```python
               if v.startswith("fixed"):
                   return FixedType.parse(v)
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,363 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+class ValidationError(Exception):
+    ...
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:

Review Comment:
   To elaborate, we can't set it to 0 by default, because this would also affect V2. If there would be no sort order supplied in V2, then it would default to 0.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   But then we're still parsing in the validate method, but returning a `dict` that describes the `FixedType` instead of an instantiated `FixedType`. Let me check if there is an alternative way to implement this other than a validator.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0
+        else:
+            check_sort_orders(data["sort_orders"])
+        return data
+
+    def to_v2(self) -> "TableMetadataV2":
+        metadata = copy(self.dict())
+        metadata["format_version"] = 2
+        return TableMetadataV2(**metadata)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")

Review Comment:
   Since this is defaulted in a pre-validator, I think you could move it to common and remove the `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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Okay, I see the calls to `parse`. That brings us back to my original question, which is why are we doing parsing in a validate method? Shouldn't this validate the input rather than returning the deserialized value?
   
   I would expect this to do something like the schema validator, which would change `"fixed[12]"` into `{"__root__": "fixed", "length": 12} or something.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0

Review Comment:
   Yes, this is now in line with schemas and partitions-specs



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]

Review Comment:
   I think the logic should be:
   
   ```python
   if not data["partition_specs"]:
       fields = data.get("partition_spec")
       data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
   ```



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Got it, that makes sense. I think we still want to add `main` if `current-snapshot-id` is set, so we probably do want to use a validator.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):

Review Comment:
   This name doesn't quite describe what the method does right 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   Does this correctly translate?
   
   When reading, we want to accept a `schema`, but convert it into a `schemas` list and `current-schema-id`. In v1, we want to write both `schema` and `schemas` / `current-schema-id` because adding that metadata is compatible.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()

Review Comment:
   What is `__root__`? Is there a reason not to name it `__as_str__` or something simliar?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")

Review Comment:
   Should this be a UUID class rather than a string? The Avro code uses a UUID: https://github.com/apache/iceberg/pull/4920/files#diff-cf9ad84745a2a233b007a20fc7c8fa1862de0d38a93aa7f89166114624e045b1R136



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")

Review Comment:
   Great 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):
+    class Config:
+        allow_population_by_field_name = True
+
+    def dict(self, exclude_none=True, **kwargs):
+        return super().dict(exclude_none=exclude_none, **kwargs)
+
+    def json(self, exclude_none=True, by_alias=True, **kwargs):
+        return super().json(exclude_none=exclude_none, by_alias=True, **kwargs)

Review Comment:
   Thanks for explaining! That makes sense. Can you make sure that this is clear in a comment? That way future readers will also know.
   
   Also, should the `by_alias` passed to the method be passed to the `super().json(...)` call? Right now it ignores 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'

Review Comment:
   Isn't this type invalid because the scale is greater than the precision? (I think we check that in Java)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/catalog/test_base.py:
##########
@@ -157,7 +158,7 @@ def update_namespace_properties(
 TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
 TEST_TABLE_NAMESPACE = ("com", "organization", "department")
 TEST_TABLE_NAME = "my_table"
-TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_SCHEMA = Schema(NestedField(1, "foo", StringType(), True), schema_id=1)

Review Comment:
   Is it still okay to have an empty table 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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'
+
+
+def test_deserialization_decimal():
+    decimal = TestType.parse_raw('"decimal(19, 25)"')
+    assert decimal == DecimalType(19, 25)
+
+    inner = decimal.__root__
+    assert isinstance(inner, DecimalType)
+    assert inner.precision == 19
+    assert inner.scale == 25
+
+
+def test_str_decimal():
+    assert str(DecimalType(19, 25)) == "decimal(19, 25)"
+
+
+def test_repr_decimal():
+    assert repr(DecimalType(19, 25)) == "DecimalType(precision=19, scale=25)"
+
+
+def test_serialization_nestedfield():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    actual = NestedField(1, "required_field", StringType(), True, "this is a doc").json()
+    assert expected == actual
+
+
+def test_serialization_nestedfield_no_doc():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true}'
+    actual = NestedField(1, "required_field", StringType(), True).json()
+    assert expected == actual
+
+
+def test_str_nestedfield():
+    assert str(NestedField(1, "required_field", StringType(), True)) == "1: required_field: required string"
+
+
+def test_repr_nestedfield():
+    assert (
+        repr(NestedField(1, "required_field", StringType(), True))
+        == "NestedField(field_id=1, name='required_field', field_type=StringType(), required=True)"
+    )
+
+
+def test_nestedfield_by_alias():
+    # We should be able to initialize a NestedField by alias
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField(**{"id": 1, "name": "required_field", "type": "string", "required": True, "doc": "this is a doc"})
+    assert expected == actual
+
+
+def test_deserialization_nestedfield():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField.parse_raw(
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    )
+    assert expected == actual
+
+
+def test_deserialization_nestedfield_inner():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = TestType.parse_raw('{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}')
+    assert expected == actual.__root__
+
+
+def test_serialization_struct():
+    actual = StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    ).json()
+    expected = (
+        '{"type": "struct", "fields": ['
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}, '
+        '{"id": 2, "name": "optional_field", "type": "int", "required": true}'
+        "]}"
+    )
+    assert actual == expected
+
+
+def test_deserialization_struct():
+    actual = StructType.parse_raw(
+        """
+    {
+        "type": "struct",
+        "fields": [{
+                "id": 1,
+                "name": "required_field",
+                "type": "string",
+                "required": true,
+                "doc": "this is a doc"
+            },
+            {
+                "id": 2,
+                "name": "optional_field",
+                "type": "int",
+                "required": true,
+                "doc": null

Review Comment:
   This is a deserialization input, so that should be possible. 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)

Review Comment:
   Can you create a constant for the regex instead of compiling it every time?



##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Why is the parsing done in the validate 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")

Review Comment:
   This is actually quite nice of Pydantic. The default json dumps doesn't allow us to encode a uuid into json:
   ```python
   ➜  python git:(fd-read-avro) python3     
   Python 3.9.13 (main, May 19 2022, 13:48:47) 
   [Clang 13.1.6 (clang-1316.0.21.2)] on darwin
   Type "help", "copyright", "credits" or "license" for more information.
   >>> from uuid import UUID
   >>> example = UUID("f79c3e09-677c-4bbd-a479-3f349cb785e7")
   >>> import json
   >>> json.dumps({"uuid": example})
   Traceback (most recent call last):
     File "<stdin>", line 1, in <module>
     File "/opt/homebrew/Cellar/python@3.9/3.9.13/Frameworks/Python.framework/Versions/3.9/lib/python3.9/json/__init__.py", line 231, in dumps
       return _default_encoder.encode(obj)
     File "/opt/homebrew/Cellar/python@3.9/3.9.13/Frameworks/Python.framework/Versions/3.9/lib/python3.9/json/encoder.py", line 199, in encode
       chunks = self.iterencode(o, _one_shot=True)
     File "/opt/homebrew/Cellar/python@3.9/3.9.13/Frameworks/Python.framework/Versions/3.9/lib/python3.9/json/encoder.py", line 257, in iterencode
       return _iterencode(o, 0)
     File "/opt/homebrew/Cellar/python@3.9/3.9.13/Frameworks/Python.framework/Versions/3.9/lib/python3.9/json/encoder.py", line 179, in default
       raise TypeError(f'Object of type {o.__class__.__name__} '
   TypeError: Object of type UUID is not JSON serializable
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -134,7 +138,7 @@ def _lazy_id_to_accessor(self) -> dict[int, Accessor]:
 
     def as_struct(self) -> StructType:
         """Returns the underlying struct"""
-        return self._struct
+        return StructType(*self.fields)

Review Comment:
   I tried this, but that creates a lot of awkward code since it serializes as fields. I can cache it on the first call, wdyt?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]
+            data["default_spec_id"] = INITIAL_SPEC_ID
+            data["last_partition_id"] = max(spec["spec-id"] for spec in data["partition_specs"])
+        return data
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(skip_on_failure=True)
+    def check_if_schema_is_found(cls, data: Dict[str, Any]):
+        current_schema_id = data["current_schema_id"]
+
+        for schema in data["schemas"]:
+            if schema.schema_id == current_schema_id:
+                return data
+
+        raise ValueError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+    @root_validator
+    def check_partition_spec(cls, data: Dict[str, Any]):
+        default_spec_id = data["default_spec_id"]
+
+        for spec in data["partition_specs"]:
+            if spec["spec-id"] == default_spec_id:
+                return data
+
+        raise ValueError(f"default-spec-id {default_spec_id} can't be found")
+
+    @root_validator(skip_on_failure=True)
+    def check_sort_order(cls, data: Dict[str, Any]):
+        default_sort_order_id = data["default_sort_order_id"]
+
+        # 0 == unsorted
+        if default_sort_order_id != 0:
+            for sort in data["sort_orders"]:
+                if sort["order-id"] == default_sort_order_id:
+                    return data
+
+            raise ValueError(f"default-sort-order-id {default_sort_order_id} can't be found")
+        return data
+
+    format_version: Literal[2] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    last_sequence_number: int = Field(alias="last-sequence-number", default=_INITIAL_SEQUENCE_NUMBER)
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)

Review Comment:
   This can be in common metadata. There's no reason why v1 tables can't also have refs. Adding refs is a forward-compatible change.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]

Review Comment:
   This currently defaults to an empty list:
   ```python
   class TableMetadataCommonFields(IcebergBaseModel):
   ...
       schemas: List[Schema] = Field(default_factory=list)
       """A list of schemas, stored as objects with schema-id."""
   ```
   I figured, that for v2 you could also supply an empty list of schemas. This would be accepted for the field validator (the field is there) but will fail when we check if the `current_schema_id` is present.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):

Review Comment:
   I went all-in on the spec today, and added them 👍🏻 



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]

Review Comment:
   Don't we use the `schema` alias for serialization? Or is that translation done in the JSON to dict mapping, so we will have `schema_` in the 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:

Review Comment:
   Nit: Should this be `construct_schemas` (plural, like `construct_partition_specs` below)?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object

Review Comment:
   Nit: copy & paste not quite updated, since this talks about sort-order.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    )
+
+
+@pytest.fixture(scope="session")
+def simple_list():
+    return ListType(element_id=22, element=StringType(), element_required=True)
+
+
+@pytest.fixture(scope="session")
+def simple_map():
+    return MapType(key_id=19, key_type=StringType(), value_id=25, value_type=DoubleType(), value_required=False)
+
+
+class LocalOutputFile(OutputFile):
+    """An OutputFile implementation for local files (for test use only)"""
+
+    def __init__(self, location: str):
+
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if parsed_location.scheme and parsed_location.scheme != "file":  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("LocalOutputFile location must have a scheme of `file`")
+        elif parsed_location.netloc:
+            raise ValueError(f"Network location is not allowed for LocalOutputFile: {parsed_location.netloc}")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    def __len__(self):
+        return os.path.getsize(self.parsed_location.path)
+
+    def exists(self):
+        return os.path.exists(self.parsed_location.path)
+
+    def to_input_file(self):
+        return LocalInputFile(location=self.location)
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        output_file = open(self.parsed_location.path, "wb" if overwrite else "xb")
+        if not isinstance(output_file, OutputStream):
+            raise TypeError("Object returned from LocalOutputFile.create(...) does not match the OutputStream protocol.")
+        return output_file
+
+
+class LocalFileIO(FileIO):
+    """A FileIO implementation for local files (for test use only)"""
+
+    def new_input(self, location: str):
+        return LocalInputFile(location=location)
+
+    def new_output(self, location: str):
+        return LocalOutputFile(location=location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]):
+        location = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        try:
+            os.remove(location)
+        except FileNotFoundError as e:
+            raise FileNotFoundError(f"Cannot delete file, does not exist: {location}") from e

Review Comment:
   If the file doesn't exist, why not just return? It doesn't seem worth throwing an exception.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -181,29 +275,33 @@ class ListType(IcebergType):
         ListType(element_id=3, element_type=StringType(), element_required=True)
     """
 
-    element_id: int = field()
-    element_type: IcebergType = field()
-    element_required: bool = field(default=True)
-    element: NestedField = field(init=False, repr=False)
-
-    def __post_init__(self):
-        object.__setattr__(
-            self,
-            "element",
-            NestedField(
-                name="element",
-                required=self.element_required,
-                field_id=self.element_id,
-                field_type=self.element_type,
-            ),
+    class Config:
+        fields = {"element_field": {"exclude": True}}

Review Comment:
   Why use `Config`? Other classes use `PrivateField`. What's the difference?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())

Review Comment:
   Nit: passing `True` without a keyword.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):

Review Comment:
   Is there a better name for this? How about `IcebergObject`? I don't think `BaseModel` is a clear name that we want to carry forward, unless this is a convention that I'm missing. To me, `IcebergObject` does signal that it is layering in some common functionality.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]
+            data["default_spec_id"] = INITIAL_SPEC_ID
+            data["last_partition_id"] = max(spec["spec-id"] for spec in data["partition_specs"])
+        return data
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(skip_on_failure=True)
+    def check_if_schema_is_found(cls, data: Dict[str, Any]):
+        current_schema_id = data["current_schema_id"]
+
+        for schema in data["schemas"]:
+            if schema.schema_id == current_schema_id:
+                return data
+
+        raise ValueError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+    @root_validator
+    def check_partition_spec(cls, data: Dict[str, Any]):
+        default_spec_id = data["default_spec_id"]
+
+        for spec in data["partition_specs"]:
+            if spec["spec-id"] == default_spec_id:
+                return data
+
+        raise ValueError(f"default-spec-id {default_spec_id} can't be found")
+
+    @root_validator(skip_on_failure=True)

Review Comment:
   Got it. So just don't run if other validators have failed. Makes sense!



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]

Review Comment:
   I agree, it is a bit confusing. I think we should as little as pre-validators as possible, and try to standardize the use of the underscores.
   
   > It's also a bit odd that this uses partition_specs, but spec-id at the inner level... Seems like following up to use pre-validation everywhere would help readability. What do you think?
   
   This will be fixed once we start moving the downstream objects to Pydantic as well. `spec["spec-id"]` will change into `spec.spec_id` since it is then an object (in the post-validation at least :)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'

Review Comment:
   Good one, I've created an issue https://github.com/apache/iceberg/issues/5038



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -134,7 +138,7 @@ def _lazy_id_to_accessor(self) -> dict[int, Accessor]:
 
     def as_struct(self) -> StructType:
         """Returns the underlying struct"""
-        return self._struct
+        return StructType(*self.fields)

Review Comment:
   No worries, we can keep it as is for 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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'
+
+
+def test_deserialization_decimal():
+    decimal = TestType.parse_raw('"decimal(19, 25)"')
+    assert decimal == DecimalType(19, 25)
+
+    inner = decimal.__root__
+    assert isinstance(inner, DecimalType)
+    assert inner.precision == 19
+    assert inner.scale == 25
+
+
+def test_str_decimal():
+    assert str(DecimalType(19, 25)) == "decimal(19, 25)"
+
+
+def test_repr_decimal():
+    assert repr(DecimalType(19, 25)) == "DecimalType(precision=19, scale=25)"
+
+
+def test_serialization_nestedfield():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    actual = NestedField(1, "required_field", StringType(), True, "this is a doc").json()
+    assert expected == actual
+
+
+def test_serialization_nestedfield_no_doc():
+    expected = '{"id": 1, "name": "required_field", "type": "string", "required": true}'
+    actual = NestedField(1, "required_field", StringType(), True).json()
+    assert expected == actual
+
+
+def test_str_nestedfield():
+    assert str(NestedField(1, "required_field", StringType(), True)) == "1: required_field: required string"
+
+
+def test_repr_nestedfield():
+    assert (
+        repr(NestedField(1, "required_field", StringType(), True))
+        == "NestedField(field_id=1, name='required_field', field_type=StringType(), required=True)"
+    )
+
+
+def test_nestedfield_by_alias():
+    # We should be able to initialize a NestedField by alias
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField(**{"id": 1, "name": "required_field", "type": "string", "required": True, "doc": "this is a doc"})
+    assert expected == actual
+
+
+def test_deserialization_nestedfield():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = NestedField.parse_raw(
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}'
+    )
+    assert expected == actual
+
+
+def test_deserialization_nestedfield_inner():
+    expected = NestedField(1, "required_field", StringType(), True, "this is a doc")
+    actual = TestType.parse_raw('{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}')
+    assert expected == actual.__root__
+
+
+def test_serialization_struct():
+    actual = StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    ).json()
+    expected = (
+        '{"type": "struct", "fields": ['
+        '{"id": 1, "name": "required_field", "type": "string", "required": true, "doc": "this is a doc"}, '
+        '{"id": 2, "name": "optional_field", "type": "int", "required": true}'
+        "]}"
+    )
+    assert actual == expected
+
+
+def test_deserialization_struct():
+    actual = StructType.parse_raw(
+        """
+    {
+        "type": "struct",
+        "fields": [{
+                "id": 1,
+                "name": "required_field",
+                "type": "string",
+                "required": true,
+                "doc": "this is a doc"
+            },
+            {
+                "id": 2,
+                "name": "optional_field",
+                "type": "int",
+                "required": true,
+                "doc": null

Review Comment:
   Minor: Is it possible to omit the field rather than making it explicitly null?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/schema.py:
##########
@@ -343,23 +339,23 @@ def _(obj: StructType, visitor: SchemaVisitor[T]) -> T:
 def _(obj: ListType, visitor: SchemaVisitor[T]) -> T:
     """Visit a ListType with a concrete SchemaVisitor"""
 
-    visitor.before_list_element(obj.element)
-    result = visit(obj.element.field_type, visitor)
-    visitor.after_list_element(obj.element)
+    visitor.before_list_element(obj.element_field)
+    result = visit(obj.element_type, visitor)
+    visitor.after_list_element(obj.element_field)
 
     return visitor.list(obj, result)
 
 
 @visit.register(MapType)
 def _(obj: MapType, visitor: SchemaVisitor[T]) -> T:
     """Visit a MapType with a concrete SchemaVisitor"""
-    visitor.before_map_key(obj.key)
-    key_result = visit(obj.key.field_type, visitor)
-    visitor.after_map_key(obj.key)
+    visitor.before_map_key(obj.key_field)

Review Comment:
   We have to change these since we call them by the dataclass alias, but with Pydantic you need to call them by the variable name, which I like since it is more explicit.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id

Review Comment:
   If `current-schema-id` is already set, we should use it. If you want to check against the `schema` field's ID, then I think we should make that a check that throws `ValidationError`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]
+            data["default_spec_id"] = INITIAL_SPEC_ID
+            data["last_partition_id"] = max(spec["spec-id"] for spec in data["partition_specs"])
+        return data
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(skip_on_failure=True)
+    def check_if_schema_is_found(cls, data: Dict[str, Any]):
+        current_schema_id = data["current_schema_id"]
+
+        for schema in data["schemas"]:
+            if schema.schema_id == current_schema_id:
+                return data
+
+        raise ValueError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+    @root_validator
+    def check_partition_spec(cls, data: Dict[str, Any]):
+        default_spec_id = data["default_spec_id"]
+
+        for spec in data["partition_specs"]:
+            if spec["spec-id"] == default_spec_id:
+                return data
+
+        raise ValueError(f"default-spec-id {default_spec_id} can't be found")
+
+    @root_validator(skip_on_failure=True)

Review Comment:
   Why skip on failure? If the order ID is defined, it should match a sort order.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Okay, I see the calls to `parse`. That brings us back to my original question, which is why are we doing parsing in a validate method? Shouldn't this validate the input rather than returning the deserialized value?
   
   I would expect this to do something like the schema validator, which would change `"fixed[12]"` into `{"__root__": "fixed", "length": 12}` or something.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id
+        return data
+
+    @root_validator()
+    def migrate_partition_spec(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if partition_spec := data.get("partition_spec"):
+            data["partition_specs"] = [{**spec, "spec-id": INITIAL_SPEC_ID + idx} for idx, spec in enumerate(partition_spec)]
+            data["default_spec_id"] = INITIAL_SPEC_ID
+            data["last_partition_id"] = max(spec["spec-id"] for spec in data["partition_specs"])
+        return data
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: List[Dict[str, Any]] = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(skip_on_failure=True)
+    def check_if_schema_is_found(cls, data: Dict[str, Any]):
+        current_schema_id = data["current_schema_id"]
+
+        for schema in data["schemas"]:
+            if schema.schema_id == current_schema_id:
+                return data
+
+        raise ValueError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+    @root_validator
+    def check_partition_spec(cls, data: Dict[str, Any]):
+        default_spec_id = data["default_spec_id"]
+
+        for spec in data["partition_specs"]:
+            if spec["spec-id"] == default_spec_id:
+                return data
+
+        raise ValueError(f"default-spec-id {default_spec_id} can't be found")
+
+    @root_validator(skip_on_failure=True)
+    def check_sort_order(cls, data: Dict[str, Any]):
+        default_sort_order_id = data["default_sort_order_id"]
+
+        # 0 == unsorted
+        if default_sort_order_id != 0:
+            for sort in data["sort_orders"]:
+                if sort["order-id"] == default_sort_order_id:
+                    return data
+
+            raise ValueError(f"default-sort-order-id {default_sort_order_id} can't be found")
+        return data
+
+    format_version: Literal[2] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    last_sequence_number: int = Field(alias="last-sequence-number", default=_INITIAL_SEQUENCE_NUMBER)
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)

Review Comment:
   Fair point, I've just moved them.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,279 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+from uuid import UUID
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": UUID("aefee669-d568-4f9c-b732-3c0cfd3bc7b0"),
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},

Review Comment:
   In v1, this should be a list. Initially, this stored only the partition fields because we didn't support multiple partition specs. See https://iceberg.apache.org/spec/#table-metadata



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/conftest.py:
##########
@@ -301,3 +319,81 @@ def all_avro_types() -> Dict[str, Any]:
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
     return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture(scope="session")
+def simple_struct():
+    return StructType(
+        NestedField(1, "required_field", StringType(), True, "this is a doc"), NestedField(2, "optional_field", IntegerType())
+    )
+
+
+@pytest.fixture(scope="session")
+def simple_list():
+    return ListType(element_id=22, element=StringType(), element_required=True)
+
+
+@pytest.fixture(scope="session")
+def simple_map():
+    return MapType(key_id=19, key_type=StringType(), value_id=25, value_type=DoubleType(), value_required=False)
+
+
+class LocalOutputFile(OutputFile):
+    """An OutputFile implementation for local files (for test use only)"""
+
+    def __init__(self, location: str):
+
+        parsed_location = urlparse(location)  # Create a ParseResult from the uri
+        if parsed_location.scheme and parsed_location.scheme != "file":  # Validate that a uri is provided with a scheme of `file`
+            raise ValueError("LocalOutputFile location must have a scheme of `file`")
+        elif parsed_location.netloc:
+            raise ValueError(f"Network location is not allowed for LocalOutputFile: {parsed_location.netloc}")
+
+        super().__init__(location=location)
+        self._parsed_location = parsed_location
+
+    @property
+    def parsed_location(self) -> ParseResult:
+        """The parsed location
+        Returns:
+            ParseResult: The parsed results which has attributes `scheme`, `netloc`, `path`,
+            `params`, `query`, and `fragments`.
+        """
+        return self._parsed_location
+
+    def __len__(self):
+        return os.path.getsize(self.parsed_location.path)
+
+    def exists(self):
+        return os.path.exists(self.parsed_location.path)
+
+    def to_input_file(self):
+        return LocalInputFile(location=self.location)
+
+    def create(self, overwrite: bool = False) -> OutputStream:
+        output_file = open(self.parsed_location.path, "wb" if overwrite else "xb")
+        if not isinstance(output_file, OutputStream):
+            raise TypeError("Object returned from LocalOutputFile.create(...) does not match the OutputStream protocol.")
+        return output_file
+
+
+class LocalFileIO(FileIO):
+    """A FileIO implementation for local files (for test use only)"""
+
+    def new_input(self, location: str):
+        return LocalInputFile(location=location)
+
+    def new_output(self, location: str):
+        return LocalOutputFile(location=location)
+
+    def delete(self, location: Union[str, InputFile, OutputFile]):
+        location = location.location if isinstance(location, (InputFile, OutputFile)) else location
+        try:
+            os.remove(location)
+        except FileNotFoundError as e:
+            raise FileNotFoundError(f"Cannot delete file, does not exist: {location}") from e

Review Comment:
   I think for the test I think we should be more strict. If some file hasn't been written, we should know that. I think we can just remove the try-catch logic since the original error will be sufficient



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -48,107 +57,189 @@ class IcebergType(ABC, Singleton):
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                return DecimalType.parse(v)
+            elif v.startswith("fixed"):
+                return FixedType.parse(v)
+            else:
+                return PRIMITIVE_TYPES[v]
+        elif isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+        else:
+            return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()
+    _length: int = PrivateAttr()
+
+    @staticmethod
+    def parse(str_repr: str) -> "FixedType":
+        matches = FIXED_REGEX.search(str_repr)
+        if matches:
+            length = int(matches.group(1))
+            return FixedType(length)
+        raise ValueError(f"Could not parse {str_repr} into a FixedType")
+
+    def __init__(self, length: int):
+        super().__init__(__root__=f"fixed[{length}]")
+        self._length = length
 
     @property
-    def string_type(self) -> str:
-        return f"fixed[{self.length}]"
+    def length(self) -> int:
+        return self._length
+
+    def __repr__(self) -> str:
+        return f"FixedType(length={self._length})"
 
 
-@dataclass(frozen=True, eq=True)
 class DecimalType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> DecimalType(32, 3)
         DecimalType(precision=32, scale=3)
         >>> DecimalType(8, 3) == DecimalType(8, 3)
         True
     """
 
-    precision: int = field()
-    scale: int = field()
+    __root__: str = Field()
+
+    _precision: int = PrivateAttr()
+    _scale: int = PrivateAttr()
+
+    @staticmethod
+    def parse(str_repr: str) -> "DecimalType":
+        matches = DECIMAL_REGEX.search(str_repr)
+        if matches:
+            precision = int(matches.group(1))
+            scale = int(matches.group(2))
+            return DecimalType(precision, scale)
+        else:
+            raise ValueError(f"Could not parse {str_repr} into a DecimalType")
+
+    def __init__(self, precision: int, scale: int):
+        super().__init__(
+            __root__=f"decimal({precision}, {scale})",
+        )
+        self._precision = precision
+        self._scale = scale
+
+    @property
+    def precision(self) -> int:
+        return self._precision
 
     @property
-    def string_type(self) -> str:
-        return f"decimal({self.precision}, {self.scale})"
+    def scale(self) -> int:
+        return self._scale
+
+    def __repr__(self) -> str:
+        return f"DecimalType(precision={self._precision}, scale={self._scale})"
 
 
-@dataclass(frozen=True)
 class NestedField(IcebergType):
     """Represents a field of a struct, a map key, a map value, or a list element.
 
     This is where field IDs, names, docs, and nullability are tracked.
-

Review Comment:
   Accidental change?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):
+        # When we read a V1 format-version, we'll bump it to a V2 table right
+        # away by populating the required fields, and setting the version
+        data["format-version"] = 2
+        schema = data["schema"]
+        if "schemas" not in data:
+            if all([schema != other_schema for other_schema in data["schemas"]]):
+                data["schemas"].append(schema)
+        else:
+            data["schemas"] = [schema]
+        data["current-schema-id"] = schema["schema-id"]
+        data["last-sequence-number"] = _INITIAL_SEQUENCE_NUMBER
+        return TableMetadataV2(**data)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: dict = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):

Review Comment:
   For serializing to JSON, I think it makes sense to have separate classes. I don't see another way to suppress fields like `schema` or `last-sequence-number`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Looks like this is still 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/utils/iceberg_base_model.py:
##########
@@ -0,0 +1,29 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from pydantic import BaseModel
+
+
+class IcebergBaseModel(BaseModel):
+    class Config:
+        allow_population_by_field_name = True
+
+    def dict(self, exclude_none=True, **kwargs):
+        return super().dict(exclude_none=exclude_none, **kwargs)
+
+    def json(self, exclude_none=True, by_alias=True, **kwargs):
+        return super().json(exclude_none=exclude_none, by_alias=True, **kwargs)

Review Comment:
   I missed your previous comments, sorry about that!
   
   > Also, should the by_alias passed to the method be passed to the super().json(...) call? Right now it ignores it.
   
   Thanks! That was a copy-paste error.
   
   > I've added the following comment:
   
   ```
   This class extends the Pydantic BaseModel to set default values by overriding them.
   
   This is because we always want to set by_alias to True. In Python, the dash can't
   be used in variable names, and this is used throughout the Iceberg spec.
   
   The same goes for exclude_none, if a field is None we want to omit it from
   serialization, for example, the doc attribute on the NestedField object.
   Default non-null values will be serialized.
   
   This is recommended by Pydantic:
   https://pydantic-docs.helpmanual.io/usage/model_config/#change-behaviour-globally
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   @Fokko, is this supposed to be left in?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):
+        # Set some sensible defaults for V1, so we comply with the schema
+        # this is in pre=True, meaning that this will be done before validation
+        # we don't want to make them optional, since we do require them for V2
+        data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        data["default-spec-id"] = INITIAL_SPEC_ID
+        data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        data["default-sort-order-id"] = 0
+        return data
+
+    @root_validator()
+    def migrate_schema(cls, data: Dict[str, Any]):
+        # Migrate schemas
+        schema = data["schema_"]
+        schemas = data["schemas"]
+        if all([schema != other_schema for other_schema in schemas]):
+            data["schemas"].append(schema)
+        data["current_schema_id"] = schema.schema_id

Review Comment:
   Makes sense. I've made it conditionally and added a `ValidationError`. It feels a bit weird that both of them can be set, and they serve the same purpose. I've added a check as well.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,363 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+class ValidationError(Exception):
+    ...

Review Comment:
   Can you move this to `exceptions.py`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Okay, I see the calls to `parse`. That brings us back to my original question, which is why are we doing parsing in a validate method? Shouldn't this validate the input rather than returning the deserialized value?
   
   I would expect this to do something like the schema validator, which would change `"fixed[12]"` into `{"_root_": "fixed", "length": 12} or something.



##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Okay, I see the calls to `parse`. That brings us back to my original question, which is why are we doing parsing in a validate method? Shouldn't this validate the input rather than returning the deserialized value?
   
   I would expect this to do something like the schema validator, which would change `"fixed[12]"` into `{"__root_"_: "fixed", "length": 12}` or something.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_schema_id(cls, data: Dict[str, Any]):

Review Comment:
   It used to do, but it grew a bit out of control. I renamed it to `set_v2_compatible_defaults`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,259 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from enum import Enum
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+
+
+class SnapshotRefType(str, Enum):
+    branch = "branch"
+    tag = "tag"
+
+
+class SnapshotRef(IcebergBaseModel):
+    snapshot_id: int = Field(alias="snapshot-id")
+    snapshot_ref_type: SnapshotRefType = Field(alias="type")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    max_ref_age_ms: int = Field(alias="max-ref-age-ms")
+
+
+class TableMetadataCommonFields(IcebergBaseModel):

Review Comment:
   Yes, just added that from the logic in:  https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/TableMetadataParser.java#L309
   
   Came up with:
   
   ```python
       @root_validator(skip_on_failure=True)
       def construct_refs(cls, data: Dict[str, Any]):
           # This is going to be much nicer as soon as sort-order is an actual pydantic object
           if not data.get("refs"):
               if current_snapshot_id := data.get("current_snapshot_id"):
                   if current_snapshot_id != -1:
                       data["refs"] = {
                           MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
                       }
           return data
   ```
   
   



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,168 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+from uuid import UUID
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    def __new__(cls, *_, **data):
+        # When we read a V1 format-version, we'll bump it to a V2 table right
+        # away by populating the required fields, and setting the version

Review Comment:
   We shouldn't actually change the format of the table. In the library, we always want to work with v2 structures. But we still need to maintain compatibility with older readers and writers, so upgrading a table from v1 to v2 must be done by users.
   
   What we want to do here is deserialize to a common representation so that the rest of the library doesn't need to worry about v1/v2. Then when writing metadata, this should serialize according to the version. For example, if the table is v1, then write `schema` in addition to `schemas` and `current-schema-id`. If the table version is v2, then omit `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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/serializers.py:
##########
@@ -0,0 +1,75 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import codecs
+import json
+from typing import Union
+
+from iceberg.io.base import InputFile, InputStream, OutputFile
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+
+
+class FromByteStream:
+    """A collection of methods that deserialize dictionaries into Iceberg objects"""
+
+    @staticmethod
+    def table_metadata(byte_stream: InputStream, encoding: str = "utf-8") -> TableMetadata:
+        """Instantiate a TableMetadata object from a byte stream
+
+        Args:
+            byte_stream: A file-like byte stream object
+            encoding (default "utf-8"): The byte encoder to use for the reader

Review Comment:
   I don't think that we allow any other encoding, but it should be fine to leave this in.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -48,107 +57,189 @@ class IcebergType(ABC, Singleton):
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                return DecimalType.parse(v)
+            elif v.startswith("fixed"):
+                return FixedType.parse(v)
+            else:
+                return PRIMITIVE_TYPES[v]
+        elif isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+        else:
+            return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()
+    _length: int = PrivateAttr()
+
+    @staticmethod
+    def parse(str_repr: str) -> "FixedType":
+        matches = FIXED_REGEX.search(str_repr)
+        if matches:
+            length = int(matches.group(1))
+            return FixedType(length)
+        raise ValueError(f"Could not parse {str_repr} into a FixedType")
+
+    def __init__(self, length: int):
+        super().__init__(__root__=f"fixed[{length}]")
+        self._length = length
 
     @property
-    def string_type(self) -> str:
-        return f"fixed[{self.length}]"
+    def length(self) -> int:
+        return self._length
+
+    def __repr__(self) -> str:
+        return f"FixedType(length={self._length})"
 
 
-@dataclass(frozen=True, eq=True)
 class DecimalType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> DecimalType(32, 3)
         DecimalType(precision=32, scale=3)
         >>> DecimalType(8, 3) == DecimalType(8, 3)
         True
     """
 
-    precision: int = field()
-    scale: int = field()
+    __root__: str = Field()
+
+    _precision: int = PrivateAttr()
+    _scale: int = PrivateAttr()
+
+    @staticmethod
+    def parse(str_repr: str) -> "DecimalType":
+        matches = DECIMAL_REGEX.search(str_repr)
+        if matches:
+            precision = int(matches.group(1))
+            scale = int(matches.group(2))
+            return DecimalType(precision, scale)
+        else:
+            raise ValueError(f"Could not parse {str_repr} into a DecimalType")
+
+    def __init__(self, precision: int, scale: int):
+        super().__init__(
+            __root__=f"decimal({precision}, {scale})",
+        )
+        self._precision = precision
+        self._scale = scale
+
+    @property
+    def precision(self) -> int:
+        return self._precision
 
     @property
-    def string_type(self) -> str:
-        return f"decimal({self.precision}, {self.scale})"
+    def scale(self) -> int:
+        return self._scale
+
+    def __repr__(self) -> str:
+        return f"DecimalType(precision={self._precision}, scale={self._scale})"
 
 
-@dataclass(frozen=True)
 class NestedField(IcebergType):
     """Represents a field of a struct, a map key, a map value, or a list element.
 
     This is where field IDs, names, docs, and nullability are tracked.
-

Review Comment:
   I blame my new keyboard, 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,178 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Union,
+)
+from uuid import UUID
+
+from pydantic import Field, root_validator
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: UUID = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    @root_validator(pre=True)
+    def populate_schemas(cls, data: Dict[str, Any]):
+        # When we read a V1 format-version, we'll make sure to populate the fields

Review Comment:
   Ah, that one slipped in. 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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]
+        else:
+            check_partition_specs(data["partition_specs"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def set_sort_orders(cls, data: Dict[str, Any]):
+        """Sets the sort_orders if not provided
+
+        For V1 sort_orders is optional, and if they aren't set, we'll set them
+        in this validator.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the sort_orders set, if not provided
+        """
+        # This is going to be much nicer as soon as sort-order is an actual pydantic object
+        # Probably we'll just create a UNSORTED_ORDER constant then
+        if not data.get("sort_orders"):
+            data["sort_orders"] = [{"order_id": 0, "fields": []}]
+            data["default_sort_order_id"] = 0
+        else:
+            check_sort_orders(data["sort_orders"])
+        return data
+
+    def to_v2(self) -> "TableMetadataV2":
+        metadata = copy(self.dict())
+        metadata["format_version"] = 2
+        return TableMetadataV2(**metadata)
+
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    table_uuid: Optional[UUID] = Field(alias="table-uuid")

Review Comment:
   Done! Keep in mind that we keep it nullable in Java: https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/TableMetadataParser.java#L317



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)
+    """ID of the current table snapshot."""
+
+    snapshots: list = Field(default_factory=list)
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: List[Dict[str, Any]] = Field(alias="snapshot-log", default_factory=list)
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: List[Dict[str, Any]] = Field(alias="metadata-log", default_factory=list)
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: List[Dict[str, Any]] = Field(alias="sort-orders", default_factory=list)
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+    refs: Dict[str, SnapshotRef] = Field(default_factory=dict)
+    """A map of snapshot references.
+    The map keys are the unique snapshot reference names in the table,
+    and the map values are snapshot reference objects.
+    There is always a main branch reference pointing to the
+    current-snapshot-id even if the refs map is null."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    """Represents version 1 of the Table Metadata
+
+    More information about the specification:
+    https://iceberg.apache.org/spec/#version-1-analytic-data-tables
+    """
+
+    # When we read a V1 format-version, we'll make sure to populate the fields
+    # for V2 as well. This makes it easier downstream because we can just
+    # assume that everything is a TableMetadataV2.
+    # When writing, we should stick to the same version that it was,
+    # because bumping the version should be an explicit operation that is up
+    # to the owner of the table.
+
+    @root_validator(pre=True)
+    def set_v2_compatible_defaults(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Sets default values to be compatible with the format v2
+
+        Set some sensible defaults for V1, so we comply with the schema
+        this is in pre=True, meaning that this will be done before validation.
+        We don't want to make the fields optional, since they are required for V2
+
+        Args:
+            data: The raw arguments when initializing a V1 TableMetadata
+
+        Returns:
+            The TableMetadata with the defaults applied
+        """
+        if "schema-id" not in data["schema"]:
+            data["schema"]["schema-id"] = DEFAULT_SCHEMA_ID
+        if "default-spec-id" not in data:
+            data["default-spec-id"] = INITIAL_SPEC_ID
+        if "last-partition-id" not in data:
+            data["last-partition-id"] = max(spec["field-id"] for spec in data["partition-spec"])
+        if "default-sort-order-id" not in data:
+            data["default-sort-order-id"] = DEFAULT_SORT_ORDER_UNSORTED
+        if "table-uuid" not in data:
+            data["table-uuid"] = uuid4()
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_schema(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the schema into schemas
+
+        For V1 schemas is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the schemas when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the schemas set, if not provided
+        """
+        if not data.get("schemas"):
+            schema = data["schema_"]
+            data["schemas"] = [schema]
+            if "current_schema_id" not in data:
+                data["current_schema_id"] = schema.schema_id
+        else:
+            check_schemas(data["schemas"])
+        return data
+
+    @root_validator(skip_on_failure=True)
+    def construct_partition_specs(cls, data: Dict[str, Any]) -> Dict[str, Any]:
+        """Converts the partition_spec into partition_specs
+
+        For V1 partition_specs is optional, and if they aren't set, we'll set them
+        in this validator. This was we can always use the partition_specs when reading
+        table metadata, and we don't have to worry if it is a v1 or v2 format.
+
+        Args:
+            data: The raw data after validation, meaning that the aliases are applied
+
+        Returns:
+            The TableMetadata with the partition_specs set, if not provided
+        """
+        # This is going to be much nicer as soon as partition-spec is also migrated to pydantic
+        if not data.get("partition_specs"):
+            fields = data["partition_spec"]
+            data["partition_specs"] = [{"spec-id": INITIAL_SPEC_ID, "fields": fields}]

Review Comment:
   Hmm, this is actually a remained of some refactoring along the way. Since this is not a pre-validator, the field already needs to be set. The `default_schema_id` has a default. I've also set a default for the partition-spec, so everything is consistent again.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,360 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from copy import copy
+from typing import (
+    Any,
+    Dict,
+    List,
+    Literal,
+    Optional,
+    Union,
+)
+from uuid import UUID, uuid4
+
+from pydantic import Field, root_validator
+
+from iceberg.exceptions import ValidationError
+from iceberg.schema import Schema
+from iceberg.table.refs import MAIN_BRANCH, SnapshotRef, SnapshotRefType
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+_INITIAL_SEQUENCE_NUMBER = 0
+INITIAL_SPEC_ID = 0
+DEFAULT_SCHEMA_ID = 0
+DEFAULT_SORT_ORDER_UNSORTED = 0
+
+
+def check_schemas(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the current-schema-id is actually present in schemas"""
+    current_schema_id = values["current_schema_id"]
+
+    for schema in values["schemas"]:
+        if schema.schema_id == current_schema_id:
+            return values
+
+    raise ValidationError(f"current-schema-id {current_schema_id} can't be found in the schemas")
+
+
+def check_partition_specs(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default-spec-id is present in partition-specs"""
+    default_spec_id = values["default_spec_id"]
+
+    for spec in values["partition_specs"]:
+        if spec["spec-id"] == default_spec_id:
+            return values
+
+    raise ValidationError(f"default-spec-id {default_spec_id} can't be found")
+
+
+def check_sort_orders(values: Dict[str, Any]) -> Dict[str, Any]:
+    """Validator to check if the default_sort_order_id is present in sort-orders"""
+    default_sort_order_id = values["default_sort_order_id"]
+
+    # 0 == unsorted
+    if default_sort_order_id != 0:
+        for sort in values["sort_orders"]:
+            if sort["order-id"] == default_sort_order_id:
+                return values
+
+        raise ValidationError(f"default-sort-order-id {default_sort_order_id} can't be found")
+    return values
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    @root_validator(skip_on_failure=True)
+    def construct_refs(cls, data: Dict[str, Any]):
+        # This is going to be much nicer as soon as refs is an actual pydantic object
+        if not data.get("refs"):
+            if current_snapshot_id := data.get("current_snapshot_id"):
+                if current_snapshot_id != -1:
+                    data["refs"] = {
+                        MAIN_BRANCH: SnapshotRef(snapshot_id=current_snapshot_id, snapshot_ref_type=SnapshotRefType.branch)
+                    }
+        return data
+
+    location: str = Field()
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure fields are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field(default_factory=list)
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id", default=0)
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs", default_factory=list)
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: Dict[str, str] = Field(default_factory=dict)
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: Optional[int] = Field(alias="current-snapshot-id", default=-1)

Review Comment:
   Non-blocker: we use -1 in the format, but if I could do it over, I would probably avoid it and use `null` / `None` instead. You may want to consider converting in a pre-validation from -1 to clean this up.



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

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

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


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


[GitHub] [iceberg] rdblue merged pull request #5011: Python: Use Pydantic for (de)serialization

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


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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Good one, I've moved this into the Type itself.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   schema is a reserved keyword in Pydantic:
   
   print(Model.schema())
   ```python
   >>> table_metadata.schema()
   {
   	'title': 'TableMetadataV2',
   	'description': 'Metadata for an Iceberg table as specified in the Apache Iceberg\nspec (https://iceberg.apache.org/spec/#iceberg-table-spec)',
   	'type': 'object',
   	'properties': {
   		'table-uuid': {
   			'title': 'Table-Uuid',
   			'type': 'string',
   			'format': 'uuid'
   		},
   		'location': {
   			'title': 'Location',
   			'type': 'string'
   		},
   		'last-updated-ms': {
   			'title': 'Last-Updated-Ms',
   			'type': 'integer'
   		},
   		'last-column-id': {
   			'title': 'Last-Column-Id',
   			'type': 'integer'
   		},
   		'schemas': {
   			'title': 'Schemas',
   			'type': 'array',
   			'items': {
   				'$ref': '#/definitions/Schema'
   			}
   		},
   		'current-schema-id': {
   			'title': 'Current-Schema-Id',
   			'type': 'integer'
   		},
   		'partition-specs': {
   			'title': 'Partition-Specs',
   			'type': 'array',
   			'items': {}
   		},
   		'default-spec-id': {
   			'title': 'Default-Spec-Id',
   			'type': 'integer'
   		},
   		'last-partition-id': {
   			'title': 'Last-Partition-Id',
   			'type': 'integer'
   		},
   		'properties': {
   			'title': 'Properties',
   			'type': 'object'
   		},
   		'current-snapshot-id': {
   			'title': 'Current-Snapshot-Id',
   			'type': 'integer'
   		},
   		'snapshots': {
   			'title': 'Snapshots',
   			'type': 'array',
   			'items': {}
   		},
   		'snapshot-log': {
   			'title': 'Snapshot-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'metadata-log': {
   			'title': 'Metadata-Log',
   			'type': 'array',
   			'items': {}
   		},
   		'sort-orders': {
   			'title': 'Sort-Orders',
   			'type': 'array',
   			'items': {}
   		},
   		'default-sort-order-id': {
   			'title': 'Default-Sort-Order-Id',
   			'type': 'integer'
   		},
   		'format-version': {
   			'title': 'Format-Version',
   			'enum': [2],
   			'type': 'integer'
   		},
   		'last-sequence-number': {
   			'title': 'Last-Sequence-Number',
   			'type': 'integer'
   		}
   	},
   	'required': ['table-uuid', 'location', 'last-updated-ms', 'last-column-id', 'schemas', 'current-schema-id', 'partition-specs', 'default-spec-id', 'last-partition-id', 'properties', 'current-snapshot-id', 'snapshots', 'snapshot-log', 'metadata-log', 'sort-orders', 'default-sort-order-id', 'format-version', 'last-sequence-number'],
   	'definitions': {
   		'IcebergType': {
   			'title': 'IcebergType',
   			'description': "Base type for all Iceberg Types\n\nExample:\n    >>> str(IcebergType())\n    'IcebergType()'\n    >>> repr(IcebergType())\n    'IcebergType()'",
   			'type': 'object',
   			'properties': {}
   		},
   		'NestedField': {
   			'title': 'NestedField',
   			'description': 'Represents a field of a struct, a map key, a map value, or a list element.\n\nThis is where field IDs, names, docs, and nullability are tracked.\nExample:\n    >>> str(NestedField(\n    ...     field_id=1,\n    ...     name=\'foo\',\n    ...     field_type=FixedType(22),\n    ...     required=False,\n    ... ))\n    \'1: foo: optional fixed[22]\'\n    >>> str(NestedField(\n    ...     field_id=2,\n    ...     name=\'bar\',\n    ...     field_type=LongType(),\n    ...     is_optional=False,\n    ...     doc="Just a long"\n    ... ))\n    \'2: bar: required long (Just a long)\'',
   			'type': 'object',
   			'properties': {
   				'id': {
   					'title': 'Id',
   					'type': 'integer'
   				},
   				'name': {
   					'title': 'Name',
   					'type': 'string'
   				},
   				'type': {
   					'$ref': '#/definitions/IcebergType'
   				},
   				'required': {
   					'title': 'Required',
   					'default': True,
   					'type': 'boolean'
   				},
   				'doc': {
   					'title': 'Doc',
   					'type': 'string'
   				}
   			},
   			'required': ['id', 'name', 'type']
   		},
   		'Schema': {
   			'title': 'Schema',
   			'description': 'A table Schema\n\nExample:\n    >>> from iceberg import schema\n    >>> from iceberg import types',
   			'type': 'object',
   			'properties': {
   				'fields': {
   					'title': 'Fields',
   					'type': 'array',
   					'items': {
   						'$ref': '#/definitions/NestedField'
   					}
   				},
   				'schema-id': {
   					'title': 'Schema-Id',
   					'type': 'integer'
   				},
   				'identifier-field-ids': {
   					'title': 'Identifier-Field-Ids',
   					'type': 'array',
   					'items': {
   						'type': 'integer'
   					}
   				}
   			},
   			'required': ['schema-id']
   		}
   	}
   }
   """
   
   Therefore we need to set the alias to properly map it.
   
   I've added code to properly map it. In v1 the schema's are optional, if it is set, I'll check if the schema is in there, otherwise, I'll add 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] Fokko commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):

Review Comment:
   Good one, I've moved this into the Types itself.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/types.py:
##########
@@ -59,134 +62,176 @@ class IcebergType:
         'IcebergType()'
     """
 
-    @property
-    def string_type(self) -> str:
-        return self.__repr__()
-
-    def __str__(self) -> str:
-        return self.string_type
+    @classmethod
+    def __get_validators__(cls):
+        # 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):
+        # When Pydantic is unable to determine the subtype
+        # In this case we'll help pydantic a bit by parsing the
+        # primitive type ourselves, or pointing it at the correct
+        # complex type by looking at the type field
+
+        if isinstance(v, str):
+            if v.startswith("decimal"):
+                m = re.search(r"decimal\((\d+),\s*(\d+)\)", v)
+                precision = int(m.group(1))
+                scale = int(m.group(2))
+                return DecimalType(precision, scale)
+            elif v.startswith("fixed"):
+                m = re.search(r"fixed\[(\d+)\]", v)
+                length = int(m.group(1))
+                return FixedType(length)
+            else:
+                return PRIMITIVE_TYPES[v]
+
+        if isinstance(v, dict):
+            if v.get("type") == "struct":
+                return StructType(**v)
+            elif v.get("type") == "list":
+                return ListType(**v)
+            elif v.get("type") == "map":
+                return MapType(**v)
+            else:
+                return NestedField(**v)
+
+        return v
 
     @property
     def is_primitive(self) -> bool:
         return isinstance(self, PrimitiveType)
 
 
-@dataclass(frozen=True, eq=True)
 class PrimitiveType(IcebergType):
-    """Base class for all Iceberg Primitive Types
+    """Base class for all Iceberg Primitive Types"""
 
-    Example:
-        >>> str(PrimitiveType())
-        'PrimitiveType()'
-    """
+    __root__: str = Field()
+
+    def __repr__(self) -> str:
+        return f"{type(self).__name__}()"
+
+    def __str__(self) -> str:
+        return self.__root__
 
 
-@dataclass(frozen=True)
 class FixedType(PrimitiveType):
     """A fixed data type in Iceberg.
-
     Example:
         >>> FixedType(8)
         FixedType(length=8)
         >>> FixedType(8) == FixedType(8)
         True
+        >>> FixedType(19) == FixedType(25)
+        False
     """
 
-    length: int = field()
+    __root__: str = Field()

Review Comment:
   What is `__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] rdblue commented on a diff in pull request #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")
+    """The table’s current schema. (Deprecated: use schemas and
+    current-schema-id instead)"""
+
+    partition_spec: dict = Field(alias="partition-spec")
+    """The table’s current partition spec, stored as only fields.
+    Note that this is used by writers to partition data, but is
+    not used when reading because reads use the specs stored in
+    manifest files. (Deprecated: use partition-specs and default-spec-id
+    instead)"""
+
+
+class TableMetadataV2(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[2] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    last_sequence_number: int = Field(alias="last-sequence-number")
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+
+class TableMetadata:

Review Comment:
   Why is this distinct from `TableMetadataCommenFields`? It seems like all of the table metadata classes should be `TableMetadata`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/src/iceberg/table/metadata.py:
##########
@@ -0,0 +1,151 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from typing import List, Literal, Union
+
+from pydantic import Field
+
+from iceberg.schema import Schema
+from iceberg.utils.iceberg_base_model import IcebergBaseModel
+
+
+class TableMetadataCommonFields(IcebergBaseModel):
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    table_uuid: str = Field(alias="table-uuid")
+    """A UUID that identifies the table, generated when the table is created.
+    Implementations must throw an exception if a table’s UUID does not match
+    the expected UUID after refreshing metadata."""
+
+    location: str
+    """The table’s base location. This is used by writers to determine where
+    to store data files, manifest files, and table metadata files."""
+
+    last_updated_ms: int = Field(alias="last-updated-ms")
+    """Timestamp in milliseconds from the unix epoch when the table
+    was last updated. Each table metadata file should update this
+    field just before writing."""
+
+    last_column_id: int = Field(alias="last-column-id")
+    """An integer; the highest assigned column ID for the table.
+    This is used to ensure columns are always assigned an unused ID
+    when evolving schemas."""
+
+    schemas: List[Schema] = Field()
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int = Field(alias="current-schema-id")
+    """ID of the table’s current schema."""
+
+    partition_specs: list = Field(alias="partition-specs")
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int = Field(alias="default-spec-id")
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int = Field(alias="last-partition-id")
+    """An integer; the highest assigned partition field ID across all
+    partition specs for the table. This is used to ensure partition fields
+    are always assigned an unused ID when evolving specs."""
+
+    properties: dict
+    """	A string to string map of table properties. This is used to
+    control settings that affect reading and writing and is not intended
+    to be used for arbitrary metadata. For example, commit.retry.num-retries
+    is used to control the number of commit retries."""
+
+    current_snapshot_id: int = Field(alias="current-snapshot-id")
+    """ID of the current table snapshot."""
+
+    snapshots: list
+    """A list of valid snapshots. Valid snapshots are snapshots for which
+    all data files exist in the file system. A data file must not be
+    deleted from the file system until the last snapshot in which it was
+    listed is garbage collected."""
+
+    snapshot_log: list = Field(alias="snapshot-log")
+    """A list (optional) of timestamp and snapshot ID pairs that encodes
+    changes to the current snapshot for the table. Each time the
+    current-snapshot-id is changed, a new entry should be added with the
+    last-updated-ms and the new current-snapshot-id. When snapshots are
+    expired from the list of valid snapshots, all entries before a snapshot
+    that has expired should be removed."""
+
+    metadata_log: list = Field(alias="metadata-log")
+    """A list (optional) of timestamp and metadata file location pairs that
+    encodes changes to the previous metadata files for the table. Each time
+    a new metadata file is created, a new entry of the previous metadata
+    file location should be added to the list. Tables can be configured to
+    remove oldest metadata log entries and keep a fixed-size log of the most
+    recent entries after a commit."""
+
+    sort_orders: list = Field(alias="sort-orders")
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int = Field(alias="default-sort-order-id")
+    """Default sort order id of the table. Note that this could be used by
+    writers, but is not used when reading because reads use the specs stored
+     in manifest files."""
+
+
+class TableMetadataV1(TableMetadataCommonFields, IcebergBaseModel):
+    format_version: Literal[1] = Field(alias="format-version")
+    """An integer version number for the format. Currently, this can be 1 or 2
+    based on the spec. Implementations must throw an exception if a table’s
+    version is higher than the supported version."""
+
+    schema_: Schema = Field(alias="schema")

Review Comment:
   Does this correctly translate?
   
   When reading, we want to accept a `schema`, but convert it into a `schemas` list and `current-schema-id`. In v1, we want to write both `schema` and `schemas` / `current-schema-id` because adding that metadata is compatible.
   
   Is there a way to hook into parsing to fix up the way things were read? So if this reads `schema` but not `schemas` then it would automatically convert to the newer metadata.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/table/test_metadata.py:
##########
@@ -0,0 +1,328 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import io
+import json
+import os
+import tempfile
+
+import pytest
+
+from iceberg.schema import Schema
+from iceberg.serializers import FromByteStream, FromInputFile, ToOutputFile
+from iceberg.table.metadata import TableMetadata, TableMetadataV1, TableMetadataV2
+from iceberg.types import NestedField, StringType
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": 1,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schema": {
+        "schema-id": 0,
+        "fields": [
+            {"id": 1, "name": "foo", "required": True, "type": "string"},
+            {"id": 2, "name": "bar", "required": True, "type": "string"},
+            {"id": 3, "name": "baz", "required": True, "type": "string"},
+            {"id": 4, "name": "qux", "required": True, "type": "string"},
+        ],
+        "identifier-field-ids": [],
+    },
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        },
+    ],
+    "current-schema-id": 0,
+    "partition-spec": {},
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet"},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+EXAMPLE_TABLE_METADATA_V2 = {
+    "format-version": 2,
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "last-sequence-number": 1,
+    "schemas": [
+        {
+            "schema-id": 0,
+            "fields": [
+                {"id": 1, "name": "foo", "required": True, "type": "string"},
+                {"id": 2, "name": "bar", "required": True, "type": "string"},
+                {"id": 3, "name": "baz", "required": True, "type": "string"},
+                {"id": 4, "name": "qux", "required": True, "type": "string"},
+            ],
+            "identifier-field-ids": [],
+        }
+    ],
+    "current-schema-id": 0,
+    "default-spec-id": 0,
+    "partition-specs": [{"spec-id": 0, "fields": []}],
+    "last-partition-id": 999,
+    "default-sort-order-id": 0,
+    "sort-orders": [{"order-id": 0, "fields": []}],
+    "properties": {"owner": "root", "write.format.default": "parquet", "read.split.target.size": 134217728},
+    "current-snapshot-id": 7681945274687743099,
+    "snapshots": [
+        {
+            "snapshot-id": 7681945274687743099,
+            "timestamp-ms": 1637943123188,
+            "summary": {
+                "operation": "append",
+                "added-data-files": "6",
+                "added-records": "237993",
+                "added-files-size": "3386901",
+                "changed-partition-count": "1",
+                "total-records": "237993",
+                "total-files-size": "3386901",
+                "total-data-files": "6",
+                "total-delete-files": "0",
+                "total-position-deletes": "0",
+                "total-equality-deletes": "0",
+            },
+            "manifest-list": "s3://foo/bar/baz/snap-2874264644797652805-1-9cb3c3cf-5a04-40c1-bdd9-d8d7e38cd8e3.avro",
+            "schema-id": 0,
+        },
+    ],
+    "snapshot-log": [
+        {"timestamp-ms": 1637943123188, "snapshot-id": 7681945274687743099},
+    ],
+    "metadata-log": [
+        {
+            "timestamp-ms": 1637943123331,
+            "metadata-file": "3://foo/bar/baz/00000-907830f8-1a92-4944-965a-ff82c890e912.metadata.json",
+        }
+    ],
+}
+
+
+@pytest.mark.parametrize(
+    "metadata",
+    [
+        EXAMPLE_TABLE_METADATA_V1,
+        EXAMPLE_TABLE_METADATA_V2,
+    ],
+)
+def test_from_dict(metadata: dict):
+    """Test initialization of a TableMetadata instance from a dictionary"""
+    TableMetadata.parse_obj(metadata)

Review Comment:
   Is this just testing that an object is produced? Shouldn't it validate the contents?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5011: Python: Use Pydantic for (de)serialization

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


##########
python/tests/test_types.py:
##########
@@ -204,3 +206,388 @@ def test_non_parameterized_type_equality(input_index, input_type, check_index, c
         assert input_type() == check_type()
     else:
         assert input_type() != check_type()
+
+
+# Examples based on https://iceberg.apache.org/spec/#appendix-c-json-serialization
+
+
+class TestType(IcebergBaseModel):
+    __root__: IcebergType
+
+
+def test_serialization_boolean():
+    assert BooleanType().json() == '"boolean"'
+
+
+def test_deserialization_boolean():
+    assert TestType.parse_raw('"boolean"') == BooleanType()
+
+
+def test_str_boolean():
+    assert str(BooleanType()) == "boolean"
+
+
+def test_repr_boolean():
+    assert repr(BooleanType()) == "BooleanType()"
+
+
+def test_serialization_int():
+    assert IntegerType().json() == '"int"'
+
+
+def test_deserialization_int():
+    assert TestType.parse_raw('"int"') == IntegerType()
+
+
+def test_str_int():
+    assert str(IntegerType()) == "int"
+
+
+def test_repr_int():
+    assert repr(IntegerType()) == "IntegerType()"
+
+
+def test_serialization_long():
+    assert LongType().json() == '"long"'
+
+
+def test_deserialization_long():
+    assert TestType.parse_raw('"long"') == LongType()
+
+
+def test_str_long():
+    assert str(LongType()) == "long"
+
+
+def test_repr_long():
+    assert repr(LongType()) == "LongType()"
+
+
+def test_serialization_float():
+    assert FloatType().json() == '"float"'
+
+
+def test_deserialization_float():
+    assert TestType.parse_raw('"float"') == FloatType()
+
+
+def test_str_float():
+    assert str(FloatType()) == "float"
+
+
+def test_repr_float():
+    assert repr(FloatType()) == "FloatType()"
+
+
+def test_serialization_double():
+    assert DoubleType().json() == '"double"'
+
+
+def test_deserialization_double():
+    assert TestType.parse_raw('"double"') == DoubleType()
+
+
+def test_str_double():
+    assert str(DoubleType()) == "double"
+
+
+def test_repr_double():
+    assert repr(DoubleType()) == "DoubleType()"
+
+
+def test_serialization_date():
+    assert DateType().json() == '"date"'
+
+
+def test_deserialization_date():
+    assert TestType.parse_raw('"date"') == DateType()
+
+
+def test_str_date():
+    assert str(DateType()) == "date"
+
+
+def test_repr_date():
+    assert repr(DateType()) == "DateType()"
+
+
+def test_serialization_time():
+    assert TimeType().json() == '"time"'
+
+
+def test_deserialization_time():
+    assert TestType.parse_raw('"time"') == TimeType()
+
+
+def test_str_time():
+    assert str(TimeType()) == "time"
+
+
+def test_repr_time():
+    assert repr(TimeType()) == "TimeType()"
+
+
+def test_serialization_timestamp():
+    assert TimestampType().json() == '"timestamp"'
+
+
+def test_deserialization_timestamp():
+    assert TestType.parse_raw('"timestamp"') == TimestampType()
+
+
+def test_str_timestamp():
+    assert str(TimestampType()) == "timestamp"
+
+
+def test_repr_timestamp():
+    assert repr(TimestampType()) == "TimestampType()"
+
+
+def test_serialization_timestamptz():
+    assert TimestamptzType().json() == '"timestamptz"'
+
+
+def test_deserialization_timestamptz():
+    assert TestType.parse_raw('"timestamptz"') == TimestamptzType()
+
+
+def test_str_timestamptz():
+    assert str(TimestamptzType()) == "timestamptz"
+
+
+def test_repr_timestamptz():
+    assert repr(TimestamptzType()) == "TimestamptzType()"
+
+
+def test_serialization_string():
+    assert StringType().json() == '"string"'
+
+
+def test_deserialization_string():
+    assert TestType.parse_raw('"string"') == StringType()
+
+
+def test_str_string():
+    assert str(StringType()) == "string"
+
+
+def test_repr_string():
+    assert repr(StringType()) == "StringType()"
+
+
+def test_serialization_uuid():
+    assert UUIDType().json() == '"uuid"'
+
+
+def test_deserialization_uuid():
+    assert TestType.parse_raw('"uuid"') == UUIDType()
+
+
+def test_str_uuid():
+    assert str(UUIDType()) == "uuid"
+
+
+def test_repr_uuid():
+    assert repr(UUIDType()) == "UUIDType()"
+
+
+def test_serialization_fixed():
+    assert FixedType(22).json() == '"fixed[22]"'
+
+
+def test_deserialization_fixed():
+    fixed = TestType.parse_raw('"fixed[22]"')
+    assert fixed == FixedType(22)
+
+    inner = fixed.__root__
+    assert isinstance(inner, FixedType)
+    assert inner.length == 22
+
+
+def test_str_fixed():
+    assert str(FixedType(22)) == "fixed[22]"
+
+
+def test_repr_fixed():
+    assert repr(FixedType(22)) == "FixedType(length=22)"
+
+
+def test_serialization_binary():
+    assert BinaryType().json() == '"binary"'
+
+
+def test_deserialization_binary():
+    assert TestType.parse_raw('"binary"') == BinaryType()
+
+
+def test_str_binary():
+    assert str(BinaryType()) == "binary"
+
+
+def test_repr_binary():
+    assert repr(BinaryType()) == "BinaryType()"
+
+
+def test_serialization_decimal():
+    assert DecimalType(19, 25).json() == '"decimal(19, 25)"'

Review Comment:
   Somewhat unrelated: Isn't this type invalid because the scale is greater than the precision? (I think we check that in Java)



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

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

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


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