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

[GitHub] [iceberg] dhruv-pratap opened a new pull request, #4706: Python: Support iceberg base catalog in python library (#3245)

dhruv-pratap opened a new pull request, #4706:
URL: https://github.com/apache/iceberg/pull/4706

   **WORK IN PROGRESS.** This is a draft pull request just to share the progress with other collaborators who are dependent on this track of work and to get their early feedback.
   
   The current PR has changes for the base Catalog and NamespacedCatalog, with placeholders for Table and PartitionSpec.
   
   This PR also minutely touches issue #3227 i.e., support table in iceberg python library, and #3228 i.e. support partition spec in iceberg python library, as the catalog management is dependent on those classes.
   
   The future changes to this PR would focus on an In-memory Catalog implementation, that is Dictionary based, for testing purposes.


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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872505969


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,444 @@
+#  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 dataclasses import dataclass
+from typing import Dict, List, Optional, Tuple, cast
+
+import pytest
+
+from iceberg.catalog.base import Catalog
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NamespaceNotFoundError,
+    TableNotFoundError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+@dataclass(frozen=True)
+class InMemoryTable(Table):
+    """An in-memory table representation for testing purposes.
+
+    Usage:
+        table_spec = InMemoryTable(
+            namespace = ("com", "organization", "department"),
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+    """
+
+    namespace: Tuple[str, ...]
+    name: str
+    schema: Schema
+    location: str
+    partition_spec: PartitionSpec
+    properties: Dict[str, str]
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Tuple[Tuple[str, ...], str], InMemoryTable]
+    __namespaces: Dict[Tuple[str, ...], Dict[str, str]]
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        if (namespace, name) in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = InMemoryTable(
+                namespace=namespace,
+                name=name,
+                schema=schema if schema else None,
+                location=location if location else None,
+                partition_spec=partition_spec if partition_spec else None,
+                properties=properties if properties else {},
+            )
+            self.__tables[(namespace, name)] = table
+            return table
+
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:
+        try:
+            return self.__tables[(namespace, name)]
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        try:
+            self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        try:
+            table = self.__tables.pop((from_namespace, from_name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {from_name} not found in the catalog")
+
+        renamed_table = InMemoryTable(
+            namespace=to_namespace,
+            name=to_name,
+            schema=table.schema,
+            location=table.location,
+            partition_spec=table.partition_spec,
+            properties=table.properties,
+        )
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[(to_namespace, to_name)] = renamed_table
+        return renamed_table
+
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        try:
+            table = self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+        replaced_table = InMemoryTable(
+            namespace=namespace if namespace else table.namespace,
+            name=name if name else table.name,
+            schema=schema if schema else table.schema,
+            location=location if location else table.location,
+            partition_spec=partition_spec if partition_spec else table.partition_spec,
+            properties={**table.properties, **properties},
+        )
+        self.__tables[(replaced_table.namespace, replaced_table.name)] = replaced_table
+        return replaced_table
+
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        if [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        if namespace:
+            list_tables = [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return cast(List[Tuple[Tuple[str, ...], str]], list_tables)
+
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        if namespace in self.__namespaces:
+            self.__namespaces[namespace] = metadata
+        else:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+
+@pytest.fixture
+def catalog() -> InMemoryCatalog:
+    return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture
+def table_spec() -> InMemoryTable:

Review Comment:
   As suggested above, I've removed the table_spec fixture and just using constants for these test values in my test now.
   
   I've removed the `InMemoryTable` implementation of `Table(ABC)` and trimmed down the InMemoryCatalog implementation to not use it and just use a bland `Table` instance. The only reason `InMemoryTable` existed was because I didn't want to add any table attributes to `Table(ABC)` in this PR to keep it as slim as possible. I had to sacrifice some of the `InMemoryCatalog` functionality around `rename_table()` and `replace_table()` but it still does a decent job of testing the base catalog interface and we can always add that back once we have a base `Table` implementation with all the attributes.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,247 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Properties = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    The catalog table APIs accept a table identifier, which is fully classified table name. The identifier can be a string or
+    tuple of strings. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+    The catalog namespace APIs follow a similar convention wherein they also accept a namespace identifier that can be a string
+    or tuple of strings.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Properties): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Properties):

Review Comment:
   I think using `@dataclass` is a good idea, but we can move on here and refactor it later.



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

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

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


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


[GitHub] [iceberg] rdblue merged pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#issuecomment-1133159898

   @rdblue Are we good to merge?


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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        """Removes provided metadata keys and then updates metadata for a namespace.
+
+        Note: No errors are raised if a provided removal key is not found.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            removals: a set of metadata keys that need to be removed. Optional Argument.
+            updates: a dictionary of properties to be updated for the given namespace. Optional Argument.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist

Review Comment:
   Can you add `ValueError` for when `removals` and `updates` have an overlapping property name?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")

Review Comment:
   This error message should also use the same format that I noted a minute ago: `Table already exists: {identifier}`



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867216868


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   I did start with that initially but thought we would eventually need to add validators for every attribute later on to validate the table spec. That's why went with attrs. I have added a TODO in the code as well to signify the same.
   



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   Rather than copying this everywhere, how about using a simple description and clarifying how it is handled in the class documentation?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876162313


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]

Review Comment:
   @rdblue Given this, should I rename the namespace methods to `load_namespace_properties()` `update_namespace_properties()` to have the same verbiage?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876377912


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,
+        )
+
+
+def test_table(catalog: InMemoryCatalog):

Review Comment:
   Addressed. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table already exists: {identifier}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            table.identifier = identifier
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        try:
+            table = self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {from_identifier}") from error
+
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        table.identifier = to_identifier
+        self.__tables[to_identifier] = table
+        return table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace already exists: {namespace}")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace is not empty: {namespace}")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+NO_SUCH_TABLE_ERROR = "Table does not exist: \\('com', 'organization', 'department', 'my_table'\\)"

Review Comment:
   These should use string names, not tuples. Using a tuple leaks the internal details to people using the library or something built on this library, and this is needlessly confusing for 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] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]

Review Comment:
   I noticed this too. In some ways I think the double underscore here signals "These properties don't normally exist in a non-testing Catalog implementation" but I think it'd be clearer to just directly describe that in the docstring and use property names without any underscores. You're right in that the double underscore would mangle the names, but only for a child class, something like `Foo(InMemoryCatalog)`, which isn't done anywhere here. Python would then make the property name `_InMemoryCatalog__tables` for instances of `Foo`.
   
   I came to the same conclusion as you, this is a test class so it doesn't matter as much.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,

Review Comment:
   Love it, thanks!



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866421513


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):

Review Comment:
   Added all the namespace methods to just `Catalog(ABC)`. Resolving 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] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:

Review Comment:
   How about renaming this to `load_namespace` instead? Do we need the `_metadata` part?
   
   In Java, we decided not to have a `Namespace` object like there is for `Table`, which is why the Java method is called `loadNamespaceMetadata` -- it returns just the metadata. That's kind of an awkward API because namespaces don't work the same way that tables do. For example, to set a table's properties you use the table:
   
   ```java
   Table table = catalog.loadTable(tableName);
   table.updateProperties().set("a", "b").commit();
   ```
   
   But to modify a namespace, you set properties through the catalog:
   
   ```java
   catalog.setProperties(namespace, ImmutableMap.of("a", "b"));
   ```
   
   I'm okay either way, but we should consider what we want this to look like in Python.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,34 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+
+
+class Table(ABC):
+    """Placeholder for Table managed by the Catalog that points to the current Table Metadata.
+
+    To be implemented by https://github.com/apache/iceberg/issues/3227
+    """
+
+
+class PartitionSpec:

Review Comment:
   Should this create a `SortOrder` placeholder as well?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   What is your rationale for this? I'd probably just allow whatever the caller prefers.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872871245


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   Makes sense. Will remove the enforcement.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:
+        """Create a table
+
+        Args:
+            table_spec: A specification to create a table
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: str, name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: Table's namespace
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: str, name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: table namespace
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: str, from_name: str, to_namespace: str, to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace.
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned.
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(self, table_spec: TableSpec) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            table_spec: A specification to replace a table
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> List[Tuple[str, str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[str]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[str]: a List of namespace string
+        """
+
+    @abstractmethod
+    def get_namespace_metadata(self, namespace: str) -> Dict[str, str]:

Review Comment:
   I think this should be `load` rather than `get`.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870498410


##########
python/setup.cfg:
##########
@@ -43,6 +43,7 @@ package_dir =
 packages = find:
 python_requires = >=3.7
 install_requires =
+    attrs

Review Comment:
   As touched in the other thread, I am in agreement and have removed the `attrs` dependency now.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873987106


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            Dict[str, str]: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873987355


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.

Review Comment:
   Removed in a recent commit. Marking it resolved.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873989888


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,34 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+
+
+class Table(ABC):
+    """Placeholder for Table managed by the Catalog that points to the current Table Metadata.
+
+    To be implemented by https://github.com/apache/iceberg/issues/3227
+    """
+
+
+class PartitionSpec:

Review Comment:
   Marking it resolved as this will be implemented by [#3227](https://github.com/apache/iceberg/issues/3227)



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   Normally, I think the term is "fully qualified". If you use "fully qualified" then there is no need to say "with its namespace" because that's redundant.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]

Review Comment:
   `Metadata` is definitely an overloaded word. In most places, we prefer "properties" instead. How about `Properties`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,

Review Comment:
   The extra arguments aren't needed.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)

Review Comment:
   Why is this provided by `InMemoryCatalog` rather than a utility method in the `Catalog` abstract class? That seems like a more appropriate place to me, since this is behavior that all catalogs should share.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867055012


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   Added a frozen `TableSpec` class in a recent commit to hold user provided table specifications to create or replace a table. Also, I made partition_spec optional in there.



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   Since we're not using runtime type validation, we can use dataclasses here which is basically a slimmed down version of attrs that's been added to the standard library (might even be by the same author although I'm not 100% sure).
   ```py
   from dataclasses import dataclass
   
   @dataclass(kw_only=True, frozen=True)
   class TableSpec:
       ...
   ```



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867216868


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   I did start with dataclass initially but thought we would eventually need to add validators for every attribute later on to validate the table spec. That's why went with attrs. I have added a TODO in the code as well to signify the same.
   



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,251 @@
+#  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 abc import ABC, abstractmethod
+from typing import Any, Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            Dict[str, str]: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        """Update or remove metadata for a namespace.
+
+        Note: Existing metadata is overridden, use get, mutate, and then set.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+            metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+
+class TableNotFoundError(Exception):

Review Comment:
   This is the first time we'll be adding custom exceptions. Since these will probably be used in other parts of the library, I think we should create a file `src/iceberg/exceptions.py` at the root and define these there.
   
   Also, (mostly opinion here), I'd rather not override `__init__` just to provide a default message. I've actually found that default messages make debugging harder and I'd rather we simply define the exception types and raise more explicit messages based on the context where they're raise.
   
   _exceptions.py_
   ```py
   class TableNotFoundError(Exception):
       """Raised when a referenced table cannot be found"""
       pass
   ```
   Then we can require providing a more detailed message whenever it's raised:
   
   _catalog/base.py_
   ```py
   raise TableNotFoundError(f"Cannot load table {table}, not found in catalog {catalog}")
   ```
   ```py
   raise TableNotFoundError(f"Cannot drop table {table}, not found in catalog {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] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870516273


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   I've removed the `TableSpec` wrapper class in a recent commit and reverted back to taking in exploded keyword arguments, as @danielcweeks pointed out it was still an open discussion. I've made the `partition_spec` optional in the API signature now.



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/in_memory.py:
##########
@@ -0,0 +1,106 @@
+#  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 Dict, List, Optional
+
+from iceberg.catalog.base import AlreadyExistsError, Catalog, TableNotFoundError
+from iceberg.table.base import Table, TableSpec
+
+
+class InMemoryCatalog(Catalog):

Review Comment:
   Thanks for adding this! This will be great for demos/quickstarts/tests. If it's not too much would you mind breaking this out into a separate PR? I think there's value in getting just the catalog interface merged in asap so different implementations can be worked on in parallel.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870497559


##########
python/src/iceberg/catalog/in_memory.py:
##########
@@ -0,0 +1,106 @@
+#  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 Dict, List, Optional
+
+from iceberg.catalog.base import AlreadyExistsError, Catalog, TableNotFoundError
+from iceberg.table.base import Table, TableSpec
+
+
+class InMemoryCatalog(Catalog):

Review Comment:
   @samredai @danielcweeks I've moved it completely in the test sources now just as a means to test the base interface. Let me know if you still feel we should break out those tests in a separate PR, will be happy to do so.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873986283


##########
python/src/iceberg/exceptions.py:
##########
@@ -0,0 +1,32 @@
+#  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.
+
+
+class TableNotFoundError(Exception):

Review Comment:
   Fixed in a recent commit. Marking it resolved.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#issuecomment-1130774028

   Sorry for the mess above ^. Had to do a rebase from master to pull in recent PRs to add `namespace` to `spellcheck-dictionary.txt` for the CI checks to pass.


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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876329620


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):

Review Comment:
   @Fokko I am a little on the fence with having our interfaces (abstract classes) marked as `@dataclass`.  I'm not entirely sure if `ABC` and `@dataclass` play along nicely. Also, it forces all sub-classes to be annotated with `@dataclass` as well. 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,34 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+
+
+class Table(ABC):
+    """Placeholder for Table managed by the Catalog that points to the current Table Metadata.
+
+    To be implemented by https://github.com/apache/iceberg/issues/3227
+    """
+
+
+class PartitionSpec:

Review Comment:
   We could also replace this by the objects generated from the open-api spec: https://github.com/apache/iceberg/pull/4858/files#diff-4f32e455c8da9fc5dc641048dc398741b72e928f359bfb9e5ef3640e7d32873eR116-R118



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):

Review Comment:
   Do we need to have this as a subclass? I know the JDBC catalog has been updated recently to include namespaces. If the ideal implementation of a catalog should support namespaces, then I'd vote for adding these abstractmethods to `Catalog(ABC)` and an implementation that does not support namespaces will have to explicitly override it and raise some kind of error.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.

Review Comment:
   For consistency with the other docstrings, can you switch this to [google style](https://google.github.io/styleguide/pyguide.html#38-comments-and-docstrings) instead of sphinx style? (So using `Args:`)



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:
+        """
+        List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        :param namespace: given namespace
+        :return: a List of namespace names
+        """
+
+    @abstractmethod
+    def get_namespace_metadata(self, namespace: str) -> dict:

Review Comment:
   I'm not familiar with namespace metadata, is this general metadata that's persisted in the catalog?



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   This should be optional to account for unpartitioned table creation, right? Actually thinking about this more, information like name, schema, and partition spec should be contained in an instance of `Table`--would it be better here if these table related methods just took a `Table` instance?



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:

Review Comment:
   I'm not sure if this is from the java client but I can't think of an example of when you would provide a namespace here. I might be missing something but I'm imagining that `catalog.list_namespaces()` would always be called with no args and return all namespaces in the catalog.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870499328


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:
+        """Create a table
+
+        Args:
+            table_spec: A specification to create a table
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: str, name: str) -> Table:

Review Comment:
   Agreed. I have changed this for all Catalog API signatures in the most recent commit.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r869362475


##########
python/src/iceberg/catalog/in_memory.py:
##########
@@ -0,0 +1,106 @@
+#  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 Dict, List, Optional
+
+from iceberg.catalog.base import AlreadyExistsError, Catalog, TableNotFoundError
+from iceberg.table.base import Table, TableSpec
+
+
+class InMemoryCatalog(Catalog):

Review Comment:
   I am done with the In-memory catalog implementation and have committed it within this PR. I think it is important to test the Catalog interface with a concrete implementation to see if it holds firm, and will also help the reviewer visualize the interface in action better. I'll mark this PR ready for review now :)



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,444 @@
+#  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 dataclasses import dataclass
+from typing import Dict, List, Optional, Tuple, cast
+
+import pytest
+
+from iceberg.catalog.base import Catalog
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NamespaceNotFoundError,
+    TableNotFoundError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+@dataclass(frozen=True)
+class InMemoryTable(Table):
+    """An in-memory table representation for testing purposes.
+
+    Usage:
+        table_spec = InMemoryTable(
+            namespace = ("com", "organization", "department"),
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+    """
+
+    namespace: Tuple[str, ...]
+    name: str
+    schema: Schema
+    location: str
+    partition_spec: PartitionSpec
+    properties: Dict[str, str]
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Tuple[Tuple[str, ...], str], InMemoryTable]
+    __namespaces: Dict[Tuple[str, ...], Dict[str, str]]
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        if (namespace, name) in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = InMemoryTable(
+                namespace=namespace,
+                name=name,
+                schema=schema if schema else None,
+                location=location if location else None,
+                partition_spec=partition_spec if partition_spec else None,
+                properties=properties if properties else {},
+            )
+            self.__tables[(namespace, name)] = table
+            return table
+
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:
+        try:
+            return self.__tables[(namespace, name)]
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        try:
+            self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        try:
+            table = self.__tables.pop((from_namespace, from_name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {from_name} not found in the catalog")
+
+        renamed_table = InMemoryTable(
+            namespace=to_namespace,
+            name=to_name,
+            schema=table.schema,
+            location=table.location,
+            partition_spec=table.partition_spec,
+            properties=table.properties,
+        )
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[(to_namespace, to_name)] = renamed_table
+        return renamed_table
+
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        try:
+            table = self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+        replaced_table = InMemoryTable(
+            namespace=namespace if namespace else table.namespace,
+            name=name if name else table.name,
+            schema=schema if schema else table.schema,
+            location=location if location else table.location,
+            partition_spec=partition_spec if partition_spec else table.partition_spec,
+            properties={**table.properties, **properties},
+        )
+        self.__tables[(replaced_table.namespace, replaced_table.name)] = replaced_table
+        return replaced_table
+
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        if [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        if namespace:
+            list_tables = [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return cast(List[Tuple[Tuple[str, ...], str]], list_tables)
+
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        if namespace in self.__namespaces:
+            self.__namespaces[namespace] = metadata
+        else:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+
+@pytest.fixture
+def catalog() -> InMemoryCatalog:
+    return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+
+
+@pytest.fixture
+def table_spec() -> InMemoryTable:

Review Comment:
   Since we're exploding the arguments to the catalog methods now, do we need this fixture or the `InMemoryTable` class above? At first glance it may imply that catalog functionality requires you provide `Table` instances. Maybe we can just use a file-level constant that's just a dictionary.
   ```py
   EXAMPLE_TABLE1 = {
     "namespace": ("com", "organization", "department"),
     "name": "my_table",
     "schema": Schema(schema_id=1),
     "location": "protocol://some/location",
     "partition_spec": PartitionSpec(),
     "properties": {"key1": "value1", "key2": "value2"}
   }
   ```



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   Yeah, I think we should allow callers to choose. You make a good point about internal APIs, where this is probably more relevant. But forcing style on library callers seems unnecessary to me.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:

Review Comment:
   I don't think it's a good idea to default `purge` to `True` because that will remove the ability to un-delete tables in a lot of cases where the caller doesn't actually need to purge immediately. We had to do it in Java because it wasn't originally an option, but here it can be required. Alternatively, we could split this into `purge_table` and `drop_table` if you think it is cleaner to have multiple methods.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.

Review Comment:
   I think this is confusing. I believe you're trying to say that this doesn't scan data stored in the table, which would be more clear.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r877542835


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.drop_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table_that_does_not_exist_raise_error(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_purge_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.purge_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_rename_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+
+    # When
+    new_table = "new.namespace.new_table"
+    table = catalog.rename_table(TEST_TABLE_IDENTIFIER, new_table)
+
+    # Then
+    assert table
+    assert table is not given_table

Review Comment:
   @rdblue In the earlier commits I was asserting if the table name has changed, but since I had rollback any changes to the `Table` interface in this PR I resorted to checking just that the table is changing in some form post renaming. I will eventually be changing this to assert table name change once we add attributes back to `Table` in #3227 



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

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

For queries about this service, please contact Infrastructure at:
users@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 #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:

Review Comment:
   ```suggestion
       def list_tables(self) -> list[Table]:
   ```



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,

Review Comment:
   Instead of having a name and a separate schema, we could also accept a path `Tuple[str, ...]`.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867054106


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,

Review Comment:
   Added a frozen `TableSpec` class in a recent commit to hold user provided table specifications to create or replace a table.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867054444


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:

Review Comment:
   Removed the argument in the recent commit.



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

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

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


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


[GitHub] [iceberg] rdblue commented on pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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

   There are still a few things to fix, but I think we can clean up things like moving to `@dataclass` separately to unblock other work.


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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872849836


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:

Review Comment:
   IMO the model should reflect that "A catalog has namespaces that has tables". So `Catalog` --> `Namespace` --> `Table`
   
   I thought about this several times when modeling it but didn't have the context on why it wasn't done this way in the Java API. The fact that we don't allow namespaces to be dropped that are not empty is a code smell that it should be modeled that way. What do you think?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:

Review Comment:
   Understood



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace

Review Comment:
   I don't think you need "in the namespace" at the end. What namespace is that referring to?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873986068


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   This reminds me of Snowflake where splitting can be quite tricky: https://docs.snowflake.com/en/sql-reference/identifiers-syntax.html#double-quoted-identifiers
   
   For example "daabas$e"."s.ch.e.ma"."table" is valid in Snowflake :) Also when we want to convert the tuple back into a string, we also want to quote the identifier.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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'

Review Comment:
   We could also add a convenience method for this:
   
   ```python
   def table_exists(self, identifier: Union[str, Identifier]) -> bool:
       try:
           self.load_table(identifier)
           return True
       except:
           return False
   ```
   
   We could implement this in the 



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873986812


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873987797


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):

Review Comment:
   We could also generate this `__init__` very easily using the `@dataclass` by adding the decorator and defining the properties in the class itself:
   ```
       name: str
       properties: Metadata
   ```
   
   This will also autogenerate the getters below:
   ```
       @property
       def name(self) -> str:
           return self._name
   
       @property
       def properties(self) -> Metadata:
           return self._properties
   ```



##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):

Review Comment:
   If we go with a `@dataclass`, you can easily remove this init, and replace the internal fields with:
   ```
       __tables: Dict[Identifier, Table]  = field(default_factory=dict)
       __namespaces: Dict[Identifier, Metadata] = field(default_factory=dict)
   
   ```



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r877542835


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.drop_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table_that_does_not_exist_raise_error(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_purge_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.purge_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_rename_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+
+    # When
+    new_table = "new.namespace.new_table"
+    table = catalog.rename_table(TEST_TABLE_IDENTIFIER, new_table)
+
+    # Then
+    assert table
+    assert table is not given_table

Review Comment:
   @rdblue In the earlier commits I was asserting if the table name has changed, but since I had to rollback all changes to the `Table` interface in this PR I resorted to checking just that the table is changing in some form post renaming. I will eventually be changing this to assert table name change once we add attributes back to `Table` in #3227 



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]

Review Comment:
   Nit / question: I know that the usage of double underscore somewhat mangles function names to make them "private". Does that apply to class fields as well?
   
   I've always preferred single underscore for this reason, but I'm not a huge pythonista and this is a test class. So more just for my own knowledge as well as to better understand the current "best practices".



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.drop_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table_that_does_not_exist_raise_error(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_purge_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.purge_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_rename_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+
+    # When
+    new_table = "new.namespace.new_table"
+    table = catalog.rename_table(TEST_TABLE_IDENTIFIER, new_table)
+
+    # Then
+    assert table
+    assert table is not given_table

Review Comment:
   Isn't this just asserting that the table isn't the same instance? Is that useful?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866269504


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   Agreed, they should ideally be taking a `TableBuilder` instance which has the table spec and the method returns  the created 'Table' instance. In my mental model, `TableBuilder` is a spec that has all the information required to create a 'Table', and 'Table' is something that holds the existing table state. Are you thinking on similar lines as well?



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.

Review Comment:
   Ack, will do.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   In Java, we use a dedicated Unpartitioned spec.
   
   But I agree that this should be optional and then the builder (or this function… either way) should default to Unpartitioned if this is None (implicitly using the Unpartitioned spec).



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

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

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


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/setup.cfg:
##########
@@ -43,6 +43,7 @@ package_dir =
 packages = find:
 python_requires = >=3.7
 install_requires =
+    attrs

Review Comment:
   While the dataclasses support may lag, we've had a lot of pushback around library creep.  I don't feel like the added value warrants bringing in a library in this case.  This may feel overly conservative, but with Iceberg being as low-level as it will be in many frameworks moving forward, we really want to keep the dependencies as slim as possible.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   @dhruv-pratap, I agree that this should support strings for table names, like `catalog.load_table("examples.nyc_taxi_yellow")`.
   
   > do you mean using just a single fully classified table name 'str'? Or, a 'str' namespace and 'str' name?
   
   I don't think it makes sense to force the caller to parse if we are going to split on `.`. If Iceberg can handle splitting the namespace then it can also split the table name out. That also aligns with how people will naturally use this, like Spark's `writeTo("db.table")` or `spark.read("examples.nyc_taxi_yellow")`. That means we support a string table identifier.
   
   We also know that we need to support `Tuple[str]`, for cases like SQL engines where `.` can be included in an identifier by escaping and quoting that is handled by the external engine's parser.
   
   Supporting both `str` and `Tuple[str]` means that we should not have a separate `namespace` argument. Otherwise, there are ambiguous (or not obvious) cases, like `catalog.load_table(namespace=("a", "b"), name="c.d")`.
   
   Bringing all that together, the catalog methods should accept `identifier: str | Tuple[str]`. If the identifier is a string, it is split into a tuple on `.`. If it is a tuple, it is used as-is. That's simple and understandable, while supporting all of the cases that we need.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870501360


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:

Review Comment:
   As touched in the other thread, I've removed `TableSpec`, and now take in exploded keyword arguments.



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   +1, I can't think of a reason why we would want to stop someone from doing `create_table(("foodb"), "footable", schema)` and force them to do `create_table(namespace=("foodb"), name="footable", schema=schema)`. Both seem valid to me.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r869480196


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:
+        """Create a table
+
+        Args:
+            table_spec: A specification to create a table
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: str, name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: Table's namespace
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: str, name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: table namespace
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: str, from_name: str, to_namespace: str, to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace.
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned.
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(self, table_spec: TableSpec) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            table_spec: A specification to replace a table
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> List[Tuple[str, str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[str]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[str]: a List of namespace string
+        """
+
+    @abstractmethod
+    def get_namespace_metadata(self, namespace: str) -> Dict[str, str]:

Review Comment:
   Ack. Will change in the next commit.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r869479467


##########
python/setup.cfg:
##########
@@ -43,6 +43,7 @@ package_dir =
 packages = find:
 python_requires = >=3.7
 install_requires =
+    attrs

Review Comment:
   There is Python built-in [dataclasses support](https://docs.python.org/3/library/dataclasses.html), which essentially was influenced by the [attrs community](https://www.attrs.org/en/stable/why.html) but it always seems to lack behind in terms of features. Unfortunately, pre-Python 3.10 dataclasses isn't as powerful as attrs. For instance, it doesn't support `kw_only=True` out of the box and requires us to write custom decorators to support that.
   
   I did talk briefly about this with Sam in [this conversation thread](https://github.com/apache/iceberg/pull/4706#discussion_r867216868).



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r869479467


##########
python/setup.cfg:
##########
@@ -43,6 +43,7 @@ package_dir =
 packages = find:
 python_requires = >=3.7
 install_requires =
+    attrs

Review Comment:
   There is Python built-in [dataclasses support](https://docs.python.org/3/library/dataclasses.html), which essentially was influenced by the [attrs community](https://www.attrs.org/en/stable/why.html) but it always seems to lack behind in terms of features. Unfortunately, pre-Python 3.10 dataclasses isn't as powerful as attrs. For instance, it doesn't support "kw_only" out of the box and requires us to write custom decorators to support that.
   
   I did talk briefly about this with Sam in [this conversation thread](https://github.com/apache/iceberg/pull/4706#discussion_r867216868).



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,444 @@
+#  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 dataclasses import dataclass
+from typing import Dict, List, Optional, Tuple, cast
+
+import pytest
+
+from iceberg.catalog.base import Catalog
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NamespaceNotFoundError,
+    TableNotFoundError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+@dataclass(frozen=True)
+class InMemoryTable(Table):
+    """An in-memory table representation for testing purposes.
+
+    Usage:
+        table_spec = InMemoryTable(
+            namespace = ("com", "organization", "department"),
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+    """
+
+    namespace: Tuple[str, ...]
+    name: str
+    schema: Schema
+    location: str
+    partition_spec: PartitionSpec
+    properties: Dict[str, str]
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Tuple[Tuple[str, ...], str], InMemoryTable]
+    __namespaces: Dict[Tuple[str, ...], Dict[str, str]]
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        if (namespace, name) in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = InMemoryTable(
+                namespace=namespace,
+                name=name,
+                schema=schema if schema else None,
+                location=location if location else None,
+                partition_spec=partition_spec if partition_spec else None,
+                properties=properties if properties else {},
+            )
+            self.__tables[(namespace, name)] = table
+            return table
+
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:
+        try:
+            return self.__tables[(namespace, name)]
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        try:
+            self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        try:
+            table = self.__tables.pop((from_namespace, from_name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {from_name} not found in the catalog")
+
+        renamed_table = InMemoryTable(
+            namespace=to_namespace,
+            name=to_name,
+            schema=table.schema,
+            location=table.location,
+            partition_spec=table.partition_spec,
+            properties=table.properties,
+        )
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[(to_namespace, to_name)] = renamed_table
+        return renamed_table
+
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        try:
+            table = self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+        replaced_table = InMemoryTable(
+            namespace=namespace if namespace else table.namespace,
+            name=name if name else table.name,
+            schema=schema if schema else table.schema,
+            location=location if location else table.location,
+            partition_spec=partition_spec if partition_spec else table.partition_spec,
+            properties={**table.properties, **properties},
+        )
+        self.__tables[(replaced_table.namespace, replaced_table.name)] = replaced_table
+        return replaced_table
+
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        if [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        if namespace:
+            list_tables = [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return cast(List[Tuple[Tuple[str, ...], str]], list_tables)
+
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        if namespace in self.__namespaces:
+            self.__namespaces[namespace] = metadata
+        else:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+
+@pytest.fixture
+def catalog() -> InMemoryCatalog:

Review Comment:
   We may want to use this fixture in other test files so I would suggest moving it to `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] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:

Review Comment:
   How about renaming this to `load_namespace` instead? Do we need the `_metadata` part?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876368395


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]

Review Comment:
   Addressed in a recent commit. Marking it resolved.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   Addressed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,
+        )
+
+
+def test_table(catalog: InMemoryCatalog):

Review Comment:
   `test_load_table`?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876460449


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        """Removes provided metadata keys and then updates metadata for a namespace.
+
+        Note: No errors are raised if a provided removal key is not found.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            removals: a set of metadata keys that need to be removed. Optional Argument.
+            updates: a dictionary of properties to be updated for the given namespace. Optional Argument.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist

Review Comment:
   Addressed. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873988168


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870766564


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,251 @@
+#  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 abc import ABC, abstractmethod
+from typing import Any, Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            Dict[str, str]: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        """Update or remove metadata for a namespace.
+
+        Note: Existing metadata is overridden, use get, mutate, and then set.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+            metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+
+class TableNotFoundError(Exception):

Review Comment:
   Done. Removed the custom messaging for errors and moved them over to `src/iceberg/exceptions.py` as suggested



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table

Review Comment:
   Adding them in #3227 is okay. FYI, you may want to look at the tests we use to [validate catalogs](https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java) in the Java implementation. Many of the cases are the same to make sure you're getting all of the checks.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:

Review Comment:
   Changing methods to identify tables by `identifier: str | Tuple[str]` instead of by `namespace` and `name` simplifies this. It can just be `List[Tuple[Str]]` rather than having another tuple of namespace and name.
   
   You may also consider making this a bit more clear by defining type aliases, like @Fokko did in #4685 for `Position`. For example:
   
   ```python
   Identifier = Tuple[str]
   
   def load_table(self, ident: str | Identifier, ...) -> Table:
       ...
   
   def list_tables(self, namespace: Optional[Identifier] = None) -> List[Identifier]:
       ...
   ```



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.

Review Comment:
   It doesn't look like this starts a transaction because the return type is a `Table`. For now, how about removing this method until we want to add transactions? We can get a lot done with just `create_table` and `drop_table`. Transactions aren't really ready to be added yet anyway.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872838088


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.

Review Comment:
   Ack. Will change



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:

Review Comment:
   I like the idea to split it to make it explicit, will make the change.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:

Review Comment:
   Ack, will make the change. But the thinking here was the user can use method chaining here and the name `load_table` didn't quite fit in there.
   
   `
   catalog
    .table('my_table')
    .rename(...)
   `



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace

Review Comment:
   Ack. Copy-pasta error. Will amend.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,

Review Comment:
   The rationale here was just to enforce better code readability since we are accepting an exploded set of arguments many of which are optional. But it may make sense to do such enforcements on internal APIs rather than an external API I guess. I could go either way on this.



##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,34 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+
+
+class Table(ABC):
+    """Placeholder for Table managed by the Catalog that points to the current Table Metadata.
+
+    To be implemented by https://github.com/apache/iceberg/issues/3227
+    """
+
+
+class PartitionSpec:

Review Comment:
   Intentionally left out to make this PR as slim as possible. SortOrder is an indirect dependency of the catalog interface.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:

Review Comment:
   Yes :) It essentially returns a list of tuple of table-namespace and table-name.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   _Why separate namespace and name?_ 
   I feel like since they are two different concepts we should keep them explicitly separate, rather that it being implicit.
   
   _Re: namespace type_
   This was recommendation from @danielcweeks and @Fokko from [an earlier thread](https://github.com/apache/iceberg/pull/4706#discussion_r869626281). We have gone from `str` to `Tuple[str, ...] ` to now `Union[Tuple[str, ...], str]`. I think there is conversation to be had :)



##########
python/src/iceberg/exceptions.py:
##########
@@ -0,0 +1,32 @@
+#  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.
+
+
+class TableNotFoundError(Exception):

Review Comment:
   Ack. Will change.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.

Review Comment:
   Ack. Will remove this API for now.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            Dict[str, str]: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:

Review Comment:
   I like `update_namespace` accepting both `updates` and `removals`, will make the arguments optional as well.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:

Review Comment:
   This should be called `load_table`.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   Why separate namespace and name? Can this be a single `Tuple[str]`?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/exceptions.py:
##########
@@ -0,0 +1,32 @@
+#  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.
+
+
+class TableNotFoundError(Exception):

Review Comment:
   SQL Alchemy and Iceberg Java use `NoSuchTable` rather than `TableNotFound`. I think it would make sense to use the existing naming convention.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873991227


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:

Review Comment:
   API is now renamed to `load_namespace` in a recent commit.



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   Since we're not using runtime type validation, we can use dataclasses here which is basically a slimmed down version of attrs that's been added to the standard library (might even be by the same author although I'm not 100% sure).
   ```py
   from dataclasses import dataclass
   
   @dataclass(kw_only=True)
   class TableSpec:
       ...
   ```



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r877574185


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")

Review Comment:
   Addressed in a recent commit. Marking it resolved.



##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):

Review Comment:
   Addressed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876378069


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,
+        )
+
+
+def test_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError):

