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

[GitHub] [iceberg] Fokko opened a new pull request, #6323: Python: Alter table for table properties

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

   This is a first version of how to alter a table. As a first step I've implemented the ability to set and delete properties on a table.
   
   We take the TableUpdate construct from the Rest catalog as a standard for the other catalogs as well.


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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class RemoveSnapshotsUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class CommitTableRequest(IcebergBaseModel):
+    requirements: List[Any] = Field(default_factory=list)

Review Comment:
   Good point. I didn't want to make the PR even bigger. I've added it again. The code generated from the open-api spec is a bit awkward:
   
   ```python
   class TableRequirement(IcebergBaseModel):
       requirement: Requirement
       ref: Optional[str] = None
       uuid: Optional[str] = None
       snapshot_id: Optional[int] = Field(None, alias='snapshot-id')
       last_assigned_field_id: Optional[int] = Field(None, alias='last-assigned-field-id')
       current_schema_id: Optional[int] = Field(None, alias='current-schema-id')
       last_assigned_partition_id: Optional[int] = Field(
           None, alias='last-assigned-partition-id'
       )
       default_spec_id: Optional[int] = Field(None, alias='default-spec-id')
       default_sort_order_id: Optional[int] = Field(None, alias='default-sort-order-id')
   ```
   
   Do we want to change this on the spec level? I'd rather leave this out, fix the spec and then add it.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:

Review Comment:
   Looks like this is still defined. I think we should avoid adding these methods because I'm skeptical that we want to expose a way to directly set a schema. That's just going to confuse callers.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:

Review Comment:
   Removed it



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):

Review Comment:
   Since this is in the public API, why not call it `TableUpdate` rather than `BaseTableUpdate`? We use the `Base` prefix in Java, but almost never in the public API. Those are usually private classes.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/feature-support.md:
##########
@@ -42,6 +42,22 @@ The goal is that the python library will provide a functional, performant subset
 | Drop Namespace           |  X   |   X    |
 | Set Namespace Properties |  X   |   X    |
 
+## Catalog capabilities
+
+The following table projects the capabilities of each of the available Catalog implementations:
+
+|                        | Rest | Hive | Glue | DynamoDB |
+| ---------------------- | ---- | ---- | ---- | -------- |
+| CRUD Namespaces        | ✓    | ✓    | ✓    | ✓        |
+| CRUD Tables            | ✓    | ✓    | ✓    | ✓        |
+| Update format version  | ✓    | ✕    | ✕    | ✕        |
+| Alter table properties | ✓    | ✕    | ✕    | ✕        |
+| Alter schema           | ✓    | ✕    | ✕    | ✕        |
+| Alter partition spec   | ✓    | ✕    | ✕    | ✕        |
+| Alter sort order       | ✓    | ✕    | ✕    | ✕        |

Review Comment:
   Done



##########
python/pyiceberg/catalog/null.py:
##########
@@ -0,0 +1,75 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+from typing import (
+    List,
+    Optional,
+    Set,
+    Tuple,
+    Union,
+)
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.table import BaseTableUpdate, SortOrder, Table
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER
+from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties
+
+
+class NoopCatalog(Catalog):

