You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "arminnajafi (via GitHub)" <gi...@apache.org> on 2023/01/23 03:10:43 UTC

[GitHub] [iceberg] arminnajafi opened a new pull request, #6646: Implement Support for DynamoDB Catalog

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

   Fixes #6541


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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Implement Support for DynamoDB Catalog

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


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

Review Comment:
   I am not sure if we need to extract this base class. I agree there are common logic shared across catalogs, but I don't think there is too much in common between Dynamo and Glue catalog implementations. 
   
   I remember we have talked about having a base.py class when doing glue catalog implementation, but I think we ended up not doing that and just added util functions for shared functionalities. Maybe we should continue with that pattern.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   It doesn't seem to work at all:
   ```sh
   ➜  python git:(support-ddb-catalog) time pyiceberg --catalog dynamo list
   ∅
   ```
   
   The hierarchical namespaces are just that you can have nested schemas: `finance.debitors.invoices.unpaid`. The CLI will first check if there is a nested namespace, and then if there are tables. For DynamoDB, you can just check if there are multiple levels (more than one dot), and if this is the case, just return an empty list:
   
   ```python
   tuple_identifier = Catalog.identifier_to_tuple(identifier)
   if len(tuple_identifier) > 1:
       return []
   ```



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "AttributeType": "S",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "AttributeType": "S",
+        },
+    ]
+
+
+def _get_key_schema() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "KeyType": "HASH",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "KeyType": "RANGE",
+        },
+    ]
+
+
+def _get_global_secondary_indexes() -> List[Dict[str, Any]]:
+    return [
+        {
+            "IndexName": DYNAMODB_NAMESPACE_GSI,
+            "KeySchema": [
+                {
+                    "AttributeName": DYNAMODB_COL_NAMESPACE,
+                    "KeyType": "HASH",
+                },
+                {
+                    "AttributeName": DYNAMODB_COL_IDENTIFIER,
+                    "KeyType": "RANGE",
+                },
+            ],
+            "Projection": {
+                "ProjectionType": "KEYS_ONLY",
+            },
+        }
+    ]
+
+
+def _get_namespace_properties(namespace_dict: Dict[str, str]) -> Properties:
+    namespace_properties = {}
+    for key, val in namespace_dict.items():
+        if not key.startswith(PROPERTY_KEY_PREFIX):
+            continue
+
+        namespace_properties[_remove_property_prefix(key)] = val
+    return namespace_properties
+
+
+def _convert_dynamo_item_to_regular_dict(dynamo_json: Dict[str, Any]) -> Dict[str, str]:
+    """
+    Converts a dynamo json to a regular json. Example of a dynamo json:
+    {
+        "AlbumTitle": {
+            "S": "Songs About Life",
+        },
+        "Artist": {
+            "S": "Acme Band",
+        },
+        "SongTitle": {
+            "S": "Happy Day",
+        }
+    }
+
+    Converted to regular json:
+    {
+        "AlbumTitle": "Songs About Life",
+        "Artist": "Acme Band",
+        "SongTitle": "Happy Day"
+    }
+
+    Only "S" and "N" data types are supported since those are the only ones that Iceberg is utilizing.
+    """
+
+    regular_json = {}
+    for column_name, val_dict in dynamo_json.items():
+        keys = list(val_dict.keys())
+        assert len(keys) == 1

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] jackye1995 commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   If that's the case we can raise a follow up PR to address this inconsistency



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   But I updated the logic to `if len(tuple_identifier) > 1:` anyway which I think is more descriptive and accurate to what we had before. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -431,3 +440,114 @@ def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
             Identifier: Namespace identifier
         """
         return Catalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def _check_for_overlap(removals: Optional[Set[str]], updates: Properties) -> None:
+        if updates and removals:
+            overlap = set(removals) & set(updates.keys())
+            if overlap:
+                raise ValueError(f"Updates and deletes have an overlap: {overlap}")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._get_default_warehouse_location(database_name, table_name)
+        return location
+
+    def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+        database_properties = self.load_namespace_properties(database_name)
+        if database_location := database_properties.get(LOCATION):
+            database_location = database_location.rstrip("/")
+            return f"{database_location}/{table_name}"
+
+        if warehouse_path := self.properties.get(WAREHOUSE_LOCATION):
+            warehouse_path = warehouse_path.rstrip("/")
+            return f"{warehouse_path}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    @staticmethod
+    def identifier_to_database(
+        identifier: Union[str, Identifier], err: Union[Type[ValueError], Type[NoSuchNamespaceError]] = ValueError
+    ) -> str:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 1:
+            raise err(f"Invalid database, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0]
+
+    @staticmethod
+    def identifier_to_database_and_table(
+        identifier: Union[str, Identifier],
+        err: Union[Type[ValueError], Type[NoSuchTableError], Type[NoSuchNamespaceError]] = ValueError,
+    ) -> Tuple[str, str]:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 2:
+            raise err(f"Invalid path, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0], tuple_identifier[1]
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Note: This method only logs warning rather than raise exception when encountering file deletion failure
+
+        Args:
+            identifier (str | Identifier): Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        table = self.load_table(identifier)
+        self.drop_table(identifier)
+        io = load_file_io(self.properties, table.metadata_location)
+        metadata = table.metadata
+        manifest_lists_to_delete = set()
+        manifests_to_delete = []
+        for snapshot in metadata.snapshots:
+            manifests_to_delete += snapshot.manifests(io)
+            if snapshot.manifest_list is not None:
+                manifest_lists_to_delete.add(snapshot.manifest_list)
+
+        manifest_paths_to_delete = {manifest.manifest_path for manifest in manifests_to_delete}
+        prev_metadata_files = {log.metadata_file for log in metadata.metadata_log}
+
+        delete_data_files(io, manifests_to_delete)
+        delete_files(io, manifest_paths_to_delete, MANIFEST)
+        delete_files(io, manifest_lists_to_delete, MANIFEST_LIST)
+        delete_files(io, prev_metadata_files, PREVIOUS_METADATA)
+        delete_files(io, {table.metadata_location}, METADATA)
+
+    @staticmethod
+    def _write_metadata(metadata: TableMetadata, io: FileIO, metadata_path: str) -> None:
+        ToOutputFile.table_metadata(metadata, io.new_output(metadata_path))
+
+    @staticmethod
+    def _get_metadata_location(location: str) -> str:
+        return f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+
+    def _get_updated_props_and_update_summary(

Review Comment:
   Thanks for seeing this. Yeah that was my plan too. 
   
   But hive is doing this slightly different:
   
   `hive.py`:
   ```
   if removals:
       for key in removals:
           if key in parameters:
               parameters[key] = None
               removed.add(key)
   ```
   
   `glue.py` and `dynamodb.py`:
   ```
   if removals:
       for key in removals:
           if key in updated_properties:
               updated_properties.pop(key)
               removed.add(key)
   ```
   
   The difference is in `parameters[key] = None` vs `updated_properties.pop(key)` I wasn't sure if this difference was intended or we could consolidate the behavior and refactor hive too.
   
   Hopping @Fokko can clarify 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] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/mkdocs/docs/configuration.md:
##########
@@ -85,3 +85,15 @@ catalog:
   default:
     type: glue
 ```
+
+## DynamoDB Catalog
+
+If you want to use AWS DynamoDB as the catalog, you can use the last two ways to configure the pyiceberg and refer
+[How to configure AWS credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
+to set your AWS account credentials locally.
+
+```yaml
+catalog:
+  default:
+    type: dynamodb

Review Comment:
   Added. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


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

Review Comment:
   Ack. Thanks for letting me know. This `__init__.py` is removed.



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "AttributeType": "S",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "AttributeType": "S",
+        },
+    ]
+
+
+def _get_key_schema() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "KeyType": "HASH",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "KeyType": "RANGE",
+        },
+    ]
+
+
+def _get_global_secondary_indexes() -> List[Dict[str, Any]]:
+    return [
+        {
+            "IndexName": DYNAMODB_NAMESPACE_GSI,
+            "KeySchema": [
+                {
+                    "AttributeName": DYNAMODB_COL_NAMESPACE,
+                    "KeyType": "HASH",
+                },
+                {
+                    "AttributeName": DYNAMODB_COL_IDENTIFIER,
+                    "KeyType": "RANGE",
+                },
+            ],
+            "Projection": {
+                "ProjectionType": "KEYS_ONLY",
+            },
+        }
+    ]
+
+
+def _get_namespace_properties(namespace_dict: Dict[str, str]) -> Properties:
+    namespace_properties = {}

Review Comment:
   Nice. Yeah looks much better. 
   :)
   
   Changed list comprehension to dict comprehension though in your code snippet. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)

Review Comment:
   Added a new try 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] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1106265960


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -461,8 +390,10 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
             List[Identifier]: a List of namespace identifiers
         """
         # Glue does not support hierarchical namespace, therefore return an empty list
-        if namespace:
+        tuple_identifier = Catalog.identifier_to_tuple(namespace)
+        if len(tuple_identifier) > 1:

Review Comment:
   For example:
   If I run `pyiceberg list default`, now I will get all my databases:
   ```
   (pyiceberg-py3.9) (base) ➜  python git:(support-ddb-catalog) pyiceberg list default
   default                                   
   delta_lake_demo                           
   delta_playgrounddb                        
   hudi_playgrounddb                         
   migrate_delta_to_iceberg_demo             
   migrated_to_iceberg_demo                  
   mygluedatabase                            
   mygluedatabase2                           
   ...
   ```
   but previously I got all my tables in `default` database:
   ```
   (pyiceberg-py3.9) (base) ➜  python git:(master) pyiceberg list default
   default.cloudtrail_logs_aws_cloudtrail_logs_481640105715_2dd430ba
   default.cloudtrail_logs_aws_cloudtrail_logs_481640105715_de0525cc
   default.table100                                                 
   default.test2                                                    
   default.test21                                                   
   default.test22                                                   
   default.test23                                                   
   default.test24                                                   
   default.test27                                                   
   default.test28                                                   
   default.test29                                                   
   default.test3     
   ...
   ```



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1104989770


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   I think it is related to CLI's `list` logic:
   https://github.com/apache/iceberg/blob/775b3ce3a7421a9265b3fa7ef11bc31bcf212cf9/python/pyiceberg/cli/console.py#L92-L103
   
   Seems it relies on `catalog.list_namespaces` to return an emptry list or not to determine whether `list_tables` should be called. I think this is the reason that we do not throw exception in `list_namespaces` and just return an empty list in hive and glue.
   Sorry that I did not realize this initially.
   
   



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,

Review Comment:
   I tried to do a function also a variable:
   ```
   ddb_put_and_delete_exceptions: Tuple[Exception, ...] = (
               self.dynamodb.exceptions.ResourceNotFoundException,
               self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
               self.dynamodb.exceptions.TransactionConflictException,
           )
   ```
   
   But mypy fails me:
   ```
   mypy.....................................................................Failed
   - hook id: mypy
   - exit code: 1
   
   python/pyiceberg/catalog/dynamodb.py:358: error: Exception type must be derived from BaseException  [misc]
   ...
   ```
   
   I think it's not compatible somehow. I reverting this change. But please let me know if there is a good way to do this. 



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1088546035


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

Review Comment:
   [doubt] I understand having a `_init_.py` can share some util methods across different test. However, according to #5919, it seems we should not have `_init_.py` in the test packages.
   
   May be we can put these into `conftest.py`?



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=ICEBERG,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")