Review Comment:
   Addressed. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876377312


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        """Removes provided metadata keys and then updates metadata for a namespace.

Review Comment:
   Addressed. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876377746


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)

Review Comment:
   Pulled up. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   I added a comment to the other thread as well, but just keep in mind we want to be very conservative on external libraries.  The question should be less whether it's helpful and more is it necessary.  I would say we remove it for now and see if it becomes necessary in the future.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:

Review Comment:
   If this isn't returning a `Namespace` object, then it should still be named `load_namespace_metadata`. The option to remove the `_metadata` assumed that it would return a `Namespace` object rather than a properties dict.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876471928


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table

Review Comment:
   I had this in the PR initially but on [recommendations](https://github.com/apache/iceberg/pull/4706#discussion_r872505969) I intentionally trimmed this down to not make this a fat PR, as that would involve touching the `Table` class and adding attributed to 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] kbendick commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]

Review Comment:
   +1. Properties is also the name of the field and is codified in the REST OpenAPI spec 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] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876460317


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,

Review Comment:
   Addressed. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] rdblue commented on pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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

   I'm ready to commit this once the error message format is fixed.


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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r877573984


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.drop_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_drop_table_that_does_not_exist_raise_error(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_purge_table(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    catalog.purge_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):
+        catalog.load_table(TEST_TABLE_IDENTIFIER)
+
+
+def test_rename_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+
+    # When
+    new_table = "new.namespace.new_table"
+    table = catalog.rename_table(TEST_TABLE_IDENTIFIER, new_table)
+
+    # Then
+    assert table
+    assert table is not given_table

Review Comment:
   Nevermind. I worked around it for the time being. Since the `Table` class isn't frozen yet I added an attribute `identifier` post creation in the `InMemoryCatalog` and asserting on `identifier` renaming for now. This obviously would fail, and rightly so, once we make `Table` immutable/frozen.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table

Review Comment:
   When creating a table, I think this should validate that the table looks as expected by validating the schema, partition spec, location, and properties.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/conftest.py:
##########
@@ -112,3 +113,8 @@ def table_schema_nested():
 @pytest.fixture(scope="session", autouse=True)
 def foo_struct():
     return FooStruct()
+
+
+@pytest.fixture
+def catalog() -> InMemoryCatalog:
+    return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})

Review Comment:
   `test.key` isn't used anywhere, nor is `test.in.memory.catalog`. Can you add tests for properties and name?



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table already exists: {identifier}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            table.identifier = identifier
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        try:
+            table = self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {from_identifier}") from error
+
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        table.identifier = to_identifier
+        self.__tables[to_identifier] = table
+        return table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace already exists: {namespace}")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}

Review Comment:
   Less is more:
   ```suggestion
               self.__namespaces[namespace] = properties or {}
   ```



##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]

Review Comment:
   I think a single underscore is more appropriate here. It is an internal variable of the `InMemoryCatalog`.
   
   If we want to set this as a ClassVar, then we also have to annotation for it:
   ```suggestion
       _tables: ClassVar[Dict[Identifier, Table]]
   ```
   But I'm not sure if we want to do that since we would like to instantiate a fresh InMemoryCatalog for the different tests.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,247 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Properties = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    The catalog table APIs accept a table identifier, which is fully classified table name. The identifier can be a string or
+    tuple of strings. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+    The catalog namespace APIs follow a similar convention wherein they also accept a namespace identifier that can be a string
+    or tuple of strings.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Properties): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Properties):

Review Comment:
   I'll continue my quest for data classes here as well. Instead of creating the getters/setters, we could also let the dataclasses do this:
   ```python
   @dataclass
   class Catalog(ABC):
   
       name: str =  field()
       name: Properties =  field(default_factory=dict)
   ```
   This will automatically generate the getters and setters.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,247 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Properties = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    The catalog table APIs accept a table identifier, which is fully classified table name. The identifier can be a string or
+    tuple of strings. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+    The catalog namespace APIs follow a similar convention wherein they also accept a namespace identifier that can be a string
+    or tuple of strings.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Properties): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Properties):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Properties:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Table identifier.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table properties that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Table identifier.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Existing table identifier.
+            to_identifier: New table identifier.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: Namespace identifier
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: Namespace identifier
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: Namespace identifier to search.
+
+        Returns:
+            List[Identifier]: list of table identifiers.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace identifiers
+        """
+
+    @abstractmethod
+    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:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        """Removes provided property keys and updates properties 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. Optional Argument.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            ValueError: If removals and updates have overlapping keys.
+        """
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:

Review Comment:
   Naming things is one of the hardest things in Computer Science, but I find this one a bit awkward since:
   ```python
   Identifier = Tuple[str, ...]
   ```
   So it is actually already a tuple. How about:
   ```suggestion
       def to_identifier(identifier: Union[str, Identifier]) -> Identifier:
   ```



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,247 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Properties = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    The catalog table APIs accept a table identifier, which is fully classified table name. The identifier can be a string or
+    tuple of strings. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+    The catalog namespace APIs follow a similar convention wherein they also accept a namespace identifier that can be a string
+    or tuple of strings.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Properties): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Properties):

Review Comment:
   If we want to go full blown pythonic in the future, we could also validate these using Pydantic validators: https://pydantic-docs.helpmanual.io/usage/validators/



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

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

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


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:
+        """Create a table
+
+        Args:
+            table_spec: A specification to create a table
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: str, name: str) -> Table:

Review Comment:
   `namespace` here and throughout the other signatures should probably be a tuple (`Tuple[str, ...]`) as opposed to a string.  After talking with @fokko, I think that's probably a more concise representation of the namespace without having to worry about parsing levels.



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

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

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


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,222 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.table.base import Table, TableSpec
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(self, table_spec: TableSpec) -> Table:

Review Comment:
   I feel like there's an open question as to whether we want to use a wrapper like `TableSpec`, but even if we do, we should separate namespace from the table spec and make that a tuple parameter like the other signatures to be consistent.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870494950


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)

Review Comment:
   Makes sense Dan. I've removed the the `attrs` dependency, as well as the `TableSpec` class in the most recent commit.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r870516273


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,

Review Comment:
   I've removed the `TableSpec` wrapper class in a recent commit and reverted back to taking in exploded keyword arguments. I've made the `partition_spec` optional in the signature now.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r873987993


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.

Review Comment:
   Fixed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r874087557


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   We did discuss how we would be handling it for such cases over [here](https://github.com/apache/iceberg/pull/4706#discussion_r873059425). CC @rdblue 
   
   The idea is for such special use cases we expect the user to pass us a tuple so that it can be used as-is.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   Also, what about allowing people to pass `str` here, which would be parsed by splitting on `.`? That makes simple cases easier. And if someone needs more complicated parsing, they can use the `Tuple[str]` form.



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   +1, I would make this type `Union[Tuple[str], str]`



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:
+        """
+        List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        :param namespace: given namespace
+        :return: a List of namespace names
+        """
+
+    @abstractmethod
+    def get_namespace_metadata(self, namespace: str) -> dict:

Review Comment:
   I see, makes sense. Thanks!



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866415897


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:

Review Comment:
   Thanks for the suggestion Fokko! Although the built-in collection typing, as suggested above, only works in Python 3.9 and above. But I see where you are going with this, so let me return a `typing.List` instead. Ex: `List[Table]`



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866421776


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:

Review Comment:
   Added collection typing for all catalog APIs now. Thanks!



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

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

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


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


[GitHub] [iceberg] samredai commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)
+class TableSpec:
+    """An immutable user specification to create or replace a table.
+
+    Usage:
+        table_spec = TableSpec(
+            namespace = "com.organization.department",
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+
+    TODO: Validators to be added
+    """
+
+    _namespace: str
+    _name: str
+    _schema: Schema
+    _location: str = field()
+    _partition_spec: PartitionSpec = field()
+    _properties: Dict[str, str] = Factory(Dict[str, str])

Review Comment:
   Is the leading underscore needed here since freezing the properties is handled by attrs/dataclasses?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866265980


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:
+        """
+        List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        :param namespace: given namespace
+        :return: a List of namespace names
+        """
+
+    @abstractmethod
+    def get_namespace_metadata(self, namespace: str) -> dict:

Review Comment:
   It is different from catalog metadata. Some (or maybe all) catalogs maintain metadata at a namespace level as well. For instance, JDBC catalog maintains it's namespace level metadata in the table [`iceberg_namespace_properties`](https://github.com/apache/iceberg/blob/96a46c5caa841777d6fff62a17f09dcc6929603d/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java#L81).



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866259041


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):
+    """
+    Base catalog for catalogs that support namespaces.
+    """
+
+    @abstractmethod
+    def create_namespace(self, namespace: str, properties: Optional[dict] = None) -> None:
+        """
+        Create a namespace in the catalog.
+
+        :param namespace: The namespace to be created.
+        :param properties: A string dict of properties for the given namespace
+        :return: Nothing
+        :raises AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: str) -> None:
+        """
+        Drop a namespace.
+
+        :param namespace: The namespace to be dropped.
+        :return: Nothing
+        :raises NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        :raises NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[str] = None) -> list:
+        """
+        List tables under the given namespace in the catalog. If namespace not provided, will list all tables in the
+        catalog.
+
+        :param namespace: the namespace to search
+        :return: list of table names under this namespace.
+        :raises NamespaceNotFoundError: If no such namespace exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self, namespace: Optional[str] = None) -> list:

Review Comment:
   Yep, that was in the Java client. I think it to narrow dow your list under a certain level. But I'll get rid of it since it sounds like it is not a prominent use-case, if needed the API consumer user can always list all namespaces and filter on namespace levels.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/setup.cfg:
##########
@@ -43,6 +43,7 @@ package_dir =
 packages = find:
 python_requires = >=3.7
 install_requires =
+    attrs

Review Comment:
   Is there an alternative in the standard library we can use?
   
   The fewer libraries we add as a dependency, the more we can allow users to pick their own versions and not have to worry about conflicts.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r866252658


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,198 @@
+#  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 abc import ABC, abstractmethod
+from typing import Optional
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """
+    Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(dict): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: dict):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> dict:
+        return self._properties
+
+    @abstractmethod
+    def list_tables(self) -> list:
+        """
+        List tables in the catalog.
+
+        :return: list of table names in the catalog.
+        """
+
+    @abstractmethod
+    def create_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Create a table
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the created table instance
+        :raises AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, name: str) -> 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 does not load table's data in any form.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :return: the table instance with its metadata
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, name: str, purge: bool = True) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param name: table name
+        :param purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_name: str, to_name: str) -> None:
+        """
+        Drop a table; Optionally purge all data and metadata files.
+
+        :param from_name: Existing table's name. Fully classified table name, if it is a namespaced catalog.
+        :param to_name: New Table name to be assigned. Fully classified table name, if it is a namespaced catalog.
+        :return: Nothing
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        name: str,
+        schema: Schema,
+        partition_spec: PartitionSpec,
+        *,
+        location: Optional[str] = None,
+        properties: Optional[dict] = None
+    ) -> Table:
+        """
+        Starts a transaction and replaces the table with the provided spec.
+
+        :param name: Table's name. Fully classified table name, if it is a namespaced catalog.
+        :param schema: Table's schema
+        :param partition_spec: A partition spec for the table
+        :param location: a location for the table; Optional Keyword Argument
+        :param properties: a string dictionary of table properties; Optional Keyword Argument
+        :return: the replaced table instance
+        :raises TableNotFoundError: If a table with the name does not exist
+        """
+
+
+class NamespacedCatalog(Catalog):