Review Comment:
   Fine by 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 diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -323,6 +323,17 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
             NoSuchTableError: If a table with the name does not exist
         """
 
+    @abstractmethod
+    def update_table(self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...]) -> Table:

Review Comment:
   FYI, we're adding a similar method in @nastra's transaction work: https://github.com/apache/iceberg/pull/7569
   
   We probably want to align the naming so maybe this should be `commit_table` to align with the `commit_transaction`.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -489,6 +493,39 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def _commit(self, *table_requests: CommitTableRequest) -> CommitTableResponse:

Review Comment:
   Yes, thinking of it, that makes sense. Also, the return type is different for a table and a transaction. Thanks!



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table for table properties

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


##########
python/mkdocs/docs/index.md:
##########
@@ -434,6 +435,14 @@ Table(
 )
 ```
 
+Altering the table:
+
+```python
+table.alter().update_namespace_properties(
+    updates={"department": "finance"}, removes=["owner"]

Review Comment:
   In the REST spec, the second argument is `removals`. Should we match that?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table for table properties

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


##########
python/mkdocs/docs/index.md:
##########
@@ -434,6 +435,14 @@ Table(
 )
 ```
 
+Altering the table:
+
+```python
+table.alter().update_namespace_properties(
+    updates={"department": "finance"}, removes=["owner"]

Review Comment:
   Ah, that's correct. I went a bit back and forth because of https://github.com/apache/iceberg/pull/6317. The actual code expects `removals`, but I didn't update the docs accordingly. Thanks for spotting 👍🏻 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):

Review Comment:
   I like that, updated



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,313 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+    _requirements: Tuple[TableRequirement, ...]
+
+    def __init__(
+        self,
+        table: Table,
+        actions: Optional[Tuple[TableUpdate, ...]] = None,
+        requirements: Optional[Tuple[TableRequirement, ...]] = None,
+    ):
+        self._table = table
+        self._updates = actions or ()
+        self._requirements = requirements or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")

Review Comment:
   It looks like this class is attempting to behave like a transaction because it will stack up a set of changes and commit them all at once. That seems reasonable but then we get strange cases like this where there are odd restrictions. This would definitely happen because the changes for a real transaction would commonly include more than one `AddSnapshot` updates, but just one `SetRefSnapshotId` update.
   
   I think this is also going to hit an issue with complex changes, like `UpdateSchema`. That changes supports multiple calls and then results in a finished schema that is sent using `AddSchema` and `SetCurrentSchemaId` updates. For the API, this would either need to include all of the schema change methods here -- which will get ugly really fast -- or we need a way to have a `UpdateSchema` API that returns back to the overall transaction API.
   
   In Java, we took the second approach. There's a common `UpdateSchema` API that can be performed as a single operation on a table (`table.updateSchema().addColumn("x", IntType.get()).commit()`) or combined with others in a transaction. (`table.newTransaction()` / `transaction.updateSchema().commit()` / `transaction.commitTransaction()`).
   
   I suspect that we want to do the same thing here and have some kind of transaction that accumulates changes from other more specific APIs.
   
   It looks like the issue with this PR is trying to combine the transaction object that accumulates changes and calls `catalog.commit_table` with the public APIs for making changes to a table. I think I would take the same approach as Java and have a `Transaction` object to represent multiple changes to a table, but I would hide that from users in most cases.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_partition_spec(self, spec: PartitionSpec) -> TableUpdates:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_sort_order(self, sort_order: SortOrder) -> TableUpdates:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_properties(self, **updates: str) -> TableUpdates:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def remove_properties(self, *removals: str) -> TableUpdates:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> TableUpdates:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.update_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class TableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(TableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+
+
+class SetCurrentSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(TableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(TableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+    type: Literal["tag", "branch"]
+    snapshot_id: int = Field(alias="snapshot-id")
+    max_age_ref_ms: int = Field(alias="max-ref-age-ms")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+
+
+class RemoveSnapshotsUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(TableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class Requirement(Enum):
+    assert_create = "assert-create"
+    assert_table_uuid = "assert-table-uuid"
+    assert_ref_snapshot_id = "assert-ref-snapshot-id"
+    assert_last_assigned_field_id = "assert-last-assigned-field-id"
+    assert_current_schema_id = "assert-current-schema-id"
+    assert_last_assigned_partition_id = "assert-last-assigned-partition-id"
+    assert_default_spec_id = "assert-default-spec-id"
+    assert_default_sort_order_id = "assert-default-sort-order-id"
+
+
+class TableRequirement(IcebergBaseModel):
+    requirement: Requirement
+    ref: Optional[str] = None
+    uuid: Optional[str] = None
+    snapshot_id: Optional[int] = Field(None, alias="snapshot-id")
+    last_assigned_field_id: Optional[int] = Field(None, alias="last-assigned-field-id")
+    current_schema_id: Optional[int] = Field(None, alias="current-schema-id")
+    last_assigned_partition_id: Optional[int] = Field(None, alias="last-assigned-partition-id")
+    default_spec_id: Optional[int] = Field(None, alias="default-spec-id")
+    default_sort_order_id: Optional[int] = Field(None, alias="default-sort-order-id")
+
+
+class CommitTableRequest(IcebergBaseModel):

Review Comment:
   This is getting an optional `identifier` in Eduard's PR.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()

Review Comment:
   We should also remove it from docs until we know what API we will provide.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class RemoveSnapshotsUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class CommitTableRequest(IcebergBaseModel):
+    requirements: List[Any] = Field(default_factory=list)

Review Comment:
   I'd prefer to generate the code from the spec, so we know that the spec makes sense. I've updated it here: https://github.com/apache/iceberg/pull/7710. I also created a PR to get more insights into what kind of code the yaml is generating: https://github.com/apache/iceberg/pull/7751 I believe the open-api yaml is very handy when implementing clients in other languages (Rust, Go, etc), but then we need to make sure that we generate what we expect (and for me the open-api yaml is sometimes not very straightforward).



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,313 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+    _requirements: Tuple[TableRequirement, ...]
+
+    def __init__(
+        self,
+        table: Table,
+        actions: Optional[Tuple[TableUpdate, ...]] = None,
+        requirements: Optional[Tuple[TableRequirement, ...]] = None,
+    ):
+        self._table = table
+        self._updates = actions or ()
+        self._requirements = requirements or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")

Review Comment:
   The whole idea of this check is to avoid multiple similar operations. I agree that when you change a schema, all the updates to the schema are accumulated into one `AddSchemaUpdate`. If you try to add another update to the transaction of the same type, it will throw the `ValueError` that we see above.
   
   The whole public API is currently:
   
   ```python
   table.new_transaction.set_table_version(2).commit()
   table.new_transaction.set_properties(**{
       "lifecycle": "true"
   }).commit()
   table.new_transaction.remove_properties("lifecycle").commit()
   table.new_transaction.update_location("s3://...").commit()
   ```
   
   And you can combine them:
   ```python
   table.new_transaction.set_table_version(2).update_location("s3://...").commit()
   ```
   
   Coming multiple updates of identical type will raise a `ValueError`:
   ```python
   table.new_transaction.set_table_version(2).set_table_version(2).commit()
   ```
   I think this will guard us from getting into nasty situations. We can always relax this in the future to allow multiple snapshots, but then the requirements should be in order as well.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   Would it be possible to add more tests later once we get the schema, partitioning, and sorting figured out? I did some refactoring on the integration tests. But that's being blocked on this one: https://github.com/apache/iceberg/pull/7704



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[BaseTableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        print(updates)

Review Comment:
   Great catch, thanks



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_partition_spec(self, spec: PartitionSpec) -> TableUpdates:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_sort_order(self, sort_order: SortOrder) -> TableUpdates:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_properties(self, **updates: str) -> TableUpdates:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def remove_properties(self, *removals: str) -> TableUpdates:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> TableUpdates:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.update_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class TableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(TableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+
+
+class SetCurrentSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(TableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(TableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+    type: Literal["tag", "branch"]
+    snapshot_id: int = Field(alias="snapshot-id")
+    max_age_ref_ms: int = Field(alias="max-ref-age-ms")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+
+
+class RemoveSnapshotsUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(TableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class Requirement(Enum):
+    assert_create = "assert-create"
+    assert_table_uuid = "assert-table-uuid"
+    assert_ref_snapshot_id = "assert-ref-snapshot-id"
+    assert_last_assigned_field_id = "assert-last-assigned-field-id"
+    assert_current_schema_id = "assert-current-schema-id"
+    assert_last_assigned_partition_id = "assert-last-assigned-partition-id"
+    assert_default_spec_id = "assert-default-spec-id"
+    assert_default_sort_order_id = "assert-default-sort-order-id"
+
+
+class TableRequirement(IcebergBaseModel):

Review Comment:
   I think we should definitely make this look like the update classes. This requirement structure is difficult to work with.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")

Review Comment:
   While this is sent by the Java side, it isn't a required field because it can be computed from the max ID in the schema and the current `last-column-id`. No problem sending it as well, but it isn't required.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):

Review Comment:
   Does all of this need to be in the `__init__.py` class? Maybe we have a separate `updates.py`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -191,11 +445,14 @@ def from_metadata(cls, metadata_location: str, properties: Properties = EMPTY_DI
 
         metadata = FromInputFile.table_metadata(file)
 
+        from pyiceberg.catalog.null import NoopCatalog
+
         return cls(
             identifier=("static-table", metadata_location),
             metadata_location=metadata_location,
             metadata=metadata,
             io=load_file_io({**properties, **metadata.properties}),
+            catalog=NoopCatalog("static-table"),

Review Comment:
   Couldn't we just check that the catalog is `None` and throw an error? Seems like we don't necessarily need the `NoopCatalog`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +500,34 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def update_table(self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...]) -> TableResponse:

Review Comment:
   This needs requirements, right?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/feature-support.md:
##########
@@ -42,6 +42,22 @@ The goal is that the python library will provide a functional, performant subset
 | Drop Namespace           |  X   |   X    |
 | Set Namespace Properties |  X   |   X    |
 
+## Catalog capabilities
+
+The following table projects the capabilities of each of the available Catalog implementations:
+
+|                        | Rest | Hive | Glue | DynamoDB |
+| ---------------------- | ---- | ---- | ---- | -------- |
+| CRUD Namespaces        | ✓    | ✓    | ✓    | ✓        |
+| CRUD Tables            | ✓    | ✓    | ✓    | ✓        |
+| Update format version  | ✓    | ✕    | ✕    | ✕        |
+| Alter table properties | ✓    | ✕    | ✕    | ✕        |
+| Alter schema           | ✓    | ✕    | ✕    | ✕        |
+| Alter partition spec   | ✓    | ✕    | ✕    | ✕        |
+| Alter sort order       | ✓    | ✕    | ✕    | ✕        |

Review Comment:
   I think we should start with table properties and move on to the other updates after that.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:

Review Comment:
   Shouldn't this be `update_table` rather than `alter`? Alter is typically used for metadata changes and this route is used for both.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[BaseTableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        print(updates)
+        payload = CommitTableRequest(updates=updates).json()
+        response = self._session.post(
+            self.url(Endpoints.update_table, prefixed=True, **self._split_identifier_for_path(identifier)),
+            data=payload,
+        )
+        try:
+            response.raise_for_status()
+        except HTTPError as exc:
+            self._handle_non_200_response(exc, {})

Review Comment:
   Great catch



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +501,36 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def commit_table(
+        self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequirement, ...]
+    ) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[TableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        payload = CommitTableRequest(updates=updates).json()

Review Comment:
   I just noticed this as well. Added them here and in all the `commit_table` 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] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +501,36 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def commit_table(
+        self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...], requirements: Tuple[TableRequirement, ...]
+    ) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[TableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        payload = CommitTableRequest(updates=updates).json()

Review Comment:
   Missing requirements here.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -64,25 +67,326 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class Transaction:

Review Comment:
   I'd rather expect a `List[Transaction]`



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()

Review Comment:
   I really think it is unlikely that we will want to expose a `set_schema` function. That may be a simple API, but it makes it harder for both the caller and for the library to do things correctly.
   
   It would force the caller to understand field IDs and how they are used to correctly construct new schemas. The caller also needs to know that the library will reassign field IDs when a table is created, so they can't just modify the original schema used to create a table. This has been really confusing for people in the past.
   
   For the library, we have to detect the changes that are being made and validate them. It's not too bad to detect changes, but it's still harder.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/null.py:
##########
@@ -0,0 +1,75 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+from typing import (
+    List,
+    Optional,
+    Set,
+    Tuple,
+    Union,
+)
+
+from pyiceberg.catalog import Catalog, PropertiesUpdateSummary
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.table import BaseTableUpdate, SortOrder, Table
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER
+from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties
+
+
+class NoopCatalog(Catalog):

Review Comment:
   What about `noop.py` instead of `null`? Or could this be in `catalog/__init__.py`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[BaseTableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        print(updates)
+        payload = CommitTableRequest(updates=updates).json()
+        response = self._session.post(
+            self.url(Endpoints.update_table, prefixed=True, **self._split_identifier_for_path(identifier)),
+            data=payload,
+        )
+        try:
+            response.raise_for_status()
+        except HTTPError as exc:
+            self._handle_non_200_response(exc, {})

Review Comment:
   Commit error handling needs [some different logic](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java#L71-L90).
   * 409 should throw `CommitFailedException` so that the caller can refresh metadata and retry
   * 500, 502, and 504 should throw `CommitStateUnknownException` because it isn't clear whether the attempt succeeded or failed.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   Oh, I think you're right. This is the one to set the state of the ref, not to create a new ref.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   Sorry, but I think you're right and that the extra fields are not supposed to be here.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -200,7 +105,7 @@ table = StaticTable.from_metadata(
 
 ## Create a table
 
-To create a table from a catalog:
+To create a table by passing the table to the catalog:

Review Comment:
   In between changes :) thanks!



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   Good one. First of all, if we as a community decide that we really want to make things strict (which is a good thing), we should do checks on the server side that forbid this kind of operation. We can also add this on the Python side, for example, set the IDs of the existing fields (using the WithPartnerVisitor). People always will find ways to do weird stuff :D (for example, by copying Python snippets).
   
   I would expect something like this:
   
   ```python
   catalog = load_catalog(...)
   
   schema = Schema(
       NestedField(field_id=0, name='id', field_type=LongType()),
       schema_id=1
   )
   
   table = catalog.create_table(identifier='test', schema=schema)
   
   new_schema = table.schema.alter.add_field(
       name="data",
       type=StringType(),
       required=False
   )
   
   new_schema = table.schema.alter.drop_field(
       name="id"
   )
   
   table.set_schema(
       new_schema
   )
   ```
   
   However, this gets awkward for lists, maps, and structs.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   > Would it be possible to add more tests later once we get the schema, partitioning, and sorting figured out?
   
   Yes! I think we should limit this to setting and removing properties. Then we can handle the schema, spec, and write order changes later.
   
   > However, this gets awkward for lists, maps, and structs.
   
   We already handle these cases in the Java API, using the same approach that you outlined above with methods to make individual changes to the schema that correspond to SQL DDL operations (e.g. ADD COLUMN). There are [thorough tests](https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java#L669-L737) as well.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   > Would it be possible to add more tests later once we get the schema, partitioning, and sorting figured out?
   
   Yes! I think we should limit this to setting and removing properties. Then we can handle the schema, spec, and write order changes later.
   
   > However, this gets awkward for lists, maps, and structs.
   
   We already handle these cases in the Java API, using the same approach that you outlined above with methods to make individual changes to the schema that correspond to SQL DDL operations (e.g. ADD COLUMN). There are [thorough tests](https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java#L669-L737) as well.
   
   > we should do checks on the server side that forbid this kind of operation
   
   I agree that we should do as much as we can, but the client is responsible for this logic for non-REST catalogs. Plus, even in the REST catalog we can only validate using `last-column-id`. There are plenty of changes that are valid, but not what the user intended to do because they don't understand IDs.
   
   For example, if I modify my example above slightly:
   
   ```python
   catalog = load_catalog(...)
   
   schema1 = Schema(
       NestedField(field_id=0, name='id', field_type=IntegerType()),
       NestedField(field_id=1, name='data', field_type=LongType()),
       schema_id=1
   )
   schema2 = Schema(
       NestedField(field_id=1, name='data', field_type=LongType()),
       schema_id=1
   )
   
   table = catalog.create_table(identifier='test', schema=schema1)
   table.set_schema(schema2).commit()
   ```
   
   It's clear that the user's intent was to drop the `id` field. But what they actually requested was to drop `data` (id 2 assigned during table create), rename `id` to `data` (id 1), and promote `id` from int to long. All of those changes are perfectly valid, but they're not what the user was trying to do.



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

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

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


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


[GitHub] [iceberg] Fokko merged pull request #6323: Python: Alter table plumbing and REST support

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


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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")

Review Comment:
   I've removed it. Less is more



##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   That's interesting. We have multiple `ref`'s in the Open-API Spec: https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1567-L1576
   
   This is actually not yet supported: https://github.com/OAI/OpenAPI-Specification/issues/417
   
   Updated



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   I think the rest of this PR is about ready, my remaining concern is this API. I don't think that we want to expose methods that directly set schema, partitioning, or other table metadata. Schema is a great example because people are going to really mess up the field IDs. We don't want to encourage that through the API.
   
   For example, this is going to break:
   
   ```python
   catalog = load_catalog(...)
   
   schema1 = Schema(
       NestedField(field_id=0, name='id', field_type=LongType()),
       schema_id=1
   )
   schema2 = Schema(
       NestedField(field_id=0, name='id', field_type=LongType()),
       NestedField(field_id=1, name='data', field_type=StringType()),
       schema_id=1
   )
   
   table = catalog.create_table(identifier='test', schema=schema1)
   table.set_schema(schema2).commit()
   ```
   
   The table schema is incompatible with the second schema because the field IDs are reassigned.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[BaseTableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        print(updates)

Review Comment:
   Great catch, thanks. I don't think we should print this to debug, could be sensitive stuff in there



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_partition_spec(self, spec: PartitionSpec) -> TableUpdates:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_sort_order(self, sort_order: SortOrder) -> TableUpdates:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_properties(self, **updates: str) -> TableUpdates:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def remove_properties(self, *removals: str) -> TableUpdates:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> TableUpdates:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.update_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class TableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(TableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+
+
+class SetCurrentSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(TableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(TableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+    type: Literal["tag", "branch"]
+    snapshot_id: int = Field(alias="snapshot-id")
+    max_age_ref_ms: int = Field(alias="max-ref-age-ms")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+
+
+class RemoveSnapshotsUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(TableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class Requirement(Enum):
+    assert_create = "assert-create"
+    assert_table_uuid = "assert-table-uuid"
+    assert_ref_snapshot_id = "assert-ref-snapshot-id"
+    assert_last_assigned_field_id = "assert-last-assigned-field-id"
+    assert_current_schema_id = "assert-current-schema-id"
+    assert_last_assigned_partition_id = "assert-last-assigned-partition-id"
+    assert_default_spec_id = "assert-default-spec-id"
+    assert_default_sort_order_id = "assert-default-sort-order-id"
+
+
+class TableRequirement(IcebergBaseModel):
+    requirement: Requirement
+    ref: Optional[str] = None
+    uuid: Optional[str] = None
+    snapshot_id: Optional[int] = Field(None, alias="snapshot-id")
+    last_assigned_field_id: Optional[int] = Field(None, alias="last-assigned-field-id")
+    current_schema_id: Optional[int] = Field(None, alias="current-schema-id")
+    last_assigned_partition_id: Optional[int] = Field(None, alias="last-assigned-partition-id")
+    default_spec_id: Optional[int] = Field(None, alias="default-spec-id")
+    default_sort_order_id: Optional[int] = Field(None, alias="default-sort-order-id")
+
+
+class CommitTableRequest(IcebergBaseModel):

Review Comment:
   Thanks! https://github.com/apache/iceberg/commit/71711e18b6a2a01da9c2ca1e7a8b4daa903399c5#diff-02549ca620d020dc9ead80088cc14e311e12a69651fa8d394cd41a4308debb2e



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -323,6 +323,17 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
             NoSuchTableError: If a table with the name does not exist
         """
 
