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 2021/12/06 18:17:12 UTC

[GitHub] [iceberg] samredai opened a new pull request #3677: Adding TableMetadata object from dict, bytestream, and s3 sources

samredai opened a new pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677


   This is the table metadata portion of #3227. This is complete but wanted it to be visible to get feedback on the direction. The equivalent logic in the legacy implementation is stored in [table_metadata.py](https://github.com/apache/iceberg/blob/master/python_legacy/iceberg/core/table_metadata.py) and [table_metadata_parser.py](https://github.com/apache/iceberg/blob/master/python_legacy/iceberg/core/table_metadata_parser.py).
   
   The idea here is that, instead of including very explicit parsing and validation logic for table metadata files, we can rely on the standard library in conjunction with [jsonschema](https://json-schema.org/) tooling to accomplish both. The `TABLE_METADATA_V2_SCHEMA` jsonschema definition found in metadata.py is an example of how this can be done (still needs to be tuned to the spec exactly). The `TableMetadata` class itself naively parses a given json object and includes a `validate()` method that validates against the defined jsonschema. (`validate()` simply calls the `validate_v1()` or `validate_v2()` static method.) Table metadata values can then be retrieved using simple dot notation and can be updating as well.
   ```py
   table_metadata = TableMetadata.from_s3("s3://foo/bar/baz.metadata.json", version=2)
   print(table_metadata.properties.read_split_target_size) # 134217728
   table_metadata.properties.read_split_target_size = 268435456
   print(table_metadata.properties.read_split_target_size) # 268435456
   ```
   
   Once tweaked, the jsonschema definition should prove re-usable since jsonschema parsers exist in almost every language. It may also be valuable to include a blessed jsonschema definition in the Iceberg docs.
   
   A proposal for editing table metadata is to use a collection of functions that update value(s), validate that the schema is still valid, and return the updating `TableMetadata` instance:
   ```py
   from copy import deepcopy
   import time
   
   def rollback(table_metadata: TableMetadata, snapshot_id: int):
     new_table_metadata = deepcopy(table_metadata)
     now_millis = int(time.time() * 1000)
     new_table_metadata.snapshot_log.append({"timestamp_millis": now_millis, "snapshot_id":  snapshot_id})
     new_table_metadata. current_snapshot_id = snapshot_id
     return new_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] rdblue commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763314154



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [

Review comment:
       Ah, I see that this is for v1. It doesn't quite align with v1, and there are some constructs that are optional. v1 requires `schema`, but allows setting `schemas` and `current-schema-id`. We'd probably need to keep these up to date.




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763408253



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances

Review comment:
       The class name does make this sound odd but the logic is essentially a DFS through the metadata to assign everything as class attributes. If a value in the json is an object (like properties), it instantiates that as a sort of "partial" `TableMetadata` instance and then starts a new traversal through the contents of that object. In other words 
   ```py
   table_metadata = TableMetadata(...)
   isinstance(table_metadata, TableMetadata)  # True
   isinstance(table_metadata.properties, TableMetadata)  # True
   
   isinstance(table_metadata.snapshot_log, list)  # True
   isinstance(table_metadata.snapshot_log[0], TableMetadata)  # True
   ```
   If the idea of this partial metadata existing as a `TableMetadata` instance doesn't sit well, I could instead have a generic `Metadata` or `Config` class that actually traverses the json object to create a class with all of the class attributes, and have that as an argument to a `TableMetadata` class that handles validation and other table metadata related things.
   
   Something like:
   ```py
   config = Config({"table-uuid": "foo", ...})
   table_metadata = TableMetadata(config, version="2")
   ```
   
   The `_wrap()` method would live in the `Config` class and the docstring would read as:
   
   > A recursive function that drills into iterable values and returns
           nested Config instances




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

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

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



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


[GitHub] [iceberg] samredai commented on pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#issuecomment-987096895


   @CircArgs


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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763315297



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances

Review comment:
       Why would there be nested table metadata instances?




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763395732



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [

Review comment:
       Yeah I figured there could be a separate schema definition for v1 and v2 (and future v3). The version argument then essentially detemines which jsonschema is used. Sorry I should have mentioned this in the PR description (or better yet a TODO comment). I still have to complete the schemas to actually match the v1 and v2 specs exactly.




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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763477134



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances
+
+        Args:
+            value: A table metadata value.
+            - If it's a string, number, or boolean, the value is returned
+            - If it's an array, the values will be iterated through and processed and
+                added to an array that's returned
+            - If it's an object, another TableMetadata instance is returned
+
+        """
+
+        if isinstance(value, (tuple, list, set, frozenset)):
+            return type(value)([self._wrap(v) for v in value])
+        else:
+            return (
+                TableMetadata(value, version=self._version)
+                if (value and isinstance(value, dict))
+                else value
+            )
+
+    @staticmethod
+    def _clean_attribute_name(value):
+        """Fixes attribute names to be python friendly"""
+        return value.replace("-", "_").replace(".", "_")
+
+    def validate(self):

Review comment:
       Okay, sounds reasonable.




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

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

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



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


[GitHub] [iceberg] samredai edited a comment on pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai edited a comment on pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#issuecomment-1030135791


   I think this is ready for another look. I've rebased with the commits from the file io PR that was merged. The table metadata class has a `from_input_file` and `to_output_file` method that takes an `InputFile` or `OutputFile` instance.
   
   Another thing I did was move the `LocalFileIO`, `LocalInputFile`, and `LocalOutputFile` implementations to `conftest.py` as a fixture that can be used across all tests (see the docstring at the top of `conftest.py` for more details).
   
   If we can settle on the design here, I think a thorough inspection of the V1 and V2 jsonschemas defined here is needed before merging.


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

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

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r789171429



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,354 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.file import FileIO
+from iceberg.utils.io import get_file_io
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int
+    """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
+    """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
+    """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
+    """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
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int
+    """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."""
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        if self.format_version == 1:
+            self.validate_v1(self.to_dict())
+        elif self.format_version == 2:
+            self.validate_v2(self.to_dict())
+        else:
+            raise ValueError(f"Unknown table metadata version {self.format_version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, encoding="utf-8"):
+        """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
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls.from_dict(metadata)
+
+    @classmethod
+    def from_file(cls, path: str, custom_file_io: FileIO = None, **kwargs):

Review comment:
       api nit (apologies if the style has been discussed elsewhere) but it it might be more user-friendly to be less specific   on individual parameters.  e.g. have one parameter "file" which can be  ["str", "FileIO", ...] and make the decision internally on what to use.  Passing the base class as a factory seems a little strange since presumably clients know they have a custom_file_io and combine it with the path outside of this function.




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763304666



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances
+
+        Args:
+            value: A table metadata value.
+            - If it's a string, number, or boolean, the value is returned
+            - If it's an array, the values will be iterated through and processed and
+                added to an array that's returned
+            - If it's an object, another TableMetadata instance is returned
+
+        """
+
+        if isinstance(value, (tuple, list, set, frozenset)):
+            return type(value)([self._wrap(v) for v in value])
+        else:
+            return (
+                TableMetadata(value, version=self._version)
+                if (value and isinstance(value, dict))
+                else value
+            )
+
+    @staticmethod
+    def _clean_attribute_name(value):
+        """Fixes attribute names to be python friendly"""
+        return value.replace("-", "_").replace(".", "_")
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        casted_version = int(self._version)
+        if casted_version == 1:
+            self.validate_v1(self._metadata)
+        elif casted_version == 2:
+            self.validate_v2(self._metadata)
+        else:
+            raise ValueError(f"Unknown table metadata version {self._version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, version: Union[str, int], encoding="utf-8"):
+        """Instantiate a TableMetadata object from a byte stream
+
+        Args:
+            byte_stream: A file-like byte stream object
+            version: The Iceberg table metadata version (1 or 2)
+            encoding (default "utf-8"): The byte encoder to use for the reader
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls(metadata=metadata, version=version)
+
+    @classmethod
+    def from_s3(

Review comment:
       Currently, this `from_s3` class method is here, but this should be abstracted out to a generic file-io method that uses something like a `FileIO` abstract base class.




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763392319



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},

Review comment:
       Yeah I just did a simple top-level schema as part of this draft, but the final schema would include much more and be comprehensive of the spec. All types and required/optional flags. We can also add in regex requirements for string fields. The spec doesn't have to be a single huge spec either so we could have a separate jsonschema for PartitionSpec and just reference it in the larger table metadata jsonschema definition.




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

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

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r789171753



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,354 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.file import FileIO
+from iceberg.utils.io import get_file_io
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int
+    """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
+    """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
+    """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
+    """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
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int
+    """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."""
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        if self.format_version == 1:
+            self.validate_v1(self.to_dict())
+        elif self.format_version == 2:
+            self.validate_v2(self.to_dict())
+        else:
+            raise ValueError(f"Unknown table metadata version {self.format_version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, encoding="utf-8"):
+        """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
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls.from_dict(metadata)
+
+    @classmethod
+    def from_file(cls, path: str, custom_file_io: FileIO = None, **kwargs):
+        """Create a TableMetadata instance from a file path
+
+        Args:
+            path: A file path that can be read using an available FileIO implementation.
+            The FileIO implementation is determined using a string analysis on the path
+            by the iceberg.utils.file_io.get_file_io function
+            custom_file_io: A custom implementation of the iceberg.io.file.FileIO abstract
+            base class
+
+        Keyword Args:
+            Additional keyword arguments to pass forward to the instantiation of the appropriate
+            FileIO implementation.
+
+        """
+        if custom_file_io:
+            with custom_file_io(path, **kwargs) as f:
+                table_metadata = cls.from_byte_stream(byte_stream=f.byte_stream)
+        else:
+            with get_file_io(path, custom_file_io=custom_file_io, **kwargs) as f:

Review comment:
       it seems redundant to pass through custom_file_io here?  is the first branch really necessary?




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r765236691



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances
+
+        Args:
+            value: A table metadata value.
+            - If it's a string, number, or boolean, the value is returned
+            - If it's an array, the values will be iterated through and processed and
+                added to an array that's returned
+            - If it's an object, another TableMetadata instance is returned
+
+        """
+
+        if isinstance(value, (tuple, list, set, frozenset)):
+            return type(value)([self._wrap(v) for v in value])
+        else:
+            return (
+                TableMetadata(value, version=self._version)
+                if (value and isinstance(value, dict))
+                else value
+            )
+
+    @staticmethod
+    def _clean_attribute_name(value):
+        """Fixes attribute names to be python friendly"""
+        return value.replace("-", "_").replace(".", "_")
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        casted_version = int(self._version)
+        if casted_version == 1:
+            self.validate_v1(self._metadata)
+        elif casted_version == 2:
+            self.validate_v2(self._metadata)
+        else:
+            raise ValueError(f"Unknown table metadata version {self._version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, version: Union[str, int], encoding="utf-8"):
+        """Instantiate a TableMetadata object from a byte stream
+
+        Args:
+            byte_stream: A file-like byte stream object
+            version: The Iceberg table metadata version (1 or 2)
+            encoding (default "utf-8"): The byte encoder to use for the reader
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls(metadata=metadata, version=version)
+
+    @classmethod
+    def from_s3(

Review comment:
       This has been refactored to use a generic `from_file()` method that's dependent on PR #3691 which adds the `FileIO` abstract base class




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

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

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



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


[GitHub] [iceberg] emkornfield commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and InputFile implementation

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r800125575



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,365 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.base import InputFile, OutputFile
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list

Review comment:
       Is the intention to refine collection types later with actual object types or keep these fairly generic?




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763431089



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances
+
+        Args:
+            value: A table metadata value.
+            - If it's a string, number, or boolean, the value is returned
+            - If it's an array, the values will be iterated through and processed and
+                added to an array that's returned
+            - If it's an object, another TableMetadata instance is returned
+
+        """
+
+        if isinstance(value, (tuple, list, set, frozenset)):
+            return type(value)([self._wrap(v) for v in value])
+        else:
+            return (
+                TableMetadata(value, version=self._version)
+                if (value and isinstance(value, dict))
+                else value
+            )
+
+    @staticmethod
+    def _clean_attribute_name(value):
+        """Fixes attribute names to be python friendly"""
+        return value.replace("-", "_").replace(".", "_")
+
+    def validate(self):

Review comment:
       I'd bet that we can get the jsonschema pretty accurate to the point where only bonafide validation failures raise a ValidationError. For your example, we could use some of the [conditional schema](https://json-schema.org/understanding-json-schema/reference/conditionals.html#applying-subschemas-conditionally) features, specifically the `dependentRequired` which should let us define something that says "`schema` is optional when `schemas` and `current-schema-id` exist"




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

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

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



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


[GitHub] [iceberg] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and InputFile implementation

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r800125947



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,365 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.base import InputFile, OutputFile
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list

Review comment:
       I'd say we should definitely refine these later, i.e. list[PartitionSpec] once that class has been 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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763316871



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances
+
+        Args:
+            value: A table metadata value.
+            - If it's a string, number, or boolean, the value is returned
+            - If it's an array, the values will be iterated through and processed and
+                added to an array that's returned
+            - If it's an object, another TableMetadata instance is returned
+
+        """
+
+        if isinstance(value, (tuple, list, set, frozenset)):
+            return type(value)([self._wrap(v) for v in value])
+        else:
+            return (
+                TableMetadata(value, version=self._version)
+                if (value and isinstance(value, dict))
+                else value
+            )
+
+    @staticmethod
+    def _clean_attribute_name(value):
+        """Fixes attribute names to be python friendly"""
+        return value.replace("-", "_").replace(".", "_")
+
+    def validate(self):

Review comment:
       I like how this can use JSON schema to validate, but what should we do if validation fails? Are there some things that we can recover from? For example, v1 metadata with `schemas` and `current-schema-id` but not `schema` is technically invalid. But we can still read the table just fine.




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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763314624



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version

Review comment:
       `int(version)` to be able to assume that it is an int elsewhere?




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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763313494



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [

Review comment:
       Is this listing the fields that are required for writing or reading? For reading, we have to be more relaxed because we need to accept older versions of the spec. For example, `schemas` and `current-schema-id` didn't used to exist. It used to be a single `schema` 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] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r765240638



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances

Review comment:
       I've updated the PR to remove the wrapper approach and set the top level as class attributes. I've also added a `to_dict()` method that's used by `validate()` where the `TableMetadata` instance is serialized into a python dictionary and then passed into the jsonschema validate function. I've also added freezing of instances and this punts the question of how we perform table metadata updates.
   
   I can follow-up with a PR that uses the builder pattern where a python `dict` is the mutable form. So building from an existing table metadata instance would start with calling `to_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] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r789379673



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,354 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.file import FileIO
+from iceberg.utils.io import get_file_io
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int
+    """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
+    """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
+    """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
+    """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
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int
+    """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."""
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        if self.format_version == 1:
+            self.validate_v1(self.to_dict())
+        elif self.format_version == 2:
+            self.validate_v2(self.to_dict())
+        else:
+            raise ValueError(f"Unknown table metadata version {self.format_version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, encoding="utf-8"):
+        """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
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls.from_dict(metadata)
+
+    @classmethod
+    def from_file(cls, path: str, custom_file_io: FileIO = None, **kwargs):
+        """Create a TableMetadata instance from a file path
+
+        Args:
+            path: A file path that can be read using an available FileIO implementation.
+            The FileIO implementation is determined using a string analysis on the path
+            by the iceberg.utils.file_io.get_file_io function
+            custom_file_io: A custom implementation of the iceberg.io.file.FileIO abstract
+            base class
+
+        Keyword Args:
+            Additional keyword arguments to pass forward to the instantiation of the appropriate
+            FileIO implementation.
+
+        """
+        if custom_file_io:
+            with custom_file_io(path, **kwargs) as f:
+                table_metadata = cls.from_byte_stream(byte_stream=f.byte_stream)
+        else:
+            with get_file_io(path, custom_file_io=custom_file_io, **kwargs) as f:

Review comment:
       Great catch. The `get_file_io` function (not included in this PR) should be responsible for handling when a custom file-io is provided, i.e. `custom_file_io != None`. I'm planning to do another pass at this draft once PR #3691 is finalized and I'll make sure to update this. 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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763312746



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},

Review comment:
       Would these eventually have type information?




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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763317445



##########
File path: python/tests/table/test_metadata.py
##########
@@ -0,0 +1,155 @@
+import io
+import json
+from unittest.mock import MagicMock, patch
+
+import pytest
+from jsonschema.exceptions import ValidationError
+
+from iceberg.table.metadata import TableMetadata
+
+EXAMPLE_TABLE_METADATA_V1 = {
+    "format-version": "1",
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-sequence-number": 3,
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schemas": [],
+    "current-schema-id": 5,
+    "partition-spec": [],
+    "partition-specs": [],
+    "default-spec-id": 6,
+    "last-partition-id": 7,
+    "properties": {"read.split.target-size": 134217728},
+    "current-snapshot-id": 1.1,
+    "snapshots": [],
+    "snapshot-log": [],
+    "metadata-log": [],
+    "sort-orders": [],
+    "default-sort-order-id": 8,
+}
+EXAMPLE_TABLE_METADATA_V2 = {
+    "format-version": "2",
+    "table-uuid": "foo-table-uuid",
+    "location": "s3://foo/bar/baz.metadata.json",
+    "last-sequence-number": 3,
+    "last-updated-ms": 1600000000000,
+    "last-column-id": 4,
+    "schemas": [],
+    "current-schema-id": 5,
+    "partition-spec": [],
+    "partition-specs": [],
+    "default-spec-id": 6,
+    "last-partition-id": 7,
+    "properties": {"read.split.target-size": 134217728},
+    "current-snapshot-id": 1.1,
+    "snapshots": [],
+    "snapshot-log": [],
+    "metadata-log": [],
+    "sort-orders": [],
+    "default-sort-order-id": 8,
+}
+
+
+@pytest.mark.parametrize(
+    "metadata, version",
+    [
+        (
+            EXAMPLE_TABLE_METADATA_V1,
+            1,
+        ),
+        (
+            EXAMPLE_TABLE_METADATA_V2,
+            2,
+        ),
+    ],
+)
+def test_from_dict(metadata, version):
+    """Test standard initialization of a TableMetadata instance from a dictionary"""
+    table_metadata = TableMetadata(metadata=metadata, version=version)
+    table_metadata.validate()
+
+
+def test_invalid_v1_metadata():
+    """Test raising a ValidationError on invalid v1 metadata"""
+    with pytest.raises(ValidationError):
+        TableMetadata.validate_v1(metadata={"foo": "bar"})
+
+
+def test_invalid_v2_metadata():
+    """Test raising a ValidationError on invalid v2 metadata"""
+    with pytest.raises(ValidationError):
+        TableMetadata.validate_v2(metadata={"foo": "bar"})
+
+
+def test_unknown_table_metadata_version():
+    """Test raising a ValueError when an invalid table metadata version is provided"""
+    with pytest.raises(ValueError) as exc_info:
+        TableMetadata(metadata=EXAMPLE_TABLE_METADATA_V2, version=3).validate()
+    assert "Unknown table metadata version 3" in str(exc_info)
+
+
+def test_from_byte_stream():
+    """Test generating a TableMetadata instance from a file-like byte stream"""
+    data = bytes(json.dumps(EXAMPLE_TABLE_METADATA_V2), encoding="utf-8")
+    byte_stream = io.BytesIO(data)
+    TableMetadata.from_byte_stream(byte_stream, version=1).validate()
+
+
+@patch("iceberg.table.metadata.boto3")
+def test_from_s3(mock_boto3):
+    """Test generating a TableMetadata instance from a metadata file on S3"""
+    data = bytes(json.dumps(EXAMPLE_TABLE_METADATA_V2), encoding="utf-8")
+    byte_stream = io.BytesIO(data)
+
+    mock_s3_client = MagicMock()
+    mock_s3_client.get_object.return_value = {"Body": byte_stream}
+
+    mock_boto3_session = MagicMock()
+    mock_boto3_session.client.return_value = mock_s3_client
+
+    mock_boto3.Session.return_value = mock_boto3_session
+
+    TableMetadata.from_s3("s3://foo/bar/metadata.json", version=2).validate()
+
+
+def test_metadata_parsing():
+    """Test retrieveing values from TableMetadata instance"""
+    table_metadata = TableMetadata(metadata=EXAMPLE_TABLE_METADATA_V2, version=2)
+    table_metadata.validate()
+
+    assert table_metadata.format_version == "2"
+    assert table_metadata.table_uuid == "foo-table-uuid"
+    assert table_metadata.location == "s3://foo/bar/baz.metadata.json"
+    assert table_metadata.last_sequence_number == 3
+    assert table_metadata.last_updated_ms == 1600000000000
+    assert table_metadata.last_column_id == 4
+    assert table_metadata.schemas == []
+    assert table_metadata.current_schema_id == 5
+    assert table_metadata.partition_spec == []
+    assert table_metadata.partition_specs == []
+    assert table_metadata.default_spec_id == 6
+    assert table_metadata.last_partition_id == 7
+    assert table_metadata.properties.read_split_target_size == 134217728
+    assert table_metadata.current_snapshot_id == 1.1
+    assert table_metadata.snapshots == []
+    assert table_metadata.snapshot_log == []
+    assert table_metadata.metadata_log == []
+    assert table_metadata.sort_orders == []
+    assert table_metadata.default_sort_order_id == 8
+
+    assert table_metadata.properties._metadata == {"read.split.target-size": 134217728}

Review comment:
       This test looks good 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] rdblue commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r763480075



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances

Review comment:
       Okay, I think I get it.
   
   I like being able to use `m.snapshot_log[0]` and similar ways to access the metadata. But, I'm not sure that this is changing enough that it's worth the wrapper approach, instead of just converting to a `TableMetadata` class that pulls out and stores `self.current_schema_id` (for example). Table metadata shouldn't be _that_ complicated since it's mostly a few lists of objects at the most nested level (like metadata > snapshots > snapshot > properties > key/value).
   
   While this makes it easy to get started, it would be awkward to make updates to the metadata as JSON because you'd need to produce a new JSON tree and then wrap with this class. We may also want classes for things like Snapshot, which can embed some operations like reading 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] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r765240638



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,215 @@
+import codecs
+import json
+from typing import Any, Optional, Union
+
+import boto3
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.s3 import S3Url
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-04/schema#",
+    "type": "object",
+    "properties": {
+        "format-version": {"type": "string"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "integer"},
+        "last-updated-ms": {"type": "integer"},
+        "last-column-id": {"type": "integer"},
+        "schemas": {"type": "array", "items": {}},
+        "current-schema-id": {"type": "integer"},
+        "partition-spec": {"type": "array", "items": {}},
+        "partition-specs": {"type": "array", "items": {}},
+        "default-spec-id": {"type": "integer"},
+        "last-partition-id": {"type": "integer"},
+        "properties": {"type": "object"},
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {"type": "array", "items": {}},
+        "snapshot-log": {"type": "array", "items": {}},
+        "metadata-log": {"type": "array", "items": {}},
+        "sort-orders": {"type": "array", "items": {}},
+        "default-sort-order-id": {"type": "integer"},
+    },
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-spec",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "properties",
+        "current-snapshot-id",
+        "snapshots",
+        "snapshot-log",
+        "metadata-log",
+        "sort-orders",
+        "default-sort-order-id",
+    ],
+}
+
+
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)
+
+    Args:
+      metadata (dict): Table metadata dictionary representation
+      version (str|int): The metadata spec version (1 or 2)
+    """
+
+    def __init__(self, metadata: dict, version: Union[str, int]):
+        self._version = version
+        self._metadata = metadata
+        for name, value in metadata.items():
+            setattr(self, self._clean_attribute_name(name), self._wrap(value))
+
+    def _wrap(self, value: Any):
+        """A recursive function that drills into iterable values and returns
+        nested TableMetadata instances

Review comment:
       I've updated the PR to remove the wrapper approach and set the top level as class attributes. I've also added a `to_dict()` method that's used by `validate()` where the `TableMetadata` instance is serialized into a python dictionary and then passed into the jsonschema validate function.
   
   I've also added freezing of instances and this punts the question of how we perform table metadata updates. I can follow-up with a PR that uses the builder pattern where a python `dict` is the mutable form. So building from an existing table metadata instance would start with calling `to_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] samredai commented on a change in pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on a change in pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#discussion_r790390183



##########
File path: python/src/iceberg/table/metadata.py
##########
@@ -0,0 +1,354 @@
+# 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
+
+import attr
+from jsonschema import validate as validate_json
+from jsonschema.exceptions import ValidationError
+
+from iceberg.io.file import FileIO
+from iceberg.utils.io import get_file_io
+
+TABLE_METADATA_V1_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "location",
+        "last-updated-ms",
+        "last-column-id",
+        "schema",
+        "partition-spec",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schema": {
+            "type": "object",
+        },
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-spec": {
+            "type": "object",
+        },
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+TABLE_METADATA_V2_SCHEMA = {
+    "$schema": "http://json-schema.org/draft-07/schema",
+    "type": "object",
+    "required": [
+        "format-version",
+        "table-uuid",
+        "location",
+        "last-sequence-number",
+        "last-updated-ms",
+        "last-column-id",
+        "schemas",
+        "current-schema-id",
+        "partition-specs",
+        "default-spec-id",
+        "last-partition-id",
+        "default-sort-order-id",
+        "sort-orders",
+    ],
+    "properties": {
+        "format-version": {"type": "number"},
+        "table-uuid": {"type": "string"},
+        "location": {"type": "string"},
+        "last-sequence-number": {"type": "number"},
+        "last-updated-ms": {"type": "number"},
+        "last-column-id": {"type": "number"},
+        "schemas": {
+            "type": "array",
+        },
+        "current-schema-id": {"type": "number"},
+        "partition-specs": {
+            "type": "array",
+        },
+        "default-spec-id": {"type": "number"},
+        "last-partition-id": {"type": "number"},
+        "properties": {
+            "type": "object",
+        },
+        "current-snapshot-id": {"type": "number"},
+        "snapshots": {
+            "type": "array",
+        },
+        "snapshot-log": {
+            "type": "array",
+        },
+        "metadata-log": {
+            "type": "array",
+        },
+        "sort-orders": {
+            "type": "array",
+        },
+        "default-sort-order-id": {"type": "number"},
+    },
+    "additionalProperties": False,
+}
+
+
+@attr.s(frozen=True, auto_attribs=True)
+class TableMetadata:
+    """Metadata for an Iceberg table as specified in the Apache Iceberg
+    spec (https://iceberg.apache.org/spec/#iceberg-table-spec)"""
+
+    format_version: int
+    """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: str
+    """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_sequence_number: int
+    """The table’s highest assigned sequence number, a monotonically
+    increasing long that tracks the order of snapshots in a table."""
+
+    last_updated_ms: int
+    """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
+    """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."""
+
+    schema: dict
+    """The table’s current schema. (Deprecated: use schemas and 
+    current-schema-id instead)"""
+
+    schemas: list
+    """A list of schemas, stored as objects with schema-id."""
+
+    current_schema_id: int
+    """ID of the table’s current schema."""
+
+    partition_spec: dict
+    """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)"""
+
+    partition_specs: list
+    """A list of partition specs, stored as full partition spec objects."""
+
+    default_spec_id: int
+    """ID of the “current” spec that writers should use by default."""
+
+    last_partition_id: int
+    """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
+    """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
+    """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
+    """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
+    """A list of sort orders, stored as full sort order objects."""
+
+    default_sort_order_id: int
+    """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."""
+
+    def validate(self):
+        """Checks that the table metadata object is valid. The validation schema
+        used depends on the Iceberg table metadata version."""
+        if self.format_version == 1:
+            self.validate_v1(self.to_dict())
+        elif self.format_version == 2:
+            self.validate_v2(self.to_dict())
+        else:
+            raise ValueError(f"Unknown table metadata version {self.format_version}")
+
+    @staticmethod
+    def validate_v1(metadata: dict):
+        """Perform a JSONSchema validation using the v1 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V1_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @staticmethod
+    def validate_v2(metadata: dict):
+        """Perform a JSONSchema validation using the v2 Iceberg table metadata schema"""
+        try:
+            validate_json(instance=metadata, schema=TABLE_METADATA_V2_SCHEMA)
+        except ValidationError as e:
+            # TODO Log something here
+            raise (e)
+
+    @classmethod
+    def from_byte_stream(cls, byte_stream, encoding="utf-8"):
+        """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
+        """
+        reader = codecs.getreader(encoding)
+        metadata = json.load(reader(byte_stream))
+        return cls.from_dict(metadata)
+
+    @classmethod
+    def from_file(cls, path: str, custom_file_io: FileIO = None, **kwargs):

Review comment:
       Great point, I totally agree. A custom file-io would have a custom `InputFile` implementation that would already include the path when used here. If a string uri is provided instead I'll handle that by finding the right "known" implementation.
   
   I'm going to start working on this and getting it out of draft status!




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

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

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



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


[GitHub] [iceberg] samredai commented on pull request #3677: Python: Adding TableMetadata object from dict, bytestream, and s3 sources

Posted by GitBox <gi...@apache.org>.
samredai commented on pull request #3677:
URL: https://github.com/apache/iceberg/pull/3677#issuecomment-1030135791


   I think this is ready for another look. I've rebased with the commits from the file io PR that was merged. The table metadata class has a `from_input_file` and `to_output_file` method that takes an `InputFile` or `OutputFile` instance.
   
   Another thing I did was move the `LocalFileIO`, `LocalInputFile`, and `LocalOutputFile` implementations to `conftest.py` as a fixture that can be used across all tests (see the docstring at the top of `conftest.py` for more details).


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

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

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



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