Review Comment:
   Tell me about it :) I was torn on this one as well as I too strongly felt that `Catalog(ABC)` and `NamespacedCatalog(Catalog)` could be just one. I could not find even a single catalog implementation in the Java API that did NOT support namespaces. All of them do, so it seemed like a redundant extensibility being offered.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,410 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+        name = Catalog.table_name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        from_namespace = Catalog.namespace_from(from_identifier)
+        from_name = Catalog.table_name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}") from error
+
+        renamed_table = Table()
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = Catalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = Catalog.table_name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError, match="Table my_table already exists"):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+        )
+
+
+def test_load_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError, match="Table my_table not found"):

Review Comment:
   Error messages like this can be confusing if the name is invalid: `Table  not found`. Instead, we prefer to make it clear that the name was included by using a message like `Table does not exist: my_table`. Can you update the error message?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:

Review Comment:
   Is this return type correct?



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   Let's start by considering whether to support loading the table by `str`.
   
   I think that's a good idea because that's going to cover most use cases. Do you agree, @dhruv-pratap?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872870357


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   I personally prefer to have my data types reflect what the end user see so I like 'str' and that's why I started with it in the first place. 
   
   But just so that I understand it right, when you say load table using 'str' do you mean using just a single fully classified table name 'str'? Or,  a 'str' namespace and 'str' name?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r867215217