+    @abstractmethod
+    def update_table(self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...]) -> Table:

Review Comment:
   Nice one, thanks!



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   Nice, I can add that. Thanks for pointing out the Java tests. 
   
   I had one follow-up PR on the integration tests: https://github.com/apache/iceberg/pull/7768 Would be nice to get those in a well.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,313 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+    _requirements: Tuple[TableRequirement, ...]
+
+    def __init__(
+        self,
+        table: Table,
+        actions: Optional[Tuple[TableUpdate, ...]] = None,
+        requirements: Optional[Tuple[TableRequirement, ...]] = None,
+    ):
+        self._table = table
+        self._updates = actions or ()
+        self._requirements = requirements or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")

Review Comment:
   Updated it to:
   ```python
   with table.transaction() as transaction:
       transaction.set_properties(abc="def")
   
   assert table.refresh().properties == {"abc": "def"}
   
   with table.transaction() as transaction:
       transaction.remove_properties("abc")
   
   assert table.refresh().properties == {}
   ```
   
   Or without a context manager:
   ```python
   table = table.transaction().set_properties(abc="def").commit_transaction()
   
   assert table.properties == {"abc": "def"}
   
   table = table.transaction().remove_properties("abc").commit_transaction()
   
   assert table.properties == {}
   ```



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:
+        """Updates the table
+
+        Args:
+            identifier (str | Identifier): Namespace identifier
+            updates (Tuple[BaseTableUpdate]): Updates to be applied to the table
+        Raises:
+            NoSuchTableError: If a table with the given identifier does not exist
+        """
+        print(updates)