Review Comment:
   [Doubt] Shall we just return an empty list here just like what [`hive.py`](https://github.com/apache/iceberg/blob/038091f6b65bf63d028af175dbbbc7285815d6be/python/pyiceberg/catalog/hive.py#L476-L485) do?  



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,

Review Comment:
   [Curiosity] I am curious about the reason behind this exception catch. I think these errors are dynamodb internal and not related to user input/output for `drop_table`. So what's the difference between re-throwing them as a `GenericDynamoDbError` and just not catching them at all?
   
   I am asking this because I simply ignore these exceptinos in the glueCatalog implementations. May be a good lesson for me to learn.



##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -431,3 +440,114 @@ def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
             Identifier: Namespace identifier
         """
         return Catalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def _check_for_overlap(removals: Optional[Set[str]], updates: Properties) -> None:
+        if updates and removals:
+            overlap = set(removals) & set(updates.keys())
+            if overlap:
+                raise ValueError(f"Updates and deletes have an overlap: {overlap}")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._get_default_warehouse_location(database_name, table_name)
+        return location
+
+    def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+        database_properties = self.load_namespace_properties(database_name)
+        if database_location := database_properties.get(LOCATION):
+            database_location = database_location.rstrip("/")
+            return f"{database_location}/{table_name}"
+
+        if warehouse_path := self.properties.get(WAREHOUSE_LOCATION):
+            warehouse_path = warehouse_path.rstrip("/")
+            return f"{warehouse_path}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    @staticmethod
+    def identifier_to_database(
+        identifier: Union[str, Identifier], err: Union[Type[ValueError], Type[NoSuchNamespaceError]] = ValueError
+    ) -> str:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 1:
+            raise err(f"Invalid database, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0]
+
+    @staticmethod
+    def identifier_to_database_and_table(
+        identifier: Union[str, Identifier],
+        err: Union[Type[ValueError], Type[NoSuchTableError], Type[NoSuchNamespaceError]] = ValueError,
+    ) -> Tuple[str, str]:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 2:
+            raise err(f"Invalid path, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0], tuple_identifier[1]
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Note: This method only logs warning rather than raise exception when encountering file deletion failure
+
+        Args:
+            identifier (str | Identifier): Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        table = self.load_table(identifier)
+        self.drop_table(identifier)
+        io = load_file_io(self.properties, table.metadata_location)
+        metadata = table.metadata
+        manifest_lists_to_delete = set()
+        manifests_to_delete = []
+        for snapshot in metadata.snapshots:
+            manifests_to_delete += snapshot.manifests(io)
+            if snapshot.manifest_list is not None:
+                manifest_lists_to_delete.add(snapshot.manifest_list)
+
+        manifest_paths_to_delete = {manifest.manifest_path for manifest in manifests_to_delete}
+        prev_metadata_files = {log.metadata_file for log in metadata.metadata_log}
+
+        delete_data_files(io, manifests_to_delete)
+        delete_files(io, manifest_paths_to_delete, MANIFEST)
+        delete_files(io, manifest_lists_to_delete, MANIFEST_LIST)
+        delete_files(io, prev_metadata_files, PREVIOUS_METADATA)
+        delete_files(io, {table.metadata_location}, METADATA)
+
+    @staticmethod
+    def _write_metadata(metadata: TableMetadata, io: FileIO, metadata_path: str) -> None:
+        ToOutputFile.table_metadata(metadata, io.new_output(metadata_path))
+
+    @staticmethod
+    def _get_metadata_location(location: str) -> str:
+        return f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+
+    def _get_updated_props_and_update_summary(

Review Comment:
   [Doubt] May be this method can also be utilized by `update_namespace_properties` in `hive.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] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,

Review Comment:
   Yeah this is something that we can discuss as a team and decide what's the best behavior we want to provide. 
   Generally I think it's a good idea for an app to catch all its raw exceptions and re-throw with its own format. Sometimes it's needed to change the format or add more details or sometimes even to hide the underlying raw exception from the end user. 
   
   But in this case I just wanted to be explicit about all of the exception that each of these functions throw, re-throw them as our own exception for now, and and decide later as a team what we want to do with them. 
   
   
   



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/mkdocs/docs/configuration.md:
##########
@@ -85,3 +85,16 @@ catalog:
   default:
     type: glue
 ```
+
+## DynamoDB Catalog
+
+If you want to use AWS DynamoDB as the catalog, you can use the last two ways to configure the pyiceberg and refer
+[How to configure AWS credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
+to set your AWS account credentials locally.
+
+```yaml
+catalog:
+  default:
+    type: dynamodb
+    dynamodb_table_name: iceberg

Review Comment:
   Of course. Good point. Sorry for the miss. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation

Review Comment:
   Done. I though it's more readable that way but if I had to add `# noinspection PyDictCreation` for it, it's probably not a good sign.  



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Implement Support for DynamoDB Catalog

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


##########
python/mkdocs/docs/configuration.md:
##########
@@ -85,3 +85,15 @@ catalog:
   default:
     type: glue
 ```
+
+## DynamoDB Catalog
+
+If you want to use AWS DynamoDB as the catalog, you can use the last two ways to configure the pyiceberg and refer
+[How to configure AWS credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
+to set your AWS account credentials locally.
+
+```yaml
+catalog:
+  default:
+    type: dynamodb

Review Comment:
   I think table name is required for dynamoDB 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 #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   The error appeared when `time pyiceberg --catalog dynamo list nyc` was run. It looks good now, thanks for fixing this 👍🏻 



##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   The error appeared when `pyiceberg --catalog dynamo list nyc` was run. It looks good now, thanks for fixing this 👍🏻 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -431,3 +440,114 @@ def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
             Identifier: Namespace identifier
         """
         return Catalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def _check_for_overlap(removals: Optional[Set[str]], updates: Properties) -> None:
+        if updates and removals:
+            overlap = set(removals) & set(updates.keys())
+            if overlap:
+                raise ValueError(f"Updates and deletes have an overlap: {overlap}")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._get_default_warehouse_location(database_name, table_name)
+        return location
+
+    def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+        database_properties = self.load_namespace_properties(database_name)
+        if database_location := database_properties.get(LOCATION):
+            database_location = database_location.rstrip("/")
+            return f"{database_location}/{table_name}"
+
+        if warehouse_path := self.properties.get(WAREHOUSE_LOCATION):
+            warehouse_path = warehouse_path.rstrip("/")
+            return f"{warehouse_path}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    @staticmethod
+    def identifier_to_database(
+        identifier: Union[str, Identifier], err: Union[Type[ValueError], Type[NoSuchNamespaceError]] = ValueError
+    ) -> str:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 1:
+            raise err(f"Invalid database, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0]
+
+    @staticmethod
+    def identifier_to_database_and_table(
+        identifier: Union[str, Identifier],
+        err: Union[Type[ValueError], Type[NoSuchTableError], Type[NoSuchNamespaceError]] = ValueError,
+    ) -> Tuple[str, str]:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 2:
+            raise err(f"Invalid path, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0], tuple_identifier[1]
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Note: This method only logs warning rather than raise exception when encountering file deletion failure
+
+        Args:
+            identifier (str | Identifier): Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        table = self.load_table(identifier)
+        self.drop_table(identifier)
+        io = load_file_io(self.properties, table.metadata_location)
+        metadata = table.metadata
+        manifest_lists_to_delete = set()
+        manifests_to_delete = []
+        for snapshot in metadata.snapshots:
+            manifests_to_delete += snapshot.manifests(io)
+            if snapshot.manifest_list is not None:
+                manifest_lists_to_delete.add(snapshot.manifest_list)
+
+        manifest_paths_to_delete = {manifest.manifest_path for manifest in manifests_to_delete}
+        prev_metadata_files = {log.metadata_file for log in metadata.metadata_log}
+
+        delete_data_files(io, manifests_to_delete)
+        delete_files(io, manifest_paths_to_delete, MANIFEST)
+        delete_files(io, manifest_lists_to_delete, MANIFEST_LIST)
+        delete_files(io, prev_metadata_files, PREVIOUS_METADATA)
+        delete_files(io, {table.metadata_location}, METADATA)
+
+    @staticmethod
+    def _write_metadata(metadata: TableMetadata, io: FileIO, metadata_path: str) -> None:
+        ToOutputFile.table_metadata(metadata, io.new_output(metadata_path))
+
+    @staticmethod
+    def _get_metadata_location(location: str) -> str:
+        return f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+
+    def _get_updated_props_and_update_summary(

Review Comment:
   Thanks for looking into this with such a close eye @arminnajafi. I believe this is a requirement of Hive, but I need to double-check. Without setting it to null explicitly, the property is ignored.



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1104989770


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   I think the reason to return an empty list in `list_namespaces` is related to CLI's `list` logic:
   https://github.com/apache/iceberg/blob/775b3ce3a7421a9265b3fa7ef11bc31bcf212cf9/python/pyiceberg/cli/console.py#L92-L103
   
   Seems it relies on `catalog.list_namespaces` to return an emptry list or not to determine whether `list_tables` should be called. I think this is the reason that we do not throw exception in `list_namespaces` and just return an empty list in hive and glue.
   Sorry that I did not realize this initially.
   
   



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   @JonasJ-ap 
   Great! Thanks. It makes sense now. 
   
   
   
   



##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   @Fokko 
   Not sure what happened for you there but I wasn't able to reproduce your issue. It seemed to work for me:
   ```
   (pyiceberg-py3.10) ➜  python git:(support-ddb-catalog) ✗ time pyiceberg --catalog dynamo list
   nyc
   pyiceberg --catalog dynamo list  1.25s user 1.91s system 243% cpu 1.299 total
   (pyiceberg-py3.10) ➜  python git:(support-ddb-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] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/tests/catalog/integration_test_dynamodb.py:
##########
@@ -0,0 +1,258 @@
+#  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 Generator, List
+
+import boto3
+import pytest
+from botocore.exceptions import ClientError
+
+from pyiceberg.catalog import Catalog
+from pyiceberg.catalog.dynamodb import DynamoDbCatalog
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.schema import Schema
+from tests.conftest import clean_up, get_bucket_name, get_s3_path
+
+# The number of tables/databases used in list_table/namespace test
+LIST_TEST_NUMBER = 2
+
+
+@pytest.fixture(name="dynamodb", scope="module")
+def fixture_dynamodb_client() -> boto3.client:
+    yield boto3.client("dynamodb")
+
+
+@pytest.fixture(name="test_catalog", scope="module")
+def fixture_test_catalog() -> Generator[Catalog, None, None]:
+    """The pre- and post-setting of aws integration test"""
+    test_catalog = DynamoDbCatalog("test_ddb_catalog", warehouse=get_s3_path(get_bucket_name()))
+    yield test_catalog
+    clean_up(test_catalog)
+
+
+def test_create_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested, get_s3_path(get_bucket_name(), database_name, table_name))
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_location(table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog_no_warehouse = DynamoDbCatalog("test_ddb_catalog")
+    test_catalog_no_warehouse.create_namespace(database_name)
+    with pytest.raises(ValueError):
+        test_catalog_no_warehouse.create_table(identifier, table_schema_nested)
+    test_catalog_no_warehouse.drop_namespace(database_name)
+
+
+def test_create_table_with_default_location(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_database(test_catalog: Catalog, table_schema_nested: Schema, table_name: str) -> None:
+    identifier = ("invalid", table_name)
+    with pytest.raises(NoSuchNamespaceError):
+        test_catalog.create_table(identifier, table_schema_nested)
+
+
+def test_create_duplicated_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table((database_name, table_name), table_schema_nested)
+    with pytest.raises(TableAlreadyExistsError):
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+
+
+def test_load_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    loaded_table = test_catalog.load_table(identifier)
+    assert table.identifier == loaded_table.identifier
+    assert table.metadata_location == loaded_table.metadata_location
+    assert table.metadata == loaded_table.metadata
+
+
+def test_list_tables(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_list: List[str]) -> None:
+    test_catalog.create_namespace(database_name)
+    for table_name in table_list:
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+    identifier_list = test_catalog.list_tables(database_name)
+    assert len(identifier_list) == LIST_TEST_NUMBER
+    for table_name in table_list:
+        assert (database_name, table_name) in identifier_list
+
+
+def test_rename_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    new_database_name = f"{database_name}_new"
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_namespace(new_database_name)
+    new_table_name = f"rename-{table_name}"
+    identifier = (database_name, table_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    new_identifier = (new_database_name, new_table_name)
+    test_catalog.rename_table(identifier, new_identifier)
+    new_table = test_catalog.load_table(new_identifier)
+    assert new_table.identifier == new_identifier
+    assert new_table.metadata_location == table.metadata_location
+    metadata_location = new_table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_drop_table(test_catalog: Catalog, table_schema_nested: Schema, table_name: str, database_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    test_catalog.drop_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_purge_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    test_catalog.purge_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+    with pytest.raises(ClientError):
+        s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    assert (database_name,) in test_catalog.list_namespaces()
+
+
+def test_create_duplicate_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    with pytest.raises(NamespaceAlreadyExistsError):
+        test_catalog.create_namespace(database_name)
+
+
+def test_create_namespace_with_comment_and_location(test_catalog: Catalog, database_name: str) -> None:
+    test_location = get_s3_path(get_bucket_name(), database_name)
+    test_properties = {
+        "comment": "this is a test description",
+        "location": test_location,
+    }
+    test_catalog.create_namespace(namespace=database_name, properties=test_properties)
+    loaded_database_list = test_catalog.list_namespaces()
+    assert (database_name,) in loaded_database_list
+    properties = test_catalog.load_namespace_properties(database_name)
+    assert properties["comment"] == "this is a test description"
+    assert properties["location"] == test_location
+
+
+def test_list_namespaces(test_catalog: Catalog, database_list: List[str]) -> None:
+    for database_name in database_list:
+        test_catalog.create_namespace(database_name)
+    db_list = test_catalog.list_namespaces()
+    for database_name in database_list:
+        assert (database_name,) in db_list
+
+    with pytest.raises(ValidationError, match="This API is not supported for hierarchical namespaces."):

Review Comment:
   Good catch. Fixed. I completely missed checking integ tests and GitHub actions didn't run them either.



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   If I add that, it won't be consistent with Glue and Hive catalogs. 
   Is that intentional?



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,

Review Comment:
   I tried to doa function, also a variable:
   ```
   ddb_put_and_delete_exceptions: Tuple[Exception, ...] = (
               self.dynamodb.exceptions.ResourceNotFoundException,
               self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
               self.dynamodb.exceptions.TransactionConflictException,
           )
   ```
   
   But mypy fails me:
   ```
   mypy.....................................................................Failed
   - hook id: mypy
   - exit code: 1
   
   python/pyiceberg/catalog/dynamodb.py:358: error: Exception type must be derived from BaseException  [misc]
   ...
   ```
   
   I think it's not compatible somehow. I reverting this change. But please let me know if there is a good way to do this. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


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

Review Comment:
   Removed `BaseAwsCatalog` class. I moved things around a bit to still remove duplicate code. There is some duplicate code left in `create_table()` but I think it's fine. 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/mkdocs/docs/configuration.md:
##########
@@ -85,3 +85,16 @@ catalog:
   default:
     type: glue
 ```
+
+## DynamoDB Catalog
+
+If you want to use AWS DynamoDB as the catalog, you can use the last two ways to configure the pyiceberg and refer
+[How to configure AWS credentials](https://docs.aws.amazon.com/cli/latest/userguide/cli-chap-configure.html)
+to set your AWS account credentials locally.
+
+```yaml
+catalog:
+  default:
+    type: dynamodb
+    dynamodb_table_name: iceberg

Review Comment:
   For the rest of the configuration we tend to use dashes instead of underscores:
   ```suggestion
       table-name: iceberg
   ```
   
   Also, this makes it in line with Java: https://iceberg.apache.org/docs/latest/aws/#dynamodb-catalog



##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -93,17 +101,28 @@ def load_glue(name: str, conf: Properties) -> Catalog:
         raise NotInstalledError("AWS glue support not installed: pip install 'pyiceberg[glue]'") from exc
 
 
+def load_dynamodb(name: str, conf: Properties) -> Catalog:
+    try:
+        from pyiceberg.catalog.dynamodb import DynamoDbCatalog
+
+        return DynamoDbCatalog(name, **conf)
+    except ImportError as exc:
+        raise NotInstalledError("AWS DynamoDB support not installed: pip install 'pyiceberg[dynamodb]'") from exc
+
+
 AVAILABLE_CATALOGS: dict[CatalogType, Callable[[str, Properties], Catalog]] = {
     CatalogType.REST: load_rest,
     CatalogType.HIVE: load_hive,
     CatalogType.GLUE: load_glue,
+    CatalogType.DYNAMODB: load_dynamodb,
 }
 
 
 def infer_catalog_type(name: str, catalog_properties: RecursiveDict) -> Optional[CatalogType]:
     """Tries to infer the type based on the dict
 
     Args:
+        name:

Review Comment:
   ```suggestion
           name: Name of the catalog
   ```



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)

Review Comment:
   Nit: Should we return this directly?



##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   I think we have to fix the exception for hierarchical namespaces, so it falls over to listing the nyc namespace:
   ```
   ➜  python git:(support-ddb-catalog) ✗ pyiceberg --catalog dynamo list  
   nyc
   ➜  python git:(support-ddb-catalog) ✗ pyiceberg --catalog dynamo list nyc
   This API is not supported for hierarchical namespaces.
   ➜  python git:(support-ddb-catalog) ✗ pyiceberg --catalog dynamo describe nyc.taxis
   Table format version  1                                                                                                                                      
   Metadata location     s3://emr-spark-and-iceberg/mywarehouse/nyc.db/taxis/metadata/00002-8b2a348f-e940-4d8d-a6fe-8786bd67b353.metadata.json                  
   Table UUID            2b066a47-3d03-47d6-90b0-9d3099df2fab                                                                                                   
   Last Updated          1639172467515   
   ```



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,

Review Comment:
   I agree that it is best to avoid throwing implementation-specific exceptions.



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:

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] arminnajafi commented on pull request #6646: Python: Implement Support for DynamoDB Catalog

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

   Thanks for the review @Fokko !
   
   I addressed most of the comments in the new commits but also asked a few questions.
   Let me know whenever you get a chance to go over them. 
   It might be easier to discuss some of them in Slack. 
   
   


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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -431,3 +440,114 @@ def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
             Identifier: Namespace identifier
         """
         return Catalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def _check_for_overlap(removals: Optional[Set[str]], updates: Properties) -> None:
+        if updates and removals:
+            overlap = set(removals) & set(updates.keys())
+            if overlap:
+                raise ValueError(f"Updates and deletes have an overlap: {overlap}")
+
+    def _resolve_table_location(self, location: Optional[str], database_name: str, table_name: str) -> str:
+        if not location:
+            return self._get_default_warehouse_location(database_name, table_name)
+        return location
+
+    def _get_default_warehouse_location(self, database_name: str, table_name: str) -> str:
+        database_properties = self.load_namespace_properties(database_name)
+        if database_location := database_properties.get(LOCATION):
+            database_location = database_location.rstrip("/")
+            return f"{database_location}/{table_name}"
+
+        if warehouse_path := self.properties.get(WAREHOUSE_LOCATION):
+            warehouse_path = warehouse_path.rstrip("/")
+            return f"{warehouse_path}/{database_name}.db/{table_name}"
+
+        raise ValueError("No default path is set, please specify a location when creating a table")
+
+    @staticmethod
+    def identifier_to_database(
+        identifier: Union[str, Identifier], err: Union[Type[ValueError], Type[NoSuchNamespaceError]] = ValueError
+    ) -> str:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 1:
+            raise err(f"Invalid database, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0]
+
+    @staticmethod
+    def identifier_to_database_and_table(
+        identifier: Union[str, Identifier],
+        err: Union[Type[ValueError], Type[NoSuchTableError], Type[NoSuchNamespaceError]] = ValueError,
+    ) -> Tuple[str, str]:
+        tuple_identifier = Catalog.identifier_to_tuple(identifier)
+        if len(tuple_identifier) != 2:
+            raise err(f"Invalid path, hierarchical namespaces are not supported: {identifier}")
+
+        return tuple_identifier[0], tuple_identifier[1]
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Note: This method only logs warning rather than raise exception when encountering file deletion failure
+
+        Args:
+            identifier (str | Identifier): Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        table = self.load_table(identifier)
+        self.drop_table(identifier)
+        io = load_file_io(self.properties, table.metadata_location)
+        metadata = table.metadata
+        manifest_lists_to_delete = set()
+        manifests_to_delete = []
+        for snapshot in metadata.snapshots:
+            manifests_to_delete += snapshot.manifests(io)
+            if snapshot.manifest_list is not None:
+                manifest_lists_to_delete.add(snapshot.manifest_list)
+
+        manifest_paths_to_delete = {manifest.manifest_path for manifest in manifests_to_delete}
+        prev_metadata_files = {log.metadata_file for log in metadata.metadata_log}
+
+        delete_data_files(io, manifests_to_delete)
+        delete_files(io, manifest_paths_to_delete, MANIFEST)
+        delete_files(io, manifest_lists_to_delete, MANIFEST_LIST)
+        delete_files(io, prev_metadata_files, PREVIOUS_METADATA)
+        delete_files(io, {table.metadata_location}, METADATA)
+
+    @staticmethod
+    def _write_metadata(metadata: TableMetadata, io: FileIO, metadata_path: str) -> None:
+        ToOutputFile.table_metadata(metadata, io.new_output(metadata_path))
+
+    @staticmethod
+    def _get_metadata_location(location: str) -> str:
+        return f"{location}/metadata/00000-{uuid.uuid4()}.metadata.json"
+
+    def _get_updated_props_and_update_summary(

Review Comment:
   So I keep it as is fro now and resolve the thread. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=ICEBERG,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")

Review Comment:
   Discussion here:
   https://github.com/apache/iceberg/pull/6646#discussion_r1097947897



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   I added this for dynamo but it doesn't seem to be consistent with Glue and Hive. 
   Please let me know if those are supposed to be different. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/exceptions.py:
##########
@@ -90,3 +90,19 @@ class SignError(Exception):
 
 class ResolveError(Exception):
     pass
+
+
+class DynamoDbError(Exception):
+    pass
+
+
+class ItemNotFound(DynamoDbError):

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] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1104989770


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   I think it is related to CLI's `list` logic:
   https://github.com/apache/iceberg/blob/775b3ce3a7421a9265b3fa7ef11bc31bcf212cf9/python/pyiceberg/cli/console.py#L92-L103
   
   Seems it relies on `catalog.list_namespaces` to return an emptry list or not to determine whether `list_tables` should be called. I think this is the reason that we do not throw exception in `list_namespaces` in hive and glue.
   Sorry that I did not realize this initially.
   
   



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1106265960


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -461,8 +390,10 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
             List[Identifier]: a List of namespace identifiers
         """
         # Glue does not support hierarchical namespace, therefore return an empty list
-        if namespace:
+        tuple_identifier = Catalog.identifier_to_tuple(namespace)
+        if len(tuple_identifier) > 1:

Review Comment:
   For example:
   If I run `pyiceberg list default`, now I will get all my databases:
   ```
   (pyiceberg-py3.9) (base) ➜  python git:(support-ddb-catalog) pyiceberg list default
   default                                   
   delta_lake_demo                           
   delta_playgrounddb                        
   hudi_playgrounddb                         
   migrate_delta_to_iceberg_demo             
   migrated_to_iceberg_demo                  
   mygluedatabase                            
   mygluedatabase2                           
   ...
   ```
   but previously I will get all my tables in `default` database:
   ```
   (pyiceberg-py3.9) (base) ➜  python git:(master) pyiceberg list default
   default.cloudtrail_logs_aws_cloudtrail_logs_481640105715_2dd430ba
   default.cloudtrail_logs_aws_cloudtrail_logs_481640105715_de0525cc
   default.table100                                                 
   default.test2                                                    
   default.test21                                                   
   default.test22                                                   
   default.test23                                                   
   default.test24                                                   
   default.test27                                                   
   default.test28                                                   
   default.test29                                                   
   default.test3     
   ...
   ```



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   Cool. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/tests/catalog/integration_test_dynamodb.py:
##########
@@ -0,0 +1,258 @@
+#  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 Generator, List
+
+import boto3
+import pytest
+from botocore.exceptions import ClientError
+
+from pyiceberg.catalog import Catalog
+from pyiceberg.catalog.dynamodb import DynamoDbCatalog
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.schema import Schema
+from tests.conftest import clean_up, get_bucket_name, get_s3_path
+
+# The number of tables/databases used in list_table/namespace test
+LIST_TEST_NUMBER = 2
+
+
+@pytest.fixture(name="dynamodb", scope="module")
+def fixture_dynamodb_client() -> boto3.client:
+    yield boto3.client("dynamodb")
+
+
+@pytest.fixture(name="test_catalog", scope="module")
+def fixture_test_catalog() -> Generator[Catalog, None, None]:
+    """The pre- and post-setting of aws integration test"""
+    test_catalog = DynamoDbCatalog("test_ddb_catalog", warehouse=get_s3_path(get_bucket_name()))
+    yield test_catalog
+    clean_up(test_catalog)
+
+
+def test_create_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested, get_s3_path(get_bucket_name(), database_name, table_name))
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_location(table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog_no_warehouse = DynamoDbCatalog("test_ddb_catalog")
+    test_catalog_no_warehouse.create_namespace(database_name)
+    with pytest.raises(ValueError):
+        test_catalog_no_warehouse.create_table(identifier, table_schema_nested)
+    test_catalog_no_warehouse.drop_namespace(database_name)
+
+
+def test_create_table_with_default_location(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_database(test_catalog: Catalog, table_schema_nested: Schema, table_name: str) -> None:
+    identifier = ("invalid", table_name)
+    with pytest.raises(NoSuchNamespaceError):
+        test_catalog.create_table(identifier, table_schema_nested)
+
+
+def test_create_duplicated_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table((database_name, table_name), table_schema_nested)
+    with pytest.raises(TableAlreadyExistsError):
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+
+
+def test_load_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    loaded_table = test_catalog.load_table(identifier)
+    assert table.identifier == loaded_table.identifier
+    assert table.metadata_location == loaded_table.metadata_location
+    assert table.metadata == loaded_table.metadata
+
+
+def test_list_tables(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_list: List[str]) -> None:
+    test_catalog.create_namespace(database_name)
+    for table_name in table_list:
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+    identifier_list = test_catalog.list_tables(database_name)
+    assert len(identifier_list) == LIST_TEST_NUMBER
+    for table_name in table_list:
+        assert (database_name, table_name) in identifier_list
+
+
+def test_rename_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    new_database_name = f"{database_name}_new"
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_namespace(new_database_name)
+    new_table_name = f"rename-{table_name}"
+    identifier = (database_name, table_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    new_identifier = (new_database_name, new_table_name)
+    test_catalog.rename_table(identifier, new_identifier)
+    new_table = test_catalog.load_table(new_identifier)
+    assert new_table.identifier == new_identifier
+    assert new_table.metadata_location == table.metadata_location
+    metadata_location = new_table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_drop_table(test_catalog: Catalog, table_schema_nested: Schema, table_name: str, database_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    test_catalog.drop_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_purge_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    test_catalog.purge_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+    with pytest.raises(ClientError):
+        s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    assert (database_name,) in test_catalog.list_namespaces()
+
+
+def test_create_duplicate_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    with pytest.raises(NamespaceAlreadyExistsError):
+        test_catalog.create_namespace(database_name)
+
+
+def test_create_namespace_with_comment_and_location(test_catalog: Catalog, database_name: str) -> None:
+    test_location = get_s3_path(get_bucket_name(), database_name)
+    test_properties = {
+        "comment": "this is a test description",
+        "location": test_location,
+    }
+    test_catalog.create_namespace(namespace=database_name, properties=test_properties)
+    loaded_database_list = test_catalog.list_namespaces()
+    assert (database_name,) in loaded_database_list
+    properties = test_catalog.load_namespace_properties(database_name)
+    assert properties["comment"] == "this is a test description"
+    assert properties["location"] == test_location
+
+
+def test_list_namespaces(test_catalog: Catalog, database_list: List[str]) -> None:
+    for database_name in database_list:
+        test_catalog.create_namespace(database_name)
+    db_list = test_catalog.list_namespaces()
+    for database_name in database_list:
+        assert (database_name,) in db_list
+
+    with pytest.raises(ValidationError, match="This API is not supported for hierarchical namespaces."):

Review Comment:
   Good catch. Fixed. I completely missed checking integ tests for the previous push and GitHub actions didn't run them either.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Python: Implement Support for DynamoDB Catalog

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

   Merging this, thanks for the awesome work @arminnajafi! And thanks @jackye1995 @JonasJ-ap for the reviews, much appreciated!


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

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

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


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


[GitHub] [iceberg] arminnajafi commented on pull request #6646: Implement Support for DynamoDB Catalog

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

   @JonasJ-ap 
   Thanks for reviewing this PR and also your Glue Catalog PR. This work is mostly based on that 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] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/hive.py:
##########
@@ -548,10 +511,9 @@ def update_namespace_properties(
                 for key, value in updates.items():
                     parameters[key] = value
                     updated.add(key)
+
             open_client.alter_database(database_name, _annotate_namespace(database, parameters))
 
         expected_to_change = (removals or set()).difference(removed)
 
-        return PropertiesUpdateSummary(
-            removed=list(removed or []), updated=list(updates.keys() if updates else []), missing=list(expected_to_change)
-        )
+        return PropertiesUpdateSummary(removed=list(removed or []), updated=list(updated or []), missing=list(expected_to_change))

Review Comment:
   This is done my `make lint`. I changed it to the previous formatting. `make lint` put it back in one line. 
   I didn't find another example like this. Please point them out and I'll fix it. 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)

Review Comment:
   This can throw an exception as well, should we swallow that one?



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return

Review Comment:
   ```suggestion
               return None
   ```



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,

Review Comment:
   Should we generalize these blocks? Since we're catching the same exceptions all the time. WDYT?



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation

Review Comment:
   Just curious, why not initialize the dict right away instead of adding the items one by one?



##########
python/pyiceberg/exceptions.py:
##########
@@ -90,3 +90,19 @@ class SignError(Exception):
 
 class ResolveError(Exception):
     pass
+
+
+class DynamoDbError(Exception):
+    pass
+
+
+class ItemNotFound(DynamoDbError):

Review Comment:
   What do you think of replacing this one with the build in ValueError? It is not being used a lot, and we want to be mindful of introducing new exceptions.



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   This should also include the name of the catalog:
   ```suggestion
               identifier=(self.name, database_name, table_name),
   ```



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:

Review Comment:
   instead of methods, we could make these also constants.



##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "AttributeType": "S",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "AttributeType": "S",
+        },
+    ]
+
+
+def _get_key_schema() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "KeyType": "HASH",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "KeyType": "RANGE",
+        },
+    ]
+
+
+def _get_global_secondary_indexes() -> List[Dict[str, Any]]:
+    return [
+        {
+            "IndexName": DYNAMODB_NAMESPACE_GSI,
+            "KeySchema": [
+                {
+                    "AttributeName": DYNAMODB_COL_NAMESPACE,
+                    "KeyType": "HASH",
+                },
+                {
+                    "AttributeName": DYNAMODB_COL_IDENTIFIER,
+                    "KeyType": "RANGE",
+                },
+            ],
+            "Projection": {
+                "ProjectionType": "KEYS_ONLY",
+            },
+        }
+    ]
+
+
+def _get_namespace_properties(namespace_dict: Dict[str, str]) -> Properties:
+    namespace_properties = {}

Review Comment:
   I'm a bit of a comprehension geek, but what do you think of:
   ```python
   return [
       _remove_property_prefix(key): val
       for key, val in namespace_dict.items()
       if key.startswith(PROPERTY_KEY_PREFIX)
   ]
   ```



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1088569981


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,

Review Comment:
   [Curiosity] I am curious about the reason behind this exception catch. I think these errors are dynamodb internal and not related to user input/output for `drop_table`. So what's the difference between re-throwing them as a `GenericDynamoDbError` and just not catching them at all?
   
    May be a good lesson for me to learn.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),
+            metadata=metadata,
+            metadata_location=metadata_location,
+            io=self._load_file_io(metadata.properties),
+        )
+
+
+def _get_create_table_item(database_name: str, table_name: str, properties: Properties, metadata_location: str) -> Dict[str, Any]:
+
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": f"{database_name}.{table_name}",
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    _dict[_add_property_prefix(TABLE_TYPE)] = {"S": ICEBERG.upper()}
+    _dict[_add_property_prefix(METADATA_LOCATION)] = {"S": metadata_location}
+    _dict[_add_property_prefix(PREVIOUS_METADATA_LOCATION)] = {"S": ""}
+
+    return _dict
+
+
+def _get_rename_table_item(from_dynamo_table_item: Dict[str, Any], to_database_name: str, to_table_name: str) -> Dict[str, Any]:
+    _dict = from_dynamo_table_item
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict[DYNAMODB_COL_IDENTIFIER]["S"] = f"{to_database_name}.{to_table_name}"
+    _dict[DYNAMODB_COL_NAMESPACE]["S"] = to_database_name
+    _dict[DYNAMODB_COL_VERSION]["S"] = str(uuid.uuid4())
+    _dict[DYNAMODB_COL_UPDATED_AT]["N"] = current_timestamp_ms
+    return _dict
+
+
+def _get_create_database_item(database_name: str, properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+    _dict = {
+        DYNAMODB_COL_IDENTIFIER: {
+            "S": DYNAMODB_NAMESPACE,
+        },
+        DYNAMODB_COL_NAMESPACE: {
+            "S": database_name,
+        },
+        DYNAMODB_COL_VERSION: {
+            "S": str(uuid.uuid4()),
+        },
+        DYNAMODB_COL_CREATED_AT: {
+            "N": current_timestamp_ms,
+        },
+        DYNAMODB_COL_UPDATED_AT: {
+            "N": current_timestamp_ms,
+        },
+    }
+
+    for key, val in properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_update_database_item(namespace_item: Dict[str, Any], updated_properties: Properties) -> Dict[str, Any]:
+    current_timestamp_ms = str(round(time() * 1000))
+
+    # noinspection PyDictCreation
+    _dict = {}
+    _dict[DYNAMODB_COL_IDENTIFIER] = namespace_item[DYNAMODB_COL_IDENTIFIER]
+    _dict[DYNAMODB_COL_NAMESPACE] = namespace_item[DYNAMODB_COL_NAMESPACE]
+    _dict[DYNAMODB_COL_VERSION] = {
+        "S": str(uuid.uuid4()),
+    }
+    _dict[DYNAMODB_COL_CREATED_AT] = namespace_item[DYNAMODB_COL_CREATED_AT]
+    _dict[DYNAMODB_COL_UPDATED_AT] = {
+        "N": current_timestamp_ms,
+    }
+
+    for key, val in updated_properties.items():
+        _dict[_add_property_prefix(key)] = {"S": val}
+
+    return _dict
+
+
+def _get_create_catalog_attribute_definitions() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "AttributeType": "S",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "AttributeType": "S",
+        },
+    ]
+
+
+def _get_key_schema() -> List[Dict[str, str]]:
+    return [
+        {
+            "AttributeName": DYNAMODB_COL_IDENTIFIER,
+            "KeyType": "HASH",
+        },
+        {
+            "AttributeName": DYNAMODB_COL_NAMESPACE,
+            "KeyType": "RANGE",
+        },
+    ]
+
+
+def _get_global_secondary_indexes() -> List[Dict[str, Any]]:
+    return [
+        {
+            "IndexName": DYNAMODB_NAMESPACE_GSI,
+            "KeySchema": [
+                {
+                    "AttributeName": DYNAMODB_COL_NAMESPACE,
+                    "KeyType": "HASH",
+                },
+                {
+                    "AttributeName": DYNAMODB_COL_IDENTIFIER,
+                    "KeyType": "RANGE",
+                },
+            ],
+            "Projection": {
+                "ProjectionType": "KEYS_ONLY",
+            },
+        }
+    ]
+
+
+def _get_namespace_properties(namespace_dict: Dict[str, str]) -> Properties:
+    namespace_properties = {}
+    for key, val in namespace_dict.items():
+        if not key.startswith(PROPERTY_KEY_PREFIX):
+            continue
+
+        namespace_properties[_remove_property_prefix(key)] = val
+    return namespace_properties
+
+
+def _convert_dynamo_item_to_regular_dict(dynamo_json: Dict[str, Any]) -> Dict[str, str]:
+    """
+    Converts a dynamo json to a regular json. Example of a dynamo json:
+    {
+        "AlbumTitle": {
+            "S": "Songs About Life",
+        },
+        "Artist": {
+            "S": "Acme Band",
+        },
+        "SongTitle": {
+            "S": "Happy Day",
+        }
+    }
+
+    Converted to regular json:
+    {
+        "AlbumTitle": "Songs About Life",
+        "Artist": "Acme Band",
+        "SongTitle": "Happy Day"
+    }
+
+    Only "S" and "N" data types are supported since those are the only ones that Iceberg is utilizing.
+    """
+
+    regular_json = {}
+    for column_name, val_dict in dynamo_json.items():
+        keys = list(val_dict.keys())
+        assert len(keys) == 1

Review Comment:
   We tend to use assertions only in test code since they can be disabled. Let's throw a ValueError instead.



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   Can you please explain this a little more? I don't think I completely get this hierarchical namespace thing. 
   
   Before that, can please also share the repro steps? How did you load nyc database, your local config, and ...



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)

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] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -461,8 +390,10 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
             List[Identifier]: a List of namespace identifiers
         """
         # Glue does not support hierarchical namespace, therefore return an empty list
-        if namespace:
+        tuple_identifier = Catalog.identifier_to_tuple(namespace)
+        if len(tuple_identifier) > 1:

Review Comment:
   Good catch. Changed it to what it was before:
   ```
   if namespace:
       return []
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Python: Implement Support for DynamoDB Catalog

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


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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1106250202


##########
python/pyiceberg/catalog/glue.py:
##########
@@ -461,8 +390,10 @@ def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identi
             List[Identifier]: a List of namespace identifiers
         """
         # Glue does not support hierarchical namespace, therefore return an empty list
-        if namespace:
+        tuple_identifier = Catalog.identifier_to_tuple(namespace)
+        if len(tuple_identifier) > 1:

Review Comment:
   Sorry but I may not fully understand the change here. Why is the condition `>1` instead of `>0`? 
   
   It seems now if I call `GlueCatalog.list_namespaces("test_database")` I will still get a list of databases instead of an empty list so that the CLI can switch to `list_tables("test_database")` .
   
   Related discussion: https://github.com/apache/iceberg/pull/6646#discussion_r1105510175
   
   If this is intended, currently the `test_list_namespaces` in both `integration_test_dynamodb.py` and `integration_test_glue.py` may need some changes to reflect this.



##########
python/tests/catalog/integration_test_dynamodb.py:
##########
@@ -0,0 +1,258 @@
+#  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 Generator, List
+
+import boto3
+import pytest
+from botocore.exceptions import ClientError
+
+from pyiceberg.catalog import Catalog
+from pyiceberg.catalog.dynamodb import DynamoDbCatalog
+from pyiceberg.exceptions import (
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.schema import Schema
+from tests.conftest import clean_up, get_bucket_name, get_s3_path
+
+# The number of tables/databases used in list_table/namespace test
+LIST_TEST_NUMBER = 2
+
+
+@pytest.fixture(name="dynamodb", scope="module")
+def fixture_dynamodb_client() -> boto3.client:
+    yield boto3.client("dynamodb")
+
+
+@pytest.fixture(name="test_catalog", scope="module")
+def fixture_test_catalog() -> Generator[Catalog, None, None]:
+    """The pre- and post-setting of aws integration test"""
+    test_catalog = DynamoDbCatalog("test_ddb_catalog", warehouse=get_s3_path(get_bucket_name()))
+    yield test_catalog
+    clean_up(test_catalog)
+
+
+def test_create_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested, get_s3_path(get_bucket_name(), database_name, table_name))
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_location(table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog_no_warehouse = DynamoDbCatalog("test_ddb_catalog")
+    test_catalog_no_warehouse.create_namespace(database_name)
+    with pytest.raises(ValueError):
+        test_catalog_no_warehouse.create_table(identifier, table_schema_nested)
+    test_catalog_no_warehouse.drop_namespace(database_name)
+
+
+def test_create_table_with_default_location(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, database_name: str, table_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_table_with_invalid_database(test_catalog: Catalog, table_schema_nested: Schema, table_name: str) -> None:
+    identifier = ("invalid", table_name)
+    with pytest.raises(NoSuchNamespaceError):
+        test_catalog.create_table(identifier, table_schema_nested)
+
+
+def test_create_duplicated_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table((database_name, table_name), table_schema_nested)
+    with pytest.raises(TableAlreadyExistsError):
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+
+
+def test_load_table(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    loaded_table = test_catalog.load_table(identifier)
+    assert table.identifier == loaded_table.identifier
+    assert table.metadata_location == loaded_table.metadata_location
+    assert table.metadata == loaded_table.metadata
+
+
+def test_list_tables(test_catalog: Catalog, table_schema_nested: Schema, database_name: str, table_list: List[str]) -> None:
+    test_catalog.create_namespace(database_name)
+    for table_name in table_list:
+        test_catalog.create_table((database_name, table_name), table_schema_nested)
+    identifier_list = test_catalog.list_tables(database_name)
+    assert len(identifier_list) == LIST_TEST_NUMBER
+    for table_name in table_list:
+        assert (database_name, table_name) in identifier_list
+
+
+def test_rename_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    new_database_name = f"{database_name}_new"
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_namespace(new_database_name)
+    new_table_name = f"rename-{table_name}"
+    identifier = (database_name, table_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    new_identifier = (new_database_name, new_table_name)
+    test_catalog.rename_table(identifier, new_identifier)
+    new_table = test_catalog.load_table(new_identifier)
+    assert new_table.identifier == new_identifier
+    assert new_table.metadata_location == table.metadata_location
+    metadata_location = new_table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_drop_table(test_catalog: Catalog, table_schema_nested: Schema, table_name: str, database_name: str) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    table = test_catalog.create_table(identifier, table_schema_nested)
+    assert table.identifier == identifier
+    test_catalog.drop_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+
+
+def test_purge_table(
+    test_catalog: Catalog, s3: boto3.client, table_schema_nested: Schema, table_name: str, database_name: str
+) -> None:
+    identifier = (database_name, table_name)
+    test_catalog.create_namespace(database_name)
+    test_catalog.create_table(identifier, table_schema_nested)
+    table = test_catalog.load_table(identifier)
+    assert table.identifier == identifier
+    metadata_location = table.metadata_location.split(get_bucket_name())[1][1:]
+    s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+    test_catalog.purge_table(identifier)
+    with pytest.raises(NoSuchTableError):
+        test_catalog.load_table(identifier)
+    with pytest.raises(ClientError):
+        s3.head_object(Bucket=get_bucket_name(), Key=metadata_location)
+
+
+def test_create_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    assert (database_name,) in test_catalog.list_namespaces()
+
+
+def test_create_duplicate_namespace(test_catalog: Catalog, database_name: str) -> None:
+    test_catalog.create_namespace(database_name)
+    with pytest.raises(NamespaceAlreadyExistsError):
+        test_catalog.create_namespace(database_name)
+
+
+def test_create_namespace_with_comment_and_location(test_catalog: Catalog, database_name: str) -> None:
+    test_location = get_s3_path(get_bucket_name(), database_name)
+    test_properties = {
+        "comment": "this is a test description",
+        "location": test_location,
+    }
+    test_catalog.create_namespace(namespace=database_name, properties=test_properties)
+    loaded_database_list = test_catalog.list_namespaces()
+    assert (database_name,) in loaded_database_list
+    properties = test_catalog.load_namespace_properties(database_name)
+    assert properties["comment"] == "this is a test description"
+    assert properties["location"] == test_location
+
+
+def test_list_namespaces(test_catalog: Catalog, database_list: List[str]) -> None:
+    for database_name in database_list:
+        test_catalog.create_namespace(database_name)
+    db_list = test_catalog.list_namespaces()
+    for database_name in database_list:
+        assert (database_name,) in db_list
+
+    with pytest.raises(ValidationError, match="This API is not supported for hierarchical namespaces."):

Review Comment:
   Shall we change this test to accomodate the changes made in `list_namespaces` discussed :https://github.com/apache/iceberg/pull/6646#discussion_r1105510175



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on pull request #6646: Implement Support for DynamoDB Catalog

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

   @Fokko @jackye1995 
   Ready for review


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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/hive.py:
##########
@@ -548,10 +511,9 @@ def update_namespace_properties(
                 for key, value in updates.items():
                     parameters[key] = value
                     updated.add(key)
+
             open_client.alter_database(database_name, _annotate_namespace(database, parameters))
 
         expected_to_change = (removals or set()).difference(removed)
 
-        return PropertiesUpdateSummary(
-            removed=list(removed or []), updated=list(updates.keys() if updates else []), missing=list(expected_to_change)
-        )
+        return PropertiesUpdateSummary(removed=list(removed or []), updated=list(updated or []), missing=list(expected_to_change))

Review Comment:
   I see there are multiple places with this kind of style change and newline additions. If not required by linter, can we avoid these changes?



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1088546035


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

Review Comment:
   [doubt] I understand having a `_init_.py` can share some util methods across different tests. However, according to #5919, it seems we should not have `_init_.py` in the test packages.
   
   May be we can put these into `conftest.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] arminnajafi commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=ICEBERG,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")

Review Comment:
   This is a good point. 
   
   I actually didn't think just returning an empty list would be a good idea. Because in that case if a user sends in a namespace they will just get an empty list. That will imply their dataset does not have any database under their provided namespace which which may not be the truth and without getting any exception from the app saying that this feature is not supported and it's not like the answer to your question is an empty list. 
   
   I like to hear @jackye1995 and @Fokko option on this too and decide whether we want to go with an empty list or throw. 
   
    



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

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

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


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


[GitHub] [iceberg] JonasJ-ap commented on a diff in pull request #6646: Implement Support for DynamoDB Catalog

Posted by "JonasJ-ap (via GitHub)" <gi...@apache.org>.
JonasJ-ap commented on code in PR #6646:
URL: https://github.com/apache/iceberg/pull/6646#discussion_r1088573155


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,733 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.catalog.base_aws_catalog import BaseAwsCatalog
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(BaseAwsCatalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists(name=ICEBERG):
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=ICEBERG,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self, name: str) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {name} is not {ACTIVE}")
+        else:
+            return True
+
+    def _create_table(
+        self, identifier: Union[str, Identifier], table_name: str, metadata_location: str, properties: Properties = EMPTY_DICT
+    ) -> None:
+
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=ICEBERG,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=ICEBERG,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")

Review Comment:
   [Doubt] Shall we just return an empty list here like what [`hive.py`](https://github.com/apache/iceberg/blob/038091f6b65bf63d028af175dbbbc7285815d6be/python/pyiceberg/catalog/hive.py#L476-L485) does?  



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   Done. 
   
   To my knowledge hive and glue are not following this pattern. Please let me know if you want to apple the same change to glue and hive 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] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return

Review Comment:
   I think empty return is the same as return None. But I added it anyway to be more explicit. 



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyproject.toml:
##########
@@ -103,6 +103,7 @@ hive = ["thrift"]
 s3fs = ["s3fs"]
 glue = ["boto3"]
 adlfs = ["adlfs"]
+dynamodb = ["boto3"]

Review Comment:
   In the meanwhile I made it the same as Glue and return an empty list.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def list_tables(self, namespace: Union[str, Identifier]) -> List[Identifier]:
+        """List tables under the given namespace in the catalog (including non-Iceberg tables)
+
+        Args:
+            namespace (str | Identifier): Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                IndexName=DYNAMODB_NAMESPACE_GSI,
+                KeyConditionExpression=f"{DYNAMODB_COL_NAMESPACE} = :namespace ",
+                ExpressionAttributeValues={
+                    ":namespace": {
+                        "S": database_name,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        table_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                identifier_col = _dict[DYNAMODB_COL_IDENTIFIER]
+                if identifier_col == DYNAMODB_NAMESPACE:
+                    continue
+
+                table_identifiers.append(self.identifier_to_tuple(identifier_col))
+
+        return table_identifiers
+
+    def list_namespaces(self, namespace: Union[str, Identifier] = ()) -> List[Identifier]:
+        """
+        List top-level namespaces from the catalog.
+        We do not support hierarchical namespace.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+        if namespace:
+            raise ValidationError("This API is not supported for hierarchical namespaces.")
+
+        paginator = self.dynamodb.get_paginator("query")
+
+        try:
+            page_iterator = paginator.paginate(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                KeyConditionExpression=f"{DYNAMODB_COL_IDENTIFIER} = :identifier",
+                ExpressionAttributeValues={
+                    ":identifier": {
+                        "S": DYNAMODB_NAMESPACE,
+                    }
+                },
+            )
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        database_identifiers = []
+        for page in page_iterator:
+            for item in page["Items"]:
+                _dict = _convert_dynamo_item_to_regular_dict(item)
+                namespace_col = _dict[DYNAMODB_COL_NAMESPACE]
+                database_identifiers.append(self.identifier_to_tuple(namespace_col))
+
+        return database_identifiers
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        """
+        Get properties for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Returns:
+            Properties: Properties for the given namespace
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        return _get_namespace_properties(namespace_dict=namespace_dict)
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Properties = EMPTY_DICT
+    ) -> PropertiesUpdateSummary:
+        """
+        Removes or updates provided property keys for a namespace.
+
+        Args:
+            namespace: Namespace identifier
+            removals: Set of property keys that need to be removed. Optional Argument.
+            updates: Properties to be updated for the given namespace.
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or identifier is invalid
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        namespace_item = self._get_iceberg_namespace_item(database_name=database_name)
+        namespace_dict = _convert_dynamo_item_to_regular_dict(namespace_item)
+        current_properties = _get_namespace_properties(namespace_dict=namespace_dict)
+
+        properties_update_summary, updated_properties = self._get_updated_props_and_update_summary(
+            current_properties=current_properties, removals=removals, updates=updates
+        )
+
+        try:
+            self._put_dynamo_item(
+                item=_get_update_database_item(
+                    namespace_item=namespace_item,
+                    updated_properties=updated_properties,
+                ),
+                condition_expression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database {database_name} does not exist") from e
+
+        return properties_update_summary
+
+    def _get_iceberg_table_item(self, database_name: str, table_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=f"{database_name}.{table_name}", namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+
+    def _get_iceberg_namespace_item(self, database_name: str) -> Dict[str, Any]:
+        try:
+            return self._get_dynamo_item(identifier=DYNAMODB_NAMESPACE, namespace=database_name)
+        except ItemNotFound as e:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {database_name}") from e
+
+    def _ensure_namespace_exists(self, database_name: str) -> Dict[str, Any]:
+        return self._get_iceberg_namespace_item(database_name)
+
+    def _get_dynamo_item(self, identifier: str, namespace: str) -> Dict[str, Any]:
+        try:
+            response = self.dynamodb.get_item(
+                TableName=self.dynamodb_table_name,
+                ConsistentRead=True,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": identifier,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": namespace,
+                    },
+                },
+            )
+            if ITEM in response:
+                return response[ITEM]
+            else:
+                raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}")
+        except self.dynamodb.exceptions.ResourceNotFoundException as e:
+            raise ItemNotFound(f"Item not found. identifier: {identifier} - namespace: {namespace}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _put_dynamo_item(self, item: Dict[str, Any], condition_expression: str) -> None:
+        try:
+            self.dynamodb.put_item(TableName=self.dynamodb_table_name, Item=item, ConditionExpression=condition_expression)
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise ConditionalCheckFailedException(f"Condition expression check failed: {condition_expression} - {item}") from e
+        except (
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _convert_dynamo_table_item_to_iceberg_table(self, dynamo_table_item: Dict[str, Any]) -> Table:
+        table_dict = _convert_dynamo_item_to_regular_dict(dynamo_table_item)
+
+        for prop in [_add_property_prefix(prop) for prop in (TABLE_TYPE, METADATA_LOCATION)] + [
+            DYNAMODB_COL_IDENTIFIER,
+            DYNAMODB_COL_NAMESPACE,
+            DYNAMODB_COL_CREATED_AT,
+        ]:
+            if prop not in table_dict.keys():
+                raise NoSuchPropertyException(f"Iceberg required property {prop} is missing: {dynamo_table_item}")
+
+        table_type = table_dict[_add_property_prefix(TABLE_TYPE)]
+        identifier = table_dict[DYNAMODB_COL_IDENTIFIER]
+        metadata_location = table_dict[_add_property_prefix(METADATA_LOCATION)]
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+
+        if table_type.lower() != ICEBERG:
+            raise NoSuchIcebergTableError(
+                f"Property table_type is {table_type}, expected {ICEBERG}: " f"{database_name}.{table_name}"
+            )
+
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        file = io.new_input(metadata_location)
+        metadata = FromInputFile.table_metadata(file)
+        return Table(
+            identifier=(database_name, table_name),

Review Comment:
   No, this should be the case for all of the catalogs



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

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

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


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


[GitHub] [iceberg] arminnajafi commented on a diff in pull request #6646: Python: Implement Support for DynamoDB Catalog

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


##########
python/pyiceberg/catalog/dynamodb.py:
##########
@@ -0,0 +1,776 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing,
+#  software distributed under the License is distributed on an
+#  "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+#  KIND, either express or implied.  See the License for the
+#  specific language governing permissions and limitations
+#  under the License.
+import uuid
+from time import time
+from typing import (
+    Any,
+    Dict,
+    List,
+    Optional,
+    Set,
+    Union,
+)
+
+import boto3
+
+from pyiceberg.catalog import (
+    ICEBERG,
+    METADATA_LOCATION,
+    PREVIOUS_METADATA_LOCATION,
+    TABLE_TYPE,
+    Catalog,
+    Identifier,
+    Properties,
+    PropertiesUpdateSummary,
+)
+from pyiceberg.exceptions import (
+    ConditionalCheckFailedException,
+    GenericDynamoDbError,
+    ItemNotFound,
+    NamespaceAlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchIcebergTableError,
+    NoSuchNamespaceError,
+    NoSuchPropertyException,
+    NoSuchTableError,
+    TableAlreadyExistsError,
+    ValidationError,
+)
+from pyiceberg.io import load_file_io
+from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
+from pyiceberg.schema import Schema
+from pyiceberg.serializers import FromInputFile
+from pyiceberg.table import Table
+from pyiceberg.table.metadata import new_table_metadata
+from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
+from pyiceberg.typedef import EMPTY_DICT
+
+DYNAMODB_CLIENT = "dynamodb"
+
+DYNAMODB_COL_IDENTIFIER = "identifier"
+DYNAMODB_COL_NAMESPACE = "namespace"
+DYNAMODB_COL_VERSION = "v"
+DYNAMODB_COL_UPDATED_AT = "updated_at"
+DYNAMODB_COL_CREATED_AT = "created_at"
+DYNAMODB_NAMESPACE = "NAMESPACE"
+DYNAMODB_NAMESPACE_GSI = "namespace-identifier"
+DYNAMODB_PAY_PER_REQUEST = "PAY_PER_REQUEST"
+
+DYNAMODB_TABLE_NAME = "dynamodb_table_name"
+DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"
+
+PROPERTY_KEY_PREFIX = "p."
+
+ACTIVE = "ACTIVE"
+ITEM = "Item"
+
+
+class DynamoDbCatalog(Catalog):
+    def __init__(self, name: str, **properties: str):
+        super().__init__(name, **properties)
+        self.dynamodb = boto3.client(DYNAMODB_CLIENT)
+        self.dynamodb_table_name = self.properties.get(DYNAMODB_TABLE_NAME, DYNAMODB_TABLE_NAME_DEFAULT)
+        self._ensure_catalog_table_exists_or_create()
+
+    def _ensure_catalog_table_exists_or_create(self) -> None:
+        if self._dynamodb_table_exists():
+            return
+
+        try:
+            self.dynamodb.create_table(
+                TableName=self.dynamodb_table_name,
+                AttributeDefinitions=_get_create_catalog_attribute_definitions(),
+                KeySchema=_get_key_schema(),
+                GlobalSecondaryIndexes=_get_global_secondary_indexes(),
+                BillingMode=DYNAMODB_PAY_PER_REQUEST,
+            )
+        except (
+            self.dynamodb.exceptions.ResourceInUseException,
+            self.dynamodb.exceptions.LimitExceededException,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def _dynamodb_table_exists(self) -> bool:
+        try:
+            response = self.dynamodb.describe_table(TableName=self.dynamodb_table_name)
+        except self.dynamodb.exceptions.ResourceNotFoundException:
+            return False
+        except self.dynamodb.exceptions.InternalServerError as e:
+            raise GenericDynamoDbError(e.message) from e
+
+        if response["Table"]["TableStatus"] != ACTIVE:
+            raise GenericDynamoDbError(f"DynamoDB table for catalog {self.dynamodb_table_name} is not {ACTIVE}")
+        else:
+            return True
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: PartitionSpec = UNPARTITIONED_PARTITION_SPEC,
+        sort_order: SortOrder = UNSORTED_SORT_ORDER,
+        properties: Properties = EMPTY_DICT,
+    ) -> Table:
+        """
+        Create an Iceberg table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table.
+            sort_order: SortOrder for the table.
+            properties: Table properties that can be a string based dictionary.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+            ValueError: If the identifier is invalid, or no path is given to store metadata
+
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier)
+
+        location = self._resolve_table_location(location, database_name, table_name)
+        metadata_location = self._get_metadata_location(location=location)
+        metadata = new_table_metadata(
+            location=location, schema=schema, partition_spec=partition_spec, sort_order=sort_order, properties=properties
+        )
+        io = load_file_io(properties=self.properties, location=metadata_location)
+        self._write_metadata(metadata, io, metadata_location)
+
+        self._ensure_namespace_exists(database_name=database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_table_item(
+                    database_name=database_name, table_name=table_name, properties=properties, metadata_location=metadata_location
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {database_name}.{table_name} already exists") from e
+
+        loaded_table = self.load_table(identifier=identifier)
+        return loaded_table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        """
+        Loads the table's metadata and returns the table instance.
+
+        You can also use this method to check for table existence using 'try catalog.table() except TableNotFoundError'
+        Note: This method doesn't scan data stored in the table.
+
+        Args:
+            identifier: Table identifier.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        dynamo_table_item = self._get_iceberg_table_item(database_name=database_name, table_name=table_name)
+        return self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=dynamo_table_item)
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            NoSuchTableError: If a table with the name does not exist, or the identifier is invalid
+        """
+        database_name, table_name = self.identifier_to_database_and_table(identifier, NoSuchTableError)
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": f"{database_name}.{table_name}",
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchTableError(f"Table does not exist: {database_name}.{table_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,
+            self.dynamodb.exceptions.ResourceNotFoundException,
+            self.dynamodb.exceptions.ItemCollectionSizeLimitExceededException,
+            self.dynamodb.exceptions.TransactionConflictException,
+            self.dynamodb.exceptions.RequestLimitExceeded,
+            self.dynamodb.exceptions.InternalServerError,
+        ) as e:
+            raise GenericDynamoDbError(e.message) from e
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        This method can only rename Iceberg tables in AWS Glue
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            ValueError: When from table identifier is invalid
+            NoSuchTableError: When a table with the name does not exist
+            NoSuchIcebergTableError: When from table is not a valid iceberg table
+            NoSuchPropertyException: When from table miss some required properties
+            NoSuchNamespaceError: When the destination namespace doesn't exist
+        """
+        from_database_name, from_table_name = self.identifier_to_database_and_table(from_identifier, NoSuchTableError)
+        to_database_name, to_table_name = self.identifier_to_database_and_table(to_identifier)
+
+        from_table_item = self._get_iceberg_table_item(database_name=from_database_name, table_name=from_table_name)
+
+        try:
+            # Verify that from_identifier is a valid iceberg table
+            self._convert_dynamo_table_item_to_iceberg_table(dynamo_table_item=from_table_item)
+        except NoSuchPropertyException as e:
+            raise NoSuchPropertyException(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is missing required properties"
+            ) from e
+        except NoSuchIcebergTableError as e:
+            raise NoSuchIcebergTableError(
+                f"Failed to rename table {from_database_name}.{from_table_name} since it is not a valid iceberg table"
+            ) from e
+
+        self._ensure_namespace_exists(database_name=from_database_name)
+        self._ensure_namespace_exists(database_name=to_database_name)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_rename_table_item(
+                    from_dynamo_table_item=from_table_item, to_database_name=to_database_name, to_table_name=to_table_name
+                ),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_IDENTIFIER})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise TableAlreadyExistsError(f"Table {to_database_name}.{to_table_name} already exists") from e
+
+        try:
+            self.drop_table(from_identifier)
+        except (NoSuchTableError, GenericDynamoDbError) as e:
+            self.drop_table(to_identifier)
+            raise ValueError(
+                f"Failed to drop old table {from_database_name}.{from_table_name}, "
+                f"after renaming to {to_database_name}.{to_table_name}. Rolling back to use the old one."
+            ) from e
+
+        return self.load_table(to_identifier)
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Properties = EMPTY_DICT) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            ValueError: If the identifier is invalid
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+        database_name = self.identifier_to_database(namespace)
+
+        try:
+            self._put_dynamo_item(
+                item=_get_create_database_item(database_name=database_name, properties=properties),
+                condition_expression=f"attribute_not_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except ConditionalCheckFailedException as e:
+            raise NamespaceAlreadyExistsError(f"Database {database_name} already exists") from e
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        A Glue namespace can only be dropped if it is empty
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NoSuchNamespaceError: If a namespace with the given name does not exist, or the identifier is invalid
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+        database_name = self.identifier_to_database(namespace, NoSuchNamespaceError)
+        table_identifiers = self.list_tables(namespace=database_name)
+
+        if len(table_identifiers) > 0:
+            raise NamespaceNotEmptyError(f"Database {database_name} is not empty")
+
+        try:
+            self.dynamodb.delete_item(
+                TableName=self.dynamodb_table_name,
+                Key={
+                    DYNAMODB_COL_IDENTIFIER: {
+                        "S": DYNAMODB_NAMESPACE,
+                    },
+                    DYNAMODB_COL_NAMESPACE: {
+                        "S": database_name,
+                    },
+                },
+                ConditionExpression=f"attribute_exists({DYNAMODB_COL_NAMESPACE})",
+            )
+        except self.dynamodb.exceptions.ConditionalCheckFailedException as e:
+            raise NoSuchNamespaceError(f"Database does not exist: {database_name}") from e
+        except (
+            self.dynamodb.exceptions.ProvisionedThroughputExceededException,

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