##########
python/src/iceberg/table/base.py:
##########
@@ -0,0 +1,67 @@
+#  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 __future__ import annotations
+
+from abc import ABC
+from typing import Dict
+
+from attrs import Factory, field, frozen
+
+from iceberg.schema import Schema
+
+
+@frozen(kw_only=True)
+class TableSpec:
+    """An immutable user specification to create or replace a table.
+
+    Usage:
+        table_spec = TableSpec(
+            namespace = "com.organization.department",
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+
+    TODO: Validators to be added
+    """
+
+    _namespace: str
+    _name: str
+    _schema: Schema
+    _location: str = field()
+    _partition_spec: PartitionSpec = field()
+    _properties: Dict[str, str] = Factory(Dict[str, str])

Review Comment:
   You're right. It's not actually required. Will remove it in the next commit. Thanks!



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

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

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


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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/in_memory.py:
##########
@@ -0,0 +1,106 @@
+#  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 Dict, List, Optional
+
+from iceberg.catalog.base import AlreadyExistsError, Catalog, TableNotFoundError
+from iceberg.table.base import Table, TableSpec
+
+
+class InMemoryCatalog(Catalog):

Review Comment:
   The in memory implementation should only exist in the test srcs so that it does not get confused / used for a real implementation.  Also echo Sam's comment that splitting the PR will make it easier to get the catalog in.  We can always follow up with changes if necessary.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,