Review Comment:
   Looks like an accidental print. Should this log at debug?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)

Review Comment:
   Why return a new `AlterTable` instead of just appending to this one?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class RemoveSnapshotsUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class CommitTableRequest(IcebergBaseModel):
+    requirements: List[Any] = Field(default_factory=list)

Review Comment:
   We are definitely going to need requirements in order to get this committed. Requirements are really important to validate that we can make the changes that we are requesting. For example, when updating the schema and have assigned a new field ID, the request _must_ validate that the currently loaded `last-column-id` has not changed using `assert-last-assigned-field-id`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   Looks like this is missing `snapshot-id`, `type` (`branch` or `tag`), and the optional ref settings, like `max-ref-age-ms`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:

Review Comment:
   This looks more like a Transaction to me. Is there a better name?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class RemoveSnapshotsUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class CommitTableRequest(IcebergBaseModel):
+    requirements: List[Any] = Field(default_factory=list)

Review Comment:
   Good point. I didn't want to make the PR even bigger. I've added it again. The code generated from the open-api spec is a bit awkward:
   
   ```python
   class Requirement(Enum):
       assert_create = "assert-create"
       assert_table_uuid = "assert-table-uuid"
       assert_ref_snapshot_id = "assert-ref-snapshot-id"
       assert_last_assigned_field_id = "assert-last-assigned-field-id"
       assert_current_schema_id = "assert-current-schema-id"
       assert_last_assigned_partition_id = "assert-last-assigned-partition-id"
       assert_default_spec_id = "assert-default-spec-id"
       assert_default_sort_order_id = "assert-default-sort-order-id"
   
   
   class TableRequirement(IcebergBaseModel):
       requirement: Requirement
       ref: Optional[str] = None
       uuid: Optional[str] = None
       snapshot_id: Optional[int] = Field(None, alias='snapshot-id')
       last_assigned_field_id: Optional[int] = Field(None, alias='last-assigned-field-id')
       current_schema_id: Optional[int] = Field(None, alias='current-schema-id')
       last_assigned_partition_id: Optional[int] = Field(
           None, alias='last-assigned-partition-id'
       )
       default_spec_id: Optional[int] = Field(None, alias='default-spec-id')
       default_sort_order_id: Optional[int] = Field(None, alias='default-sort-order-id')
   ```
   
   Do we want to change this on the spec level? I'd rather leave this out, fix the spec and then add it.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()
+```
+
+### Update the partition spec
+
+Updates the partition spec that will be applied for all new data that's being added to the table.
 
 ```python
-Table(
-    identifier=('default', 'bids'),
-    metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json',
-    metadata=TableMetadataV2(
-        location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
-        table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'),
-        last_updated_ms=1661847562069,
-        last_column_id=4,
-        schemas=[
-            Schema(
-                NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False),
-                NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False),
-                NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False),
-                NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)),
-                schema_id=1,
-                identifier_field_ids=[])
-        ],
-        current_schema_id=1,
-        partition_specs=[
-            PartitionSpec(
-                PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),))
-        ],
-        default_spec_id=0,
-        last_partition_id=1000,
-        properties={},
-        current_snapshot_id=None,
-        snapshots=[],
-        snapshot_log=[],
-        metadata_log=[],
-        sort_orders=[
-            SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)])
-        ],
-        default_sort_order_id=1,
-        refs={},
-        format_version=2,
-        last_sequence_number=0
+from pyiceberg.partitioning import PartitionField, PartitionSpec
+from pyiceberg.transforms import DayTransform
+
+spec = PartitionSpec(
+    PartitionField(
+        source_id=4, field_id=1000, transform=DayTransform(), name="datetime_day"
     )
 )
+
+table = table.alter().set_partition_spec(spec).commit()
+```
+
+### Update the sort order
+
+Updates the sort order of the table.
+
+```python
+from pyiceberg.table.sorting import SortOrder, SortField
+from pyiceberg.transforms import IdentityTransform
+
+order = SortOrder(SortField(source_id=2, transform=IdentityTransform()))
+
+table = table.alter().set_sort_order(order).commit()
+```
+
+### Update the properties
+
+Add, update and remove properties:
+
+```python
+assert table.properties == {}
+
+table = table.alter().set_properties(abc="def").commit()
+
+assert table.properties == {"abc": "def"}
+
+table = table.alter().unset_properties("abc").commit()

Review Comment:
   In other places, we use `remove` instead of `unset`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()
+```
+
+### Update the partition spec
+
+Updates the partition spec that will be applied for all new data that's being added to the table.
 
 ```python
-Table(
-    identifier=('default', 'bids'),
-    metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json',
-    metadata=TableMetadataV2(
-        location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
-        table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'),
-        last_updated_ms=1661847562069,
-        last_column_id=4,
-        schemas=[
-            Schema(
-                NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False),
-                NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False),
-                NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False),
-                NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)),
-                schema_id=1,
-                identifier_field_ids=[])
-        ],
-        current_schema_id=1,
-        partition_specs=[
-            PartitionSpec(
-                PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),))
-        ],
-        default_spec_id=0,
-        last_partition_id=1000,
-        properties={},
-        current_snapshot_id=None,
-        snapshots=[],
-        snapshot_log=[],
-        metadata_log=[],
-        sort_orders=[
-            SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)])
-        ],
-        default_sort_order_id=1,
-        refs={},
-        format_version=2,
-        last_sequence_number=0
+from pyiceberg.partitioning import PartitionField, PartitionSpec
+from pyiceberg.transforms import DayTransform
+
+spec = PartitionSpec(
+    PartitionField(
+        source_id=4, field_id=1000, transform=DayTransform(), name="datetime_day"
     )
 )
+
+table = table.alter().set_partition_spec(spec).commit()

Review Comment:
   Similar to schema, we want to have an API for updating this instead of a method to replace it.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   Oh, I think you're right. This is the one to set the state of the ref, not to create a new ref.



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

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

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


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