Review Comment:
   @dhruv-pratap, I agree that this should support strings for table names, like `catalog.load_table("examples.nyc_taxi_yellow")`.
   
   > do you mean using just a single fully classified table name 'str'? Or, a 'str' namespace and 'str' name?
   
   I don't think it makes sense to force the caller to parse if we are going to split on `.`. If Iceberg can handle splitting the namespace then it can also split the table name out. That also aligns with how people will naturally use this, like Spark's `writeTo("db.table")` or `spark.read("examples.nyc_taxi_yellow")`. That means we support a string table identifier.
   
   We also know that we need to support `Tuple[str]`, for cases like SQL engines where `.` can be included in an identifier by escaping and quoting.
   
   Supporting both `str` and `Tuple[str]` means that we should not have a separate `namespace` argument. Otherwise, there are ambiguous (or not obvious) cases, like `catalog.load_table(namespace=("a", "b"), name="c.d")`.
   
   Bringing all that together, the catalog methods should accept `identifier: str | Tuple[str]`. If the identifier is a string, it is split into a tuple on `.`. If it is a tuple, it is used as-is. That's simple and understandable, while supporting all of the cases that we need.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r872496987


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,444 @@
+#  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 dataclasses import dataclass
+from typing import Dict, List, Optional, Tuple, cast
+
+import pytest
+
+from iceberg.catalog.base import Catalog
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NamespaceNotFoundError,
+    TableNotFoundError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+@dataclass(frozen=True)
+class InMemoryTable(Table):
+    """An in-memory table representation for testing purposes.
+
+    Usage:
+        table_spec = InMemoryTable(
+            namespace = ("com", "organization", "department"),
+            name = "my_table",
+            schema = Schema(),
+            location = "protocol://some/location",  // Optional
+            partition_spec = PartitionSpec(),       // Optional
+            properties = [                          // Optional
+                "key1": "value1",
+                "key2": "value2",
+            ]
+        )
+    """
+
+    namespace: Tuple[str, ...]
+    name: str
+    schema: Schema
+    location: str
+    partition_spec: PartitionSpec
+    properties: Dict[str, str]
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Tuple[Tuple[str, ...], str], InMemoryTable]
+    __namespaces: Dict[Tuple[str, ...], Dict[str, str]]
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        if (namespace, name) in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = InMemoryTable(
+                namespace=namespace,
+                name=name,
+                schema=schema if schema else None,
+                location=location if location else None,
+                partition_spec=partition_spec if partition_spec else None,
+                properties=properties if properties else {},
+            )
+            self.__tables[(namespace, name)] = table
+            return table
+
+    def table(self, namespace: Tuple[str, ...], name: str) -> Table:
+        try:
+            return self.__tables[(namespace, name)]
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        try:
+            self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        try:
+            table = self.__tables.pop((from_namespace, from_name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {from_name} not found in the catalog")
+
+        renamed_table = InMemoryTable(
+            namespace=to_namespace,
+            name=to_name,
+            schema=table.schema,
+            location=table.location,
+            partition_spec=table.partition_spec,
+            properties=table.properties,
+        )
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[(to_namespace, to_name)] = renamed_table
+        return renamed_table
+
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+
+        try:
+            table = self.__tables.pop((namespace, name))
+        except KeyError:
+            raise TableNotFoundError(f"Table {name} not found in the catalog")
+
+        replaced_table = InMemoryTable(
+            namespace=namespace if namespace else table.namespace,
+            name=name if name else table.name,
+            schema=schema if schema else table.schema,
+            location=location if location else table.location,
+            partition_spec=partition_spec if partition_spec else table.partition_spec,
+            properties={**table.properties, **properties},
+        )
+        self.__tables[(replaced_table.namespace, replaced_table.name)] = replaced_table
+        return replaced_table
+
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        if [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        if namespace:
+            list_tables = [table_name_tuple for table_name_tuple in self.__tables.keys() if namespace in table_name_tuple]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return cast(List[Tuple[Tuple[str, ...], str]], list_tables)
+
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:
+        if namespace in self.__namespaces:
+            self.__namespaces[namespace] = metadata
+        else:
+            raise NamespaceNotFoundError(f"Namespace {namespace} not found in the catalog")
+
+
+@pytest.fixture
+def catalog() -> InMemoryCatalog:

Review Comment:
   Moved in the most recent commit.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,223 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Tuple
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Dict[str, str]):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Dict[str, str]:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    def table(self, namespace: Tuple[str, ...], name: str) -> 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 does not load table's data in any form.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table's name.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, namespace: Tuple[str, ...], name: str, purge: bool = True) -> None:
+        """Drop a table; Optionally purge all data and metadata files.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: table name
+            purge: Defaults to true, which deletes all data and metadata files in the table; Optional Argument
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_namespace: Tuple[str, ...], from_name: str, to_namespace: Tuple[str, ...], to_name: str) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_namespace: Existing table's namespace. A tuple of table's namespace levels. Ex: ('com','org','dept')
+            from_name: Existing table's name.
+            to_namespace: New Table namespace to be assigned. Tuple of namespace levels. Ex: ('com','org','new')
+            to_name: New Table name to be assigned.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def replace_table(
+        self,
+        *,
+        namespace: Tuple[str, ...],
+        name: str,
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Dict[str, str]] = None,
+    ) -> Table:
+        """Starts a transaction and replaces the table with the provided spec.
+
+        Args:
+            namespace: A tuple of table's namespace levels. Ex: ('com','org','dept')
+            name: Table name
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the replaced table instance with the updated state
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Tuple[str, ...], properties: Optional[Dict[str, str]] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. Tuple of namespace levels. Ex: ('com','org','dept')
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the name already exists in the namespace
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Tuple[str, ...]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Tuple[str, ...]] = None) -> List[Tuple[Tuple[str, ...], str]]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Tuple[str, ...]]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Tuple[str, ...]]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace_metadata(self, namespace: Tuple[str, ...]) -> Dict[str, str]:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: A Tuple of namespace levels. Ex: ('com','org','dept')
+
+        Returns:
+            Dict[str, str]: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the name does not exist in the namespace
+        """
+
+    @abstractmethod
+    def set_namespace_metadata(self, namespace: Tuple[str, ...], metadata: Dict[str, str]) -> None:

Review Comment:
   This needs to be very clear about behavior. The verb `set` can be interpreted 2 ways: set the whole metadata, or set the given properties. In most parts of the API, we prefer to be very clear and use separate args or methods for setting properties or removing properties.
   
   I think this should do the same. How about changing this to `update_namespace` or `update_namespace_metadata` and accepting both `updates: Dict[str, str]` to set and `removals: Set[str]` to remove?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r874094110


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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'

Review Comment:
   We could, but most of the usages I found for this API in the internal Java implementation uses it as a check to throw the exception AlreadyExistsError.  Maybe we can add it later if we really feel we need it. What do you think?



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#issuecomment-1130024805

   @rdblue We have addressed and resolved all threads on this PR. If you are cool can we please merge this to unblock other catalog implementations? 


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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table
+
+
+def test_create_table_raises_error_when_table_already_exists(catalog: InMemoryCatalog):
+    # Given
+    given_catalog_has_a_table(catalog)
+    # When
+    with pytest.raises(AlreadyExistsError):
+        catalog.create_table(
+            identifier=TEST_TABLE_IDENTIFIER,
+            schema=TEST_TABLE_SCHEMA,
+            location=TEST_TABLE_LOCATION,
+            partition_spec=TEST_TABLE_PARTITION_SPEC,
+            properties=TEST_TABLE_PROPERTIES,
+        )
+
+
+def test_table(catalog: InMemoryCatalog):
+    # Given
+    given_table = given_catalog_has_a_table(catalog)
+    # When
+    table = catalog.load_table(TEST_TABLE_IDENTIFIER)
+    # Then
+    assert table == given_table
+
+
+def test_table_raises_error_on_table_not_found(catalog: InMemoryCatalog):
+    with pytest.raises(NoSuchTableError):

Review Comment:
   Can you also validate the error message? It should include the table identifier ("Table does not exist: db.table") so that there is context.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        """Get metadata dictionary for a namespace.
+
+        Args:
+            namespace: If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Metadata: a dictionary of properties for the given namespace
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        """Removes provided metadata keys and then updates metadata for a namespace.

Review Comment:
   This should not specify an order of operations. It is an error if `removals` and `updates` have overlapping keys. There is no required ordering behavior.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876368619


##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.

Review Comment:
   Addressed in a recent commit. Marking it resolved.



##########
python/src/iceberg/catalog/base.py:
##########
@@ -0,0 +1,204 @@
+#  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 abc import ABC, abstractmethod
+from typing import Dict, List, Optional, Set, Tuple, Union
+
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+Identifier = Tuple[str, ...]
+Metadata = Dict[str, str]
+
+
+class Catalog(ABC):
+    """Base Catalog for table operations like - create, drop, load, list and others.
+
+    Attributes:
+        name(str): Name of the catalog
+        properties(Dict[str, str]): Catalog properties
+    """
+
+    def __init__(self, name: str, properties: Metadata):
+        self._name = name
+        self._properties = properties
+
+    @property
+    def name(self) -> str:
+        return self._name
+
+    @property
+    def properties(self) -> Metadata:
+        return self._properties
+
+    @abstractmethod
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+        """Create a table
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            schema: Table's schema.
+            location: Location for the table. Optional Argument.
+            partition_spec: PartitionSpec for the table. Optional Argument.
+            properties: Table metadata that can be a string based dictionary. Optional Argument.
+
+        Returns:
+            Table: the created table instance
+
+        Raises:
+            AlreadyExistsError: If a table with the name already exists
+        """
+
+    @abstractmethod
+    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: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        """Drop a table and purge all data and metadata files.
+
+        Args:
+            identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        """Rename a fully classified table name
+
+        Args:
+            from_identifier: Fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            to_identifier: New fully classified table name with its namespace. If the identifier is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            Table: the updated table instance with its metadata
+
+        Raises:
+            TableNotFoundError: If a table with the name does not exist
+        """
+
+    @abstractmethod
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        """Create a namespace in the catalog.
+
+        Args:
+            namespace: The namespace to be created. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+            properties: A string dictionary of properties for the given namespace
+
+        Raises:
+            AlreadyExistsError: If a namespace with the given name already exists
+        """
+
+    @abstractmethod
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        """Drop a namespace.
+
+        Args:
+            namespace: The namespace to be dropped. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+            NamespaceNotEmptyError: If the namespace is not empty
+        """
+
+    @abstractmethod
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        """List tables under the given namespace in the catalog.
+
+        If namespace not provided, will list all tables in the catalog.
+
+        Args:
+            namespace: the namespace to search. If the namespace is a string, it is split into a tuple on '.'. If it is a tuple, it is used as-is.
+
+        Returns:
+            List[Tuple[str, str]]: list of tuple of table namespace and their names.
+
+        Raises:
+            NamespaceNotFoundError: If a namespace with the given name does not exist
+        """
+
+    @abstractmethod
+    def list_namespaces(self) -> List[Identifier]:
+        """List namespaces from the given namespace. If not given, list top-level namespaces from the catalog.
+
+        Returns:
+            List[Identifier]: a List of namespace, where each element is a Tuple of namespace levels. Ex: ('com','org','dept')
+        """
+
+    @abstractmethod
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:

Review Comment:
   Addressed in a recent commit. Marking it resolved.



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

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

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


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


[GitHub] [iceberg] dhruv-pratap commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

Posted by GitBox <gi...@apache.org>.
dhruv-pratap commented on code in PR #4706:
URL: https://github.com/apache/iceberg/pull/4706#discussion_r876472390


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,418 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Metadata
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Metadata]
+
+    def __init__(self, name: str, properties: Metadata):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Metadata] = None,
+    ) -> Table:
+
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table {name} already exists in namespace {namespace}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = InMemoryCatalog.identifier_to_tuple(identifier)
+        namespace = InMemoryCatalog.namespace_from(identifier)
+        name = InMemoryCatalog.name_from(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {name} not found in the namespace {namespace}")
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = InMemoryCatalog.identifier_to_tuple(from_identifier)
+        from_namespace = InMemoryCatalog.namespace_from(from_identifier)
+        from_name = InMemoryCatalog.name_from(from_identifier)
+        try:
+            self.__tables.pop(from_identifier)
+        except KeyError:
+            raise NoSuchTableError(f"Table {from_name} not found in the namespace {from_namespace}")
+
+        renamed_table = Table()
+        to_identifier = InMemoryCatalog.identifier_to_tuple(to_identifier)
+        to_namespace = InMemoryCatalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        self.__tables[to_identifier] = renamed_table
+        return renamed_table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Metadata] = None) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace {namespace} already exists")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace {namespace} not empty")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        # Casting to make mypy happy
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace(self, namespace: Union[str, Identifier]) -> Metadata:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    def update_namespace_metadata(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Metadata] = None
+    ) -> None:
+        namespace = InMemoryCatalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            [self.__namespaces[namespace].pop(key) for key in removals]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace {namespace} not found in the catalog")
+
+    @staticmethod
+    def name_from(identifier: Union[str, Identifier]) -> str:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[-1]
+
+    @staticmethod
+    def namespace_from(identifier: Union[str, Identifier]) -> Identifier:
+        return InMemoryCatalog.identifier_to_tuple(identifier)[:-1]
+
+    @staticmethod
+    def identifier_to_tuple(identifier: Union[str, Identifier]) -> Identifier:
+        return identifier if isinstance(identifier, tuple) else tuple(str.split(identifier, "."))
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+
+
+def given_catalog_has_a_table(catalog: InMemoryCatalog) -> Table:
+    return catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+
+
+def test_namespace_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_namespace_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    namespace_from = InMemoryCatalog.namespace_from(identifier)
+    # Then
+    assert namespace_from == ("com", "organization", "department")
+
+
+def test_name_from_tuple():
+    # Given
+    identifier = ("com", "organization", "department", "my_table")
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_name_from_str():
+    # Given
+    identifier = "com.organization.department.my_table"
+    # When
+    name_from = InMemoryCatalog.name_from(identifier)
+    # Then
+    assert name_from == "my_table"
+
+
+def test_create_table(catalog: InMemoryCatalog):
+    table = catalog.create_table(
+        identifier=TEST_TABLE_IDENTIFIER,
+        schema=TEST_TABLE_SCHEMA,
+        location=TEST_TABLE_LOCATION,
+        partition_spec=TEST_TABLE_PARTITION_SPEC,
+        properties=TEST_TABLE_PROPERTIES,
+    )
+    assert catalog.load_table(TEST_TABLE_IDENTIFIER) == table

Review Comment:
   I'll add those checks as part of  #3227 



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4706: Python: Support iceberg base catalog in python library (#3245)

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


##########
python/tests/catalog/test_base.py:
##########
@@ -0,0 +1,401 @@
+#  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 Dict, List, Optional, Set, Union
+
+import pytest
+
+from iceberg.catalog.base import Catalog, Identifier, Properties
+from iceberg.exceptions import (
+    AlreadyExistsError,
+    NamespaceNotEmptyError,
+    NoSuchNamespaceError,
+    NoSuchTableError,
+)
+from iceberg.schema import Schema
+from iceberg.table.base import PartitionSpec, Table
+
+
+class InMemoryCatalog(Catalog):
+    """An in-memory catalog implementation for testing purposes."""
+
+    __tables: Dict[Identifier, Table]
+    __namespaces: Dict[Identifier, Properties]
+
+    def __init__(self, name: str, properties: Properties):
+        super().__init__(name, properties)
+        self.__tables = {}
+        self.__namespaces = {}
+
+    def create_table(
+        self,
+        identifier: Union[str, Identifier],
+        schema: Schema,
+        location: Optional[str] = None,
+        partition_spec: Optional[PartitionSpec] = None,
+        properties: Optional[Properties] = None,
+    ) -> Table:
+
+        identifier = Catalog.identifier_to_tuple(identifier)
+        namespace = Catalog.namespace_from(identifier)
+
+        if identifier in self.__tables:
+            raise AlreadyExistsError(f"Table already exists: {identifier}")
+        else:
+            if namespace not in self.__namespaces:
+                self.__namespaces[namespace] = {}
+
+            table = Table()
+            table.identifier = identifier
+            self.__tables[identifier] = table
+            return table
+
+    def load_table(self, identifier: Union[str, Identifier]) -> Table:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            return self.__tables[identifier]
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def drop_table(self, identifier: Union[str, Identifier]) -> None:
+        identifier = Catalog.identifier_to_tuple(identifier)
+        try:
+            self.__tables.pop(identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {identifier}") from error
+
+    def purge_table(self, identifier: Union[str, Identifier]) -> None:
+        self.drop_table(identifier)
+
+    def rename_table(self, from_identifier: Union[str, Identifier], to_identifier: Union[str, Identifier]) -> Table:
+        from_identifier = Catalog.identifier_to_tuple(from_identifier)
+        try:
+            table = self.__tables.pop(from_identifier)
+        except KeyError as error:
+            raise NoSuchTableError(f"Table does not exist: {from_identifier}") from error
+
+        to_identifier = Catalog.identifier_to_tuple(to_identifier)
+        to_namespace = Catalog.namespace_from(to_identifier)
+        if to_namespace not in self.__namespaces:
+            self.__namespaces[to_namespace] = {}
+
+        table.identifier = to_identifier
+        self.__tables[to_identifier] = table
+        return table
+
+    def create_namespace(self, namespace: Union[str, Identifier], properties: Optional[Properties] = None) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if namespace in self.__namespaces:
+            raise AlreadyExistsError(f"Namespace already exists: {namespace}")
+        else:
+            self.__namespaces[namespace] = properties if properties else {}
+
+    def drop_namespace(self, namespace: Union[str, Identifier]) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        if [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]:
+            raise NamespaceNotEmptyError(f"Namespace is not empty: {namespace}")
+        try:
+            self.__namespaces.pop(namespace)
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") from error
+
+    def list_tables(self, namespace: Optional[Union[str, Identifier]] = None) -> List[Identifier]:
+        if namespace:
+            namespace = Catalog.identifier_to_tuple(namespace)
+            list_tables = [table_identifier for table_identifier in self.__tables.keys() if namespace == table_identifier[:-1]]
+        else:
+            list_tables = list(self.__tables.keys())
+
+        return list_tables
+
+    def list_namespaces(self) -> List[Identifier]:
+        return list(self.__namespaces.keys())
+
+    def load_namespace_properties(self, namespace: Union[str, Identifier]) -> Properties:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        try:
+            return self.__namespaces[namespace]
+        except KeyError as error:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}") from error
+
+    def update_namespace_properties(
+        self, namespace: Union[str, Identifier], removals: Optional[Set[str]] = None, updates: Optional[Properties] = None
+    ) -> None:
+        namespace = Catalog.identifier_to_tuple(namespace)
+        removals = {} if not removals else removals
+        updates = [] if not updates else updates
+        if namespace in self.__namespaces:
+            for key in removals:
+                if key in self.__namespaces[namespace]:
+                    del self.__namespaces[namespace][key]
+            self.__namespaces[namespace].update(updates)
+        else:
+            raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}")
+
+
+TEST_TABLE_IDENTIFIER = ("com", "organization", "department", "my_table")
+TEST_TABLE_NAMESPACE = ("com", "organization", "department")
+TEST_TABLE_NAME = "my_table"
+TEST_TABLE_SCHEMA = Schema(schema_id=1)
+TEST_TABLE_LOCATION = "protocol://some/location"
+TEST_TABLE_PARTITION_SPEC = PartitionSpec()
+TEST_TABLE_PROPERTIES = {"key1": "value1", "key2": "value2"}
+NO_SUCH_TABLE_ERROR = "Table does not exist: \\('com', 'organization', 'department', 'my_table'\\)"

Review Comment:
   These should use string names, not tuples.



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

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

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


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