[GitHub] [iceberg] Fokko commented on pull request #6323: Python: Alter table plumbing and REST support

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

   Thank @jackye1995 for chiming in here! 🙏🏻 


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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class RemoveSnapshotsUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(BaseTableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class CommitTableRequest(IcebergBaseModel):
+    requirements: List[Any] = Field(default_factory=list)

Review Comment:
   I think we can at least fix the classes here to be specific types. We can follow up with an update to the spec that makes it generate better classes. Up to you the order.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -191,11 +445,14 @@ def from_metadata(cls, metadata_location: str, properties: Properties = EMPTY_DI
 
         metadata = FromInputFile.table_metadata(file)
 
+        from pyiceberg.catalog.null import NoopCatalog
+
         return cls(
             identifier=("static-table", metadata_location),
             metadata_location=metadata_location,
             metadata=metadata,
             io=load_file_io({**properties, **metadata.properties}),
+            catalog=NoopCatalog("static-table"),

Review Comment:
   I'd rather have the `NoopCatalog` because otherwise, we need to not-None check everywhere we use the catalog.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()

Review Comment:
   We could also add another API next to it that will allow users to easily extend the current schema. `set_schema` will just override the schema as the name implies. It is Python, so people can also script this on top of the `Schema`. I've removed it for now.



##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()
+```
+
+### Update the partition spec
+
+Updates the partition spec that will be applied for all new data that's being added to the table.
 
 ```python
-Table(
-    identifier=('default', 'bids'),
-    metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json',
-    metadata=TableMetadataV2(
-        location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
-        table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'),
-        last_updated_ms=1661847562069,
-        last_column_id=4,
-        schemas=[
-            Schema(
-                NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False),
-                NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False),
-                NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False),
-                NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)),
-                schema_id=1,
-                identifier_field_ids=[])
-        ],
-        current_schema_id=1,
-        partition_specs=[
-            PartitionSpec(
-                PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),))
-        ],
-        default_spec_id=0,
-        last_partition_id=1000,
-        properties={},
-        current_snapshot_id=None,
-        snapshots=[],
-        snapshot_log=[],
-        metadata_log=[],
-        sort_orders=[
-            SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)])
-        ],
-        default_sort_order_id=1,
-        refs={},
-        format_version=2,
-        last_sequence_number=0
+from pyiceberg.partitioning import PartitionField, PartitionSpec
+from pyiceberg.transforms import DayTransform
+
+spec = PartitionSpec(
+    PartitionField(
+        source_id=4, field_id=1000, transform=DayTransform(), name="datetime_day"
     )
 )
+
+table = table.alter().set_partition_spec(spec).commit()

Review Comment:
   Removed.



##########
python/mkdocs/docs/api.md:
##########
@@ -241,52 +146,88 @@ catalog.create_table(
 )
 ```
 
-Which returns a newly created table:
+## Altering the table metadata
+
+Using the Python API you can alter table metadata.
+
+### Update the schema
+
+Add a new field to the table:
+
+```python
+from pyiceberg.schema import Schema
+from pyiceberg.types import (
+    BooleanType,
+    DoubleType,
+    IntegerType,
+    NestedField,
+    StringType,
+    TimestampType,
+)
+
+schema = Schema(
+    NestedField(field_id=1, name="str", field_type=StringType(), required=False),
+    NestedField(field_id=2, name="int", field_type=IntegerType(), required=True),
+    NestedField(field_id=3, name="bool", field_type=BooleanType(), required=False),
+    NestedField(
+        field_id=4, name="datetime", field_type=TimestampType(), required=False
+    ),
+    # Add a new column to the table
+    NestedField(field_id=5, name="double", field_type=DoubleType(), required=False),
+)
+
+table = table.alter().set_schema(schema).commit()
+```
+
+### Update the partition spec
+
+Updates the partition spec that will be applied for all new data that's being added to the table.
 
 ```python
-Table(
-    identifier=('default', 'bids'),
-    metadata_location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids//metadata/00000-c8cd93ab-f784-474d-a167-b1a86b05195f.metadata.json',
-    metadata=TableMetadataV2(
-        location='/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/bids/',
-        table_uuid=UUID('38d4cb39-4945-4bf2-b374-984b5c4984d2'),
-        last_updated_ms=1661847562069,
-        last_column_id=4,
-        schemas=[
-            Schema(
-                NestedField(field_id=1, name='datetime', field_type=TimestampType(), required=False),
-                NestedField(field_id=2, name='bid', field_type=DoubleType(), required=False),
-                NestedField(field_id=3, name='ask', field_type=DoubleType(), required=False),
-                NestedField(field_id=4, name='symbol', field_type=StringType(), required=False)),
-                schema_id=1,
-                identifier_field_ids=[])
-        ],
-        current_schema_id=1,
-        partition_specs=[
-            PartitionSpec(
-                PartitionField(source_id=1, field_id=1000, transform=DayTransform(), name='datetime_day'),))
-        ],
-        default_spec_id=0,
-        last_partition_id=1000,
-        properties={},
-        current_snapshot_id=None,
-        snapshots=[],
-        snapshot_log=[],
-        metadata_log=[],
-        sort_orders=[
-            SortOrder(order_id=1, fields=[SortField(source_id=4, transform=IdentityTransform(), direction=SortDirection.ASC, null_order=NullOrder.NULLS_FIRST)])
-        ],
-        default_sort_order_id=1,
-        refs={},
-        format_version=2,
-        last_sequence_number=0
+from pyiceberg.partitioning import PartitionField, PartitionSpec
+from pyiceberg.transforms import DayTransform
+
+spec = PartitionSpec(
+    PartitionField(
+        source_id=4, field_id=1000, transform=DayTransform(), name="datetime_day"
     )
 )
+
+table = table.alter().set_partition_spec(spec).commit()
+```
+
+### Update the sort order
+
+Updates the sort order of the table.
+
+```python
+from pyiceberg.table.sorting import SortOrder, SortField
+from pyiceberg.transforms import IdentityTransform
+
+order = SortOrder(SortField(source_id=2, transform=IdentityTransform()))
+
+table = table.alter().set_sort_order(order).commit()
+```
+
+### Update the properties
+
+Add, update and remove properties:
+
+```python
+assert table.properties == {}
+
+table = table.alter().set_properties(abc="def").commit()
+
+assert table.properties == {"abc": "def"}
+
+table = table.alter().unset_properties("abc").commit()

Review Comment:
   Done



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):

Review Comment:
   I think keeping this in `__init__.py` makes more sense because otherwise, we would have a lot of circular imports. They are very closely related. Happy to move this if you think it should be separated in another file.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)

Review Comment:
   I believe we had a similar discussion about the `TableScan`, there we return a new one every time. I've changed this to updating in place.



##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)

Review Comment:
   I believe we had a similar discussion about the `TableScan`, there we return a new one every time. I've changed this to updating in place.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:

Review Comment:
   `TableUpdates`?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   Nice, I can add that. That's a good example of where it gets tricky as int/long is a valid promotion. Thanks for pointing out the Java tests. 
   
   I had one follow-up PR on the integration tests: https://github.com/apache/iceberg/pull/7768 Would be nice to get that one in as well.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -323,6 +323,17 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
             NoSuchTableError: If a table with the name does not exist
         """
 
+    @abstractmethod
+    def update_table(self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...]) -> Table:

Review Comment:
   FYI, we're adding a similar method in @nastra's transaction work: https://github.com/apache/iceberg/pull/7569
   
   We probably want to align the naming so maybe this should be `commit_table` to align with the `commit_transaction`.
   
   That work is also updating the request to have an optional identifier, so you could just make the identifier part of the `TableUpdate` in Python.



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

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

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


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


[GitHub] [iceberg] rdblue commented on pull request #6323: Python: Alter table plumbing and REST support

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

   Looks good to me. We can always update the method signature later since this one is internal. I'd merge, but the lock file has conflicts.


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

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

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


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


[GitHub] [iceberg] jackye1995 commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -489,6 +493,39 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def _commit(self, *table_requests: CommitTableRequest) -> CommitTableResponse:

Review Comment:
   +1, given this is called afterwards in `Transaction`, it feels more nature to call it just `commit_table` so we an use `self._table.catalog.commit_table` instead of `self._table.catalog._commit`



##########
python/pyiceberg/table/__init__.py:
##########
@@ -64,25 +67,326 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class Transaction:

Review Comment:
   Do we plan to have another `MiltiTableTransaction` for transactions across tables?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   Looks good now!



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[TableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        self._updates = self._updates + new_updates
+        return self
+
+    def set_table_version(self, format_version: int) -> TableUpdates:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_schema(self, new_schema: Schema) -> TableUpdates:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_partition_spec(self, spec: PartitionSpec) -> TableUpdates:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_sort_order(self, sort_order: SortOrder) -> TableUpdates:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def set_properties(self, **updates: str) -> TableUpdates:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def remove_properties(self, *removals: str) -> TableUpdates:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> TableUpdates:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        raise NotImplementedError("Not yet implemented")
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.update_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class TableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(TableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+
+
+class SetCurrentSchemaUpdate(TableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(TableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(TableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(TableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+    type: Literal["tag", "branch"]
+    snapshot_id: int = Field(alias="snapshot-id")
+    max_age_ref_ms: int = Field(alias="max-ref-age-ms")
+    max_snapshot_age_ms: int = Field(alias="max-snapshot-age-ms")
+    min_snapshots_to_keep: int = Field(alias="min-snapshots-to-keep")
+
+
+class RemoveSnapshotsUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshots
+    snapshot_ids: List[int] = Field(alias="snapshot-ids")
+
+
+class RemoveSnapshotRefUpdate(TableUpdate):
+    action = TableUpdateAction.remove_snapshot_ref
+    ref_name: str = Field(alias="ref-name")
+
+
+class SetLocationUpdate(TableUpdate):
+    action = TableUpdateAction.set_location
+    location: str
+
+
+class SetPropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.set_properties
+    updates: Dict[str, str]
+
+
+class RemovePropertiesUpdate(TableUpdate):
+    action = TableUpdateAction.remove_properties
+    removals: List[str]
+
+
+class Requirement(Enum):
+    assert_create = "assert-create"
+    assert_table_uuid = "assert-table-uuid"
+    assert_ref_snapshot_id = "assert-ref-snapshot-id"
+    assert_last_assigned_field_id = "assert-last-assigned-field-id"
+    assert_current_schema_id = "assert-current-schema-id"
+    assert_last_assigned_partition_id = "assert-last-assigned-partition-id"
+    assert_default_spec_id = "assert-default-spec-id"
+    assert_default_sort_order_id = "assert-default-sort-order-id"
+
+
+class TableRequirement(IcebergBaseModel):

Review Comment:
   I couldn't agree more, updated the spec here: https://github.com/apache/iceberg/pull/7710#issuecomment-1575557040 Already pulled the new version into this PR.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +500,34 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def update_table(self, identifier: Union[str, Identifier], updates: Tuple[TableUpdate, ...]) -> TableResponse:

Review Comment:
   Yes



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,288 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:

Review Comment:
   Nice, I can add that. That's a good example of where it gets tricky as int/long is a valid promotion. Thanks for pointing out the Java tests. 
   
   I had one follow-up PR on the integration tests: https://github.com/apache/iceberg/pull/7768 Would be nice to get those in a well.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +72,313 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class TableUpdates:
+    _table: Table
+    _updates: Tuple[TableUpdate, ...]
+    _requirements: Tuple[TableRequirement, ...]
+
+    def __init__(
+        self,
+        table: Table,
+        actions: Optional[Tuple[TableUpdate, ...]] = None,
+        requirements: Optional[Tuple[TableRequirement, ...]] = None,
+    ):
+        self._table = table
+        self._updates = actions or ()
+        self._requirements = requirements or ()
+
+    def _append_updates(self, *new_updates: TableUpdate) -> TableUpdates:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")

Review Comment:
   Updated it to:
   ```python
   with table.transaction() as transaction:
       transaction.set_properties(abc="def")
   
   assert table.properties == {"abc": "def"}
   
   with table.transaction() as transaction:
       transaction.remove_properties("abc")
   
   assert table.properties == {}
   ```
   
   Or without a context manager:
   ```python
   table = table.transaction().set_properties(abc="def").commit_transaction()
   
   assert table.properties == {"abc": "def"}
   
   table = table.transaction().remove_properties("abc").commit_transaction()
   
   assert table.properties == {}
   ```



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -490,6 +498,27 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def alter_table(self, identifier: Union[str, Identifier], updates: Tuple[BaseTableUpdate, ...]) -> TableResponse:

Review Comment:
   Works for me



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/table/__init__.py:
##########
@@ -69,21 +71,268 @@
     import ray
     from duckdb import DuckDBPyConnection
 
+    from pyiceberg.catalog import Catalog
 
 ALWAYS_TRUE = AlwaysTrue()
 
 
+class AlterTable:
+    _table: Table
+    _updates: Tuple[BaseTableUpdate, ...]
+
+    def __init__(self, table: Table, actions: Optional[Tuple[BaseTableUpdate, ...]] = None):
+        self._table = table
+        self._updates = actions or ()
+
+    def _append_updates(self, *new_updates: BaseTableUpdate) -> AlterTable:
+        """Appends updates to the set of staged updates
+
+        Args:
+            *new_updates: Any new updates
+
+        Raises:
+            ValueError: When the type of update is not unique.
+
+        Returns:
+            A new AlterTable object with the new updates appended
+        """
+        for new_update in new_updates:
+            type_new_update = type(new_update)
+            if any(type(update) == type_new_update for update in self._updates):
+                raise ValueError(f"Updates in a single commit need to be unique, duplicate: {type_new_update}")
+        return AlterTable(self._table, self._updates + new_updates)
+
+    def set_table_version(self, format_version: int) -> AlterTable:
+        """Sets the table to a certain version
+
+        Args:
+            format_version: The newly set version
+
+        Returns:
+            The alter table builder
+        """
+        if format_version not in {1, 2}:
+            raise ValueError(f"Format version not (yet) supported: {format_version}")
+        return self._append_updates(UpgradeFormatVersionUpdate(format_version=format_version))
+
+    def set_schema(self, new_schema: Schema) -> AlterTable:
+        """Set the schema, and updates the current-schema-id
+
+        Args:
+            new_schema: The new schema
+
+        Returns:
+            The alter table builder
+
+        Raises:
+            ValueError: When a schema with the same fields already exists
+        """
+        last_column_id = max(self._table.schema().highest_field_id, new_schema.highest_field_id)
+
+        exists = [schema for schema in self._table.schemas().values() if new_schema.fields == schema.fields]
+        if len(exists) > 0:
+            raise ValueError(f"Schema already exists, schema-id {exists[0].schema_id}")
+
+        return self._append_updates(AddSchemaUpdate(schema_=new_schema, last_column_id=last_column_id), SetCurrentSchemaUpdate())
+
+    def set_partition_spec(self, spec: PartitionSpec) -> AlterTable:
+        """Sets the partition spec, and updates the default-spec-id
+
+        Args:
+            spec: The new partition spec
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddPartitionSpecUpdate(spec=spec), SetDefaultSpecUpdate())
+
+    def set_sort_order(self, sort_order: SortOrder) -> AlterTable:
+        """Sets the sort order, and updates the default-sort-order-id
+
+        Args:
+            sort_order: The new sort order
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(AddSortOrderUpdate(sort_order=sort_order), SetDefaultSortOrderUpdate())
+
+    def set_properties(self, **updates: str) -> AlterTable:
+        """Set properties
+
+        When a property is already set, it will be overwritten
+
+        Args:
+            updates: The properties set on the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetPropertiesUpdate(updates=updates))
+
+    def unset_properties(self, *removals: str) -> AlterTable:
+        """Removes properties
+
+        Args:
+            removals: Properties to be removed
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(RemovePropertiesUpdate(removals=removals))
+
+    def update_location(self, location: str) -> AlterTable:
+        """Sets the new table location
+
+        Args:
+            location: The new location of the table
+
+        Returns:
+            The alter table builder
+        """
+        return self._append_updates(SetLocationUpdate(location=location))
+
+    def commit(self) -> Table:
+        """Commits the changes to the catalog
+
+        Returns:
+            The table with the updates applied
+        """
+        # Strip the catalog name
+        if len(self._updates) > 0:
+            table_response = self._table.catalog.alter_table(self._table.identifier[1:], self._updates)
+            return Table(
+                self._table.identifier,
+                metadata=table_response.metadata,
+                metadata_location=table_response.metadata_location,
+                io=self._table.io,
+                catalog=self._table.catalog,
+            )
+        else:
+            return self._table
+
+
+class TableUpdateAction(Enum):
+    upgrade_format_version = "upgrade-format-version"
+    add_schema = "add-schema"
+    set_current_schema = "set-current-schema"
+    add_spec = "add-spec"
+    set_default_spec = "set-default-spec"
+    add_sort_order = "add-sort-order"
+    set_default_sort_order = "set-default-sort-order"
+    add_snapshot = "add-snapshot"
+    set_snapshot_ref = "set-snapshot-ref"
+    remove_snapshots = "remove-snapshots"
+    remove_snapshot_ref = "remove-snapshot-ref"
+    set_location = "set-location"
+    set_properties = "set-properties"
+    remove_properties = "remove-properties"
+
+
+class BaseTableUpdate(IcebergBaseModel):
+    action: TableUpdateAction
+
+
+class UpgradeFormatVersionUpdate(BaseTableUpdate):
+    action = TableUpdateAction.upgrade_format_version
+    format_version: int = Field(alias="format-version")
+
+
+class AddSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_schema
+    schema_: Schema = Field(alias="schema")
+    last_column_id: int = Field(alias="last-column-id")
+
+
+class SetCurrentSchemaUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_current_schema
+    schema_id: int = Field(
+        alias="schema-id", description="Schema ID to set as current, or -1 to set last added schema", default=-1
+    )
+
+
+class AddPartitionSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_spec
+    spec: PartitionSpec
+
+
+class SetDefaultSpecUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_spec
+    spec_id: int = Field(
+        alias="spec-id", description="Partition spec ID to set as the default, or -1 to set last added spec", default=-1
+    )
+
+
+class AddSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_sort_order
+    sort_order: SortOrder = Field(alias="sort-order")
+
+
+class SetDefaultSortOrderUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_default_sort_order
+    sort_order_id: int = Field(
+        alias="sort-order-id", description="Sort order ID to set as the default, or -1 to set last added sort order", default=-1
+    )
+
+
+class AddSnapshotUpdate(BaseTableUpdate):
+    action = TableUpdateAction.add_snapshot
+    snapshot: Snapshot
+
+
+class SetSnapshotRefUpdate(BaseTableUpdate):
+    action = TableUpdateAction.set_snapshot_ref
+    ref_name: str = Field(alias="ref-name")

Review Comment:
   That's interesting. We have multiple `ref`'s in the Open-API Spec: https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml#L1567-L1576
   
   This is actually not yet supported: https://github.com/OAI/OpenAPI-Specification/issues/417



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -489,6 +493,39 @@ def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: U
 
         return self.load_table(to_identifier)
 
+    def _commit(self, *table_requests: CommitTableRequest) -> CommitTableResponse:

Review Comment:
   Do we want to combine these? Why not have separate `commit_table` and `commit_transaction` methods? Then we don't need the check that only one request is supported. I also like that you could previously pass requirements and updates to a public method. Are you trying to restrict access to those in the "public" API for some reason?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6323: Python: Alter table plumbing and REST support

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


##########
python/mkdocs/docs/api.md:
##########
@@ -200,7 +105,7 @@ table = StaticTable.from_metadata(
 
 ## Create a table
 
-To create a table from a catalog:
+To create a table by passing the table to the catalog:

Review Comment:
   I'm not sure what you mean here by "passing the table to the catalog"?



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

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

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


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


[GitHub] [iceberg] Fokko closed pull request #6323: Python: Alter table for table properties

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko closed pull request #6323: Python: Alter table for table properties
URL: https://github.com/apache/iceberg/pull/6323


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

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

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


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