You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2022/08/19 22:44:32 UTC

[iceberg] branch master updated: Python: Add configuration (#5488)

This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 460124eb68 Python: Add configuration (#5488)
460124eb68 is described below

commit 460124eb689380b9e4ab8abf8bc64e3298339ddb
Author: Fokko Driesprong <fo...@apache.org>
AuthorDate: Sat Aug 20 00:44:27 2022 +0200

    Python: Add configuration (#5488)
---
 python/pyiceberg/catalog/__init__.py | 101 +++++++++++-
 python/pyiceberg/catalog/hive.py     |   4 +-
 python/pyiceberg/catalog/rest.py     |  29 ++--
 python/pyiceberg/cli/console.py      |  40 +++--
 python/pyiceberg/exceptions.py       |   4 +
 python/pyiceberg/typedef.py          |   8 +-
 python/pyiceberg/utils/config.py     | 146 ++++++++++++++++++
 python/tests/catalog/test_base.py    |   2 +-
 python/tests/catalog/test_hive.py    |  17 ++-
 python/tests/catalog/test_rest.py    |  70 +++++----
 python/tests/cli/test_console.py     | 289 ++++++++++++++++++-----------------
 python/tests/conftest.py             |   2 +-
 python/tests/utils/test_config.py    |  54 +++++++
 13 files changed, 536 insertions(+), 230 deletions(-)

diff --git a/python/pyiceberg/catalog/__init__.py b/python/pyiceberg/catalog/__init__.py
index 871db36265..c9ce2aab1f 100644
--- a/python/pyiceberg/catalog/__init__.py
+++ b/python/pyiceberg/catalog/__init__.py
@@ -17,14 +17,92 @@
 
 from __future__ import annotations
 
+import logging
 from abc import ABC, abstractmethod
 from dataclasses import dataclass
+from enum import Enum
+from typing import Callable
 
+from pyiceberg.exceptions import NotInstalledError
 from pyiceberg.schema import Schema
 from pyiceberg.table import Table
 from pyiceberg.table.partitioning import UNPARTITIONED_PARTITION_SPEC, PartitionSpec
 from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
-from pyiceberg.typedef import EMPTY_DICT, Identifier, Properties
+from pyiceberg.typedef import (
+    EMPTY_DICT,
+    Identifier,
+    Properties,
+    RecursiveDict,
+)
+from pyiceberg.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_ENV_CONFIG = Config()
+
+TYPE = "type"
+
+
+class CatalogType(Enum):
+    REST = "rest"
+    HIVE = "hive"
+
+
+def load_rest(name: str, conf: Properties) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    return RestCatalog(name, **conf)
+
+
+def load_hive(name: str, conf: Properties) -> Catalog:
+    try:
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        return HiveCatalog(name, **conf)
+    except ImportError as exc:
+        raise NotInstalledError("Apache Hive support not installed: pip install 'pyiceberg[hive]'") from exc
+
+
+AVAILABLE_CATALOGS: dict[CatalogType, Callable[[str, Properties], Catalog]] = {
+    CatalogType.REST: load_rest,
+    CatalogType.HIVE: load_hive,
+}
+
+
+def infer_catalog_type(catalog_properties: RecursiveDict) -> CatalogType | None:
+    """Tries to infer the type based on the dict
+
+    Args:
+        catalog_properties: Catalog properties
+
+    Returns:
+        The inferred type based on the provided properties
+    """
+    if uri := catalog_properties.get("uri"):
+        if isinstance(uri, str):
+            if uri.startswith("http"):
+                return CatalogType.REST
+            elif uri.startswith("thrift"):
+                return CatalogType.HIVE
+    return None
+
+
+def load_catalog(name: str, **properties: str | None) -> Catalog:
+    env = _ENV_CONFIG.get_catalog_config(name)
+    conf = merge_config(env or {}, properties)
+
+    if provided_catalog_type := conf.get(TYPE):
+        catalog_type = CatalogType[provided_catalog_type.upper()]
+    else:
+        if inferred_catalog_type := infer_catalog_type(conf):
+            catalog_type = inferred_catalog_type
+        else:
+            raise ValueError(f"Invalid configuration. Could not determine the catalog type: {properties}")
+
+    if catalog_type:
+        return AVAILABLE_CATALOGS[catalog_type](name, conf)
+
+    raise ValueError(f"Could not initialize catalog with the following properties: {properties}")
 
 
 @dataclass
@@ -48,13 +126,30 @@ class Catalog(ABC):
         properties (Properties): Catalog properties
     """
 
-    name: str | None
+    name: str
     properties: Properties
 
-    def __init__(self, name: str | None, **properties: str):
+    def __init__(self, name: str, **properties: str):
         self.name = name
         self.properties = properties
 
+    def property(self, key: str) -> str:
+        """Returns a property from the properties variable. If it doesn't exist, it will raise an error.
+
+        Args:
+            key: The key of the property
+
+        Returns: The value of the property
+
+        Raises:
+            ValueError: When the property cannot be found, with a pointer on how to set the property.
+        """
+        if key not in self.properties:
+            raise ValueError(
+                f"{type(self).__name__} expects an {key} property. Please set in config or using environment variable PYICEBERG_CATALOG__{self.name.upper()}__{key.upper()}"
+            )
+        return self.properties[key]
+
     @abstractmethod
     def create_table(
         self,
diff --git a/python/pyiceberg/catalog/hive.py b/python/pyiceberg/catalog/hive.py
index e218051fd2..0d4e852aa4 100644
--- a/python/pyiceberg/catalog/hive.py
+++ b/python/pyiceberg/catalog/hive.py
@@ -235,9 +235,9 @@ class HiveCatalog(Catalog):
 
         return tuple_identifier[0], tuple_identifier[1]
 
-    def __init__(self, name: str, uri: str, **properties: str):
+    def __init__(self, name: str, **properties: str):
         super().__init__(name, **properties)
-        self._client = _HiveClient(uri)
+        self._client = _HiveClient(self.property("uri"))
 
     def _convert_hive_into_iceberg(self, table: HiveTable, io: FileIO) -> Table:
         properties: Dict[str, str] = table.parameters
diff --git a/python/pyiceberg/catalog/rest.py b/python/pyiceberg/catalog/rest.py
index d3500beb29..02509aad72 100644
--- a/python/pyiceberg/catalog/rest.py
+++ b/python/pyiceberg/catalog/rest.py
@@ -164,16 +164,11 @@ class OAuthErrorResponse(IcebergBaseModel):
 
 class RestCatalog(Catalog):
     token: Optional[str]
-    config: Properties
-
     uri: str
 
     def __init__(
         self,
-        name: Optional[str],
-        uri: str,
-        credential: Optional[str] = None,
-        token: Optional[str] = None,
+        name: str,
         **properties: str,
     ):
         """Rest Catalog
@@ -182,21 +177,15 @@ class RestCatalog(Catalog):
 
         Args:
             name: Name to identify the catalog
-            uri: The base-url of the REST Catalog endpoint
-            credential: The credentials for authentication against the client
-            token: The bearer token
             properties: Properties that are passed along to the configuration
         """
-        self.uri = uri
-        if credential:
-            self.token = self._fetch_access_token(credential)
-        elif token:
-            self.token = token
-        else:
-            self.token = None
-        self.config = self._fetch_config(properties)
         super().__init__(name, **properties)
 
+        self.uri = self.property("uri")
+        if credential := self.properties.get("credential"):
+            properties["token"] = self._fetch_access_token(credential)
+        super().__init__(name, **self._fetch_config(properties))
+
     def _check_valid_namespace_identifier(self, identifier: Union[str, Identifier]) -> Identifier:
         """The identifier should have at least one element"""
         identifier_tuple = Catalog.identifier_to_tuple(identifier)
@@ -210,8 +199,8 @@ class RestCatalog(Catalog):
             "Content-type": "application/json",
             "X-Client-Version": __version__,
         }
-        if self.token:
-            headers[AUTHORIZATION_HEADER] = f"{BEARER_PREFIX} {self.token}"
+        if token := self.properties.get("token"):
+            headers[AUTHORIZATION_HEADER] = f"{BEARER_PREFIX} {token}"
         return headers
 
     def url(self, endpoint: str, prefixed: bool = True, **kwargs) -> str:
@@ -229,7 +218,7 @@ class RestCatalog(Catalog):
         url = url + "v1/" if url.endswith("/") else url + "/v1/"
 
         if prefixed:
-            url += self.config.get(PREFIX, "")
+            url += self.properties.get(PREFIX, "")
             url = url if url.endswith("/") else url + "/"
 
         return url + endpoint.format(**kwargs)
diff --git a/python/pyiceberg/cli/console.py b/python/pyiceberg/cli/console.py
index 5ce6cf4c04..3f9647654a 100644
--- a/python/pyiceberg/cli/console.py
+++ b/python/pyiceberg/cli/console.py
@@ -15,7 +15,6 @@
 # specific language governing permissions and limitations
 # under the License.
 # pylint: disable=broad-except,redefined-builtin,redefined-outer-name
-import os
 from functools import wraps
 from typing import (
     Dict,
@@ -28,7 +27,7 @@ from typing import (
 import click
 from click import Context
 
-from pyiceberg.catalog import Catalog
+from pyiceberg.catalog import Catalog, load_catalog
 from pyiceberg.catalog.hive import HiveCatalog
 from pyiceberg.catalog.rest import RestCatalog
 from pyiceberg.cli.output import ConsoleOutput, JsonOutput, Output
@@ -55,19 +54,17 @@ def catch_exception():
 
 
 @click.group()
-@click.option("--catalog", default=None)
+@click.option("--catalog", default="default")
 @click.option("--output", type=click.Choice(["text", "json"]), default="text")
 @click.option("--uri")
 @click.option("--credential")
 @click.pass_context
-def run(ctx: Context, catalog: Optional[str], output: str, uri: Optional[str], credential: Optional[str]):
-    uri_env_var = "PYICEBERG_URI"
-    credential_env_var = "PYICEBERG_CREDENTIAL"
-
-    if not uri:
-        uri = os.environ.get(uri_env_var)
-    if not credential:
-        credential = os.environ.get(credential_env_var)
+def run(ctx: Context, catalog: str, output: str, uri: Optional[str], credential: Optional[str]):
+    properties = {}
+    if uri:
+        properties["uri"] = uri
+    if credential:
+        properties["credential"] = credential
 
     ctx.ensure_object(dict)
     if output == "text":
@@ -75,21 +72,18 @@ def run(ctx: Context, catalog: Optional[str], output: str, uri: Optional[str], c
     else:
         ctx.obj["output"] = JsonOutput()
 
-    if not uri:
-        ctx.obj["output"].exception(
-            ValueError(f"Missing uri. Please provide using --uri or using environment variable {uri_env_var}")
-        )
+    try:
+        try:
+            ctx.obj["catalog"] = load_catalog(catalog, **properties)
+        except ValueError as exc:
+            raise ValueError(
+                f"URI missing, please provide using --uri, the config or environment variable PYICEBERG_CATALOG__{catalog.upper()}__URI"
+            ) from exc
+    except Exception as e:
+        ctx.obj["output"].exception(e)
         ctx.exit(1)
 
-    assert uri  # for mypy
-
-    for scheme, catalog_type in SUPPORTED_CATALOGS.items():
-        if uri.startswith(scheme):
-            ctx.obj["catalog"] = catalog_type(catalog, uri=uri, credential=credential)  # type: ignore
-            break
-
     if not isinstance(ctx.obj["catalog"], Catalog):
-
         ctx.obj["output"].exception(
             ValueError("Could not determine catalog type from uri. REST (http/https) and Hive (thrift) is supported")
         )
diff --git a/python/pyiceberg/exceptions.py b/python/pyiceberg/exceptions.py
index ea2d542e90..86e8a8102f 100644
--- a/python/pyiceberg/exceptions.py
+++ b/python/pyiceberg/exceptions.py
@@ -78,3 +78,7 @@ class OAuthError(RESTError):
 
 class NoSuchPropertyException(Exception):
     """When a property is missing"""
+
+
+class NotInstalledError(Exception):
+    """When an optional dependency is not installed"""
diff --git a/python/pyiceberg/typedef.py b/python/pyiceberg/typedef.py
index f9fa9c96f6..dc95702813 100644
--- a/python/pyiceberg/typedef.py
+++ b/python/pyiceberg/typedef.py
@@ -15,7 +15,12 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from typing import Any, Dict, Tuple
+from typing import (
+    Any,
+    Dict,
+    Tuple,
+    Union,
+)
 
 
 class FrozenDict(Dict):
@@ -30,3 +35,4 @@ EMPTY_DICT = FrozenDict()
 
 Identifier = Tuple[str, ...]
 Properties = Dict[str, str]
+RecursiveDict = Dict[str, Union[str, "RecursiveDict"]]  # type: ignore
diff --git a/python/pyiceberg/utils/config.py b/python/pyiceberg/utils/config.py
new file mode 100644
index 0000000000..c13ec13272
--- /dev/null
+++ b/python/pyiceberg/utils/config.py
@@ -0,0 +1,146 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import logging
+import os
+from typing import Any, List, Optional
+
+import yaml
+
+from pyiceberg.typedef import FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg_"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yaml"
+
+logger = logging.getLogger(__name__)
+
+
+def _coalesce(lhs: Optional[Any], rhs: Optional[Any]) -> Optional[Any]:
+    return lhs or rhs
+
+
+def merge_config(lhs: RecursiveDict, rhs: RecursiveDict) -> RecursiveDict:
+    """merges right-hand side into the left-hand side"""
+    new_config = lhs.copy()
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key in new_config:
+            lhs_value = new_config[rhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                # If they are both dicts, then we have to go deeper
+                new_config[rhs_key] = merge_config(lhs_value, rhs_value)
+            else:
+                # Take the non-null value, with precedence on rhs
+                new_config[rhs_key] = _coalesce(rhs_value, lhs_value)
+        else:
+            # New key
+            new_config[rhs_key] = rhs_value
+
+    return new_config
+
+
+def _lowercase_dictionary_keys(input_dict: RecursiveDict) -> RecursiveDict:
+    """Lowers all the keys of a dictionary in a recursive manner, to make the lookup case-insensitive"""
+    return {k.lower(): _lowercase_dictionary_keys(v) if isinstance(v, dict) else v for k, v in input_dict.items()}
+
+
+class Config:
+    config: RecursiveDict
+
+    def __init__(self):
+        config = self._from_configuration_files() or {}
+        config = merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> Optional[RecursiveDict]:
+        """Loads the first configuration file that its finds
+
+        Will first look in the PYICEBERG_HOME env variable,
+        and then in the home directory.
+        """
+
+        def _load_yaml(directory: Optional[str]) -> Optional[RecursiveDict]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        file_config = yaml.safe_load(f)
+                        file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                        return file_config_lowercase
+            return None
+
+        # Give priority to the PYICEBERG_HOME directory
+        if pyiceberg_home_config := _load_yaml(os.environ.get(PYICEBERG_HOME)):
+            return pyiceberg_home_config
+        # Look into the home directory
+        if pyiceberg_home_config := _load_yaml(os.environ.get(HOME)):
+            return pyiceberg_home_config
+        # Didn't find a config
+        return None
+
+    @staticmethod
+    def _from_environment_variables(config: RecursiveDict) -> RecursiveDict:
+        """Reads the environment variables, to check if there are any prepended by PYICEBERG_
+
+        Args:
+            config: Existing configuration that's being amended with configuration from environment variables
+
+        Returns:
+            Amended configuration
+        """
+
+        def set_property(_config: RecursiveDict, path: List[str], config_value: str):
+            while len(path) > 0:
+                element = path.pop(0)
+                if len(path) == 0:
+                    # We're at the end
+                    _config[element] = config_value
+                else:
+                    # We have to go deeper
+                    if element not in _config:
+                        _config[element] = {}
+                    if isinstance(_config[element], dict):
+                        _config = _config[element]  # type: ignore
+                    else:
+                        raise ValueError(
+                            f"Incompatible configurations, merging dict with a value: {'.'.join(path)}, value: {config_value}"
+                        )
+
+        for env_var, config_value in os.environ.items():
+            # Make it lowercase to make it case-insensitive
+            env_var_lower = env_var.lower()
+            if env_var_lower.startswith(PYICEBERG.lower()):
+                key = env_var_lower[len(PYICEBERG) :]
+                parts = key.split("__")
+                parts_normalized = [part.replace("_", "-") for part in parts]
+                set_property(config, parts_normalized, config_value)
+
+        return config
+
+    def get_catalog_config(self, catalog_name: str) -> Optional[RecursiveDict]:
+        if CATALOG in self.config:
+            catalog_name_lower = catalog_name.lower()
+            catalogs = self.config[CATALOG]
+            if not isinstance(catalogs, dict):
+                raise ValueError(f"Catalog configurations needs to be an object: {catalog_name}")
+            if catalog_name_lower in catalogs:
+                catalog_conf = catalogs[catalog_name_lower]
+                assert isinstance(catalog_conf, dict), f"Configuration path catalogs.{catalog_name_lower} needs to be an object"
+                return catalog_conf
+        return None
diff --git a/python/tests/catalog/test_base.py b/python/tests/catalog/test_base.py
index 9ac65ce02e..a01b3f5788 100644
--- a/python/tests/catalog/test_base.py
+++ b/python/tests/catalog/test_base.py
@@ -53,7 +53,7 @@ class InMemoryCatalog(Catalog):
     __tables: Dict[Identifier, Table]
     __namespaces: Dict[Identifier, Properties]
 
-    def __init__(self, name: str, properties: Properties):
+    def __init__(self, name: str, **properties: str):
         super().__init__(name, **properties)
         self.__tables = {}
         self.__namespaces = {}
diff --git a/python/tests/catalog/test_hive.py b/python/tests/catalog/test_hive.py
index e6b06e762d..ea7a193c1c 100644
--- a/python/tests/catalog/test_hive.py
+++ b/python/tests/catalog/test_hive.py
@@ -165,6 +165,16 @@ def hive_database(tmp_path_factory) -> HiveDatabase:
     )
 
 
+def test_no_uri_supplied():
+    with pytest.raises(ValueError) as exc_info:
+        HiveCatalog("production")
+
+    assert (
+        "HiveCatalog expects an uri property. Please set in config or using environment variable PYICEBERG_CATALOG__PRODUCTION__URI"
+        in str(exc_info.value)
+    )
+
+
 def test_check_number_of_namespaces(table_schema_simple: Schema):
     catalog = HiveCatalog(HIVE_CATALOG_NAME, uri=HIVE_METASTORE_FAKE_URL)
 
@@ -480,13 +490,6 @@ def test_list_namespaces():
     catalog._client.__enter__().get_all_databases.assert_called()
 
 
-def test_list_namespaces_with_parent():
-    catalog = HiveCatalog(HIVE_CATALOG_NAME, uri=HIVE_METASTORE_FAKE_URL)
-
-    # Hive does not support hierarchical namespaces
-    assert catalog.list_namespaces(("accounting",)) == []
-
-
 def test_drop_table():
     catalog = HiveCatalog(HIVE_CATALOG_NAME, uri=HIVE_METASTORE_FAKE_URL)
 
diff --git a/python/tests/catalog/test_rest.py b/python/tests/catalog/test_rest.py
index cff7563ab5..2f112a6075 100644
--- a/python/tests/catalog/test_rest.py
+++ b/python/tests/catalog/test_rest.py
@@ -63,6 +63,16 @@ def rest_mock(requests_mock: Mocker):
     return requests_mock
 
 
+def test_no_uri_supplied():
+    with pytest.raises(ValueError) as exc_info:
+        RestCatalog("production")
+
+    assert (
+        "RestCatalog expects an uri property. Please set in config or using environment variable PYICEBERG_CATALOG__PRODUCTION__URI"
+        in str(exc_info.value)
+    )
+
+
 def test_token_200(rest_mock: Mocker):
     rest_mock.post(
         f"{TEST_URI}v1/oauth/tokens",
@@ -74,7 +84,7 @@ def test_token_200(rest_mock: Mocker):
         },
         status_code=200,
     )
-    assert RestCatalog("rest", TEST_URI, TEST_CREDENTIALS).token == TEST_TOKEN
+    assert RestCatalog("rest", uri=TEST_URI, credential=TEST_CREDENTIALS).property("token") == TEST_TOKEN
 
 
 def test_token_400(rest_mock: Mocker):
@@ -85,7 +95,7 @@ def test_token_400(rest_mock: Mocker):
     )
 
     with pytest.raises(OAuthError) as e:
-        RestCatalog("rest", TEST_URI, credential=TEST_CREDENTIALS)
+        RestCatalog("rest", uri=TEST_URI, credential=TEST_CREDENTIALS)
     assert str(e.value) == "invalid_client: Credentials for key invalid_key do not match"
 
 
@@ -104,7 +114,7 @@ def test_token_401(rest_mock: Mocker):
     )
 
     with pytest.raises(BadCredentialsError) as e:
-        RestCatalog("rest", TEST_URI, credential=TEST_CREDENTIALS)
+        RestCatalog("rest", uri=TEST_URI, credential=TEST_CREDENTIALS)
     assert message in str(e.value)
 
 
@@ -116,7 +126,7 @@ def test_list_tables_200(rest_mock: Mocker):
         status_code=200,
     )
 
-    assert RestCatalog("rest", TEST_URI, token=TEST_TOKEN).list_tables(namespace) == [("examples", "fooshare")]
+    assert RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).list_tables(namespace) == [("examples", "fooshare")]
 
 
 def test_list_tables_404(rest_mock: Mocker):
@@ -133,7 +143,7 @@ def test_list_tables_404(rest_mock: Mocker):
         status_code=404,
     )
     with pytest.raises(NoSuchNamespaceError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).list_tables(namespace)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).list_tables(namespace)
     assert "Namespace does not exist" in str(e.value)
 
 
@@ -143,7 +153,7 @@ def test_list_namespaces_200(rest_mock: Mocker):
         json={"namespaces": [["default"], ["examples"], ["fokko"], ["system"]]},
         status_code=200,
     )
-    assert RestCatalog("rest", TEST_URI, token=TEST_TOKEN).list_namespaces() == [
+    assert RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).list_namespaces() == [
         ("default",),
         ("examples",),
         ("fokko",),
@@ -169,7 +179,7 @@ def test_create_namespace_200(rest_mock: Mocker):
         json={"namespace": [namespace], "properties": {}},
         status_code=200,
     )
-    RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_namespace(namespace)
+    RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_namespace(namespace)
 
 
 def test_create_namespace_409(rest_mock: Mocker):
@@ -186,7 +196,7 @@ def test_create_namespace_409(rest_mock: Mocker):
         status_code=409,
     )
     with pytest.raises(NamespaceAlreadyExistsError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_namespace(namespace)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_namespace(namespace)
     assert "Namespace already exists" in str(e.value)
 
 
@@ -204,7 +214,7 @@ def test_drop_namespace_404(rest_mock: Mocker):
         status_code=404,
     )
     with pytest.raises(NoSuchNamespaceError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_namespace(namespace)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_namespace(namespace)
     assert "Namespace does not exist" in str(e.value)
 
 
@@ -215,7 +225,7 @@ def test_load_namespace_properties_200(rest_mock: Mocker):
         json={"namespace": ["fokko"], "properties": {"prop": "yes"}},
         status_code=204,
     )
-    assert RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_namespace_properties(namespace) == {"prop": "yes"}
+    assert RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_namespace_properties(namespace) == {"prop": "yes"}
 
 
 def test_load_namespace_properties_404(rest_mock: Mocker):
@@ -232,7 +242,7 @@ def test_load_namespace_properties_404(rest_mock: Mocker):
         status_code=404,
     )
     with pytest.raises(NoSuchNamespaceError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_namespace_properties(namespace)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_namespace_properties(namespace)
     assert "Namespace does not exist" in str(e.value)
 
 
@@ -242,7 +252,9 @@ def test_update_namespace_properties_200(rest_mock: Mocker):
         json={"removed": [], "updated": ["prop"], "missing": ["abc"]},
         status_code=200,
     )
-    response = RestCatalog("rest", TEST_URI, token=TEST_TOKEN).update_namespace_properties(("fokko",), {"abc"}, {"prop": "yes"})
+    response = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).update_namespace_properties(
+        ("fokko",), {"abc"}, {"prop": "yes"}
+    )
 
     assert response == PropertiesUpdateSummary(removed=[], updated=["prop"], missing=["abc"])
 
@@ -260,7 +272,7 @@ def test_update_namespace_properties_404(rest_mock: Mocker):
         status_code=404,
     )
     with pytest.raises(NoSuchNamespaceError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).update_namespace_properties(("fokko",), {"abc"}, {"prop": "yes"})
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).update_namespace_properties(("fokko",), {"abc"}, {"prop": "yes"})
     assert "Namespace does not exist" in str(e.value)
 
 
@@ -337,7 +349,7 @@ def test_load_table_200(rest_mock: Mocker):
         },
         status_code=200,
     )
-    table = RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_table(("fokko", "table"))
+    table = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_table(("fokko", "table"))
     assert table == Table(
         identifier=("rest", "fokko", "table"),
         metadata_location="s3://warehouse/database/table/metadata/00001-5f2f8166-244c-4eae-ac36-384ecdec81fc.gz.metadata.json",
@@ -431,7 +443,7 @@ def test_load_table_404(rest_mock: Mocker):
     )
 
     with pytest.raises(NoSuchTableError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_table(("fokko", "does_not_exists"))
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_table(("fokko", "does_not_exists"))
     assert "Table does not exist" in str(e.value)
 
 
@@ -449,7 +461,7 @@ def test_drop_table_404(rest_mock: Mocker):
     )
 
     with pytest.raises(NoSuchTableError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_table(("fokko", "does_not_exists"))
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_table(("fokko", "does_not_exists"))
     assert "Table does not exist" in str(e.value)
 
 
@@ -512,7 +524,7 @@ def test_create_table_200(rest_mock: Mocker, table_schema_simple: Schema):
         },
         status_code=200,
     )
-    table = RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_table(
+    table = RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_table(
         identifier=("fokko", "fokko2"),
         schema=table_schema_simple,
         location=None,
@@ -583,7 +595,7 @@ def test_create_table_409(rest_mock, table_schema_simple: Schema):
     )
 
     with pytest.raises(TableAlreadyExistsError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_table(
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_table(
             identifier=("fokko", "fokko2"),
             schema=table_schema_simple,
             location=None,
@@ -604,7 +616,7 @@ def test_delete_namespace_204(rest_mock: Mocker):
         json={},
         status_code=204,
     )
-    RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_namespace(namespace)
+    RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_namespace(namespace)
 
 
 def test_delete_table_204(rest_mock: Mocker):
@@ -613,7 +625,7 @@ def test_delete_table_204(rest_mock: Mocker):
         json={},
         status_code=204,
     )
-    RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_table(("example", "fokko"))
+    RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_table(("example", "fokko"))
 
 
 def test_delete_table_404(rest_mock: Mocker):
@@ -629,7 +641,7 @@ def test_delete_table_404(rest_mock: Mocker):
         status_code=404,
     )
     with pytest.raises(NoSuchTableError) as e:
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_table(("example", "fokko"))
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_table(("example", "fokko"))
     assert "Table does not exist" in str(e.value)
 
 
@@ -637,7 +649,7 @@ def test_create_table_missing_namespace(rest_mock: Mocker, table_schema_simple:
     table = "table"
     with pytest.raises(NoSuchTableError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_table(table, table_schema_simple)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_table(table, table_schema_simple)
     assert f"Missing namespace or invalid identifier: {table}" in str(e.value)
 
 
@@ -645,7 +657,7 @@ def test_load_table_invalid_namespace(rest_mock: Mocker):
     table = "table"
     with pytest.raises(NoSuchTableError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_table(table)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_table(table)
     assert f"Missing namespace or invalid identifier: {table}" in str(e.value)
 
 
@@ -653,7 +665,7 @@ def test_drop_table_invalid_namespace(rest_mock: Mocker):
     table = "table"
     with pytest.raises(NoSuchTableError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_table(table)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_table(table)
     assert f"Missing namespace or invalid identifier: {table}" in str(e.value)
 
 
@@ -661,33 +673,33 @@ def test_purge_table_invalid_namespace(rest_mock: Mocker):
     table = "table"
     with pytest.raises(NoSuchTableError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).purge_table(table)
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).purge_table(table)
     assert f"Missing namespace or invalid identifier: {table}" in str(e.value)
 
 
 def test_create_namespace_invalid_namespace(rest_mock: Mocker):
     with pytest.raises(NoSuchNamespaceError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).create_namespace(())
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).create_namespace(())
     assert "Empty namespace identifier" in str(e.value)
 
 
 def test_drop_namespace_invalid_namespace(rest_mock: Mocker):
     with pytest.raises(NoSuchNamespaceError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).drop_namespace(())
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).drop_namespace(())
     assert "Empty namespace identifier" in str(e.value)
 
 
 def test_load_namespace_properties_invalid_namespace(rest_mock: Mocker):
     with pytest.raises(NoSuchNamespaceError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).load_namespace_properties(())
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).load_namespace_properties(())
     assert "Empty namespace identifier" in str(e.value)
 
 
 def test_update_namespace_properties_invalid_namespace(rest_mock: Mocker):
     with pytest.raises(NoSuchNamespaceError) as e:
         # Missing namespace
-        RestCatalog("rest", TEST_URI, token=TEST_TOKEN).update_namespace_properties(())
+        RestCatalog("rest", uri=TEST_URI, token=TEST_TOKEN).update_namespace_properties(())
     assert "Empty namespace identifier" in str(e.value)
diff --git a/python/tests/cli/test_console.py b/python/tests/cli/test_console.py
index 704e6729f5..9f8ac2245a 100644
--- a/python/tests/cli/test_console.py
+++ b/python/tests/cli/test_console.py
@@ -119,20 +119,23 @@ class MockCatalog(Catalog):
             raise NoSuchNamespaceError(f"Namespace does not exist: {namespace}")
 
 
-MOCK_ENVIRONMENT = {"PYICEBERG_URI": "test://doesnotexist"}
-MOCK_CATALOGS = {"test": MockCatalog}
+MOCK_CATALOG = MockCatalog("production", uri="http://somewhere")
+MOCK_ENVIRONMENT = {"PYICEBERG_CATALOG__PRODUCTION__URI": "test://doesnotexist"}
 
 
 def test_missing_uri():
     runner = CliRunner()
     result = runner.invoke(run, ["list"])
     assert result.exit_code == 1
-    assert result.output == "Missing uri. Please provide using --uri or using environment variable \nPYICEBERG_URI\n"
+    assert (
+        result.output
+        == "URI missing, please provide using --uri, the config or environment variable \nPYICEBERG_CATALOG__DEFAULT__URI\n"
+    )
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_list_root():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_list_root(_):
     runner = CliRunner()
     result = runner.invoke(run, ["list"])
     assert result.exit_code == 0
@@ -140,8 +143,8 @@ def test_list_root():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_list_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_list_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["list", "default"])
     assert result.exit_code == 0
@@ -149,8 +152,8 @@ def test_list_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_describe_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_describe_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["describe", "default"])
     assert result.exit_code == 0
@@ -158,8 +161,8 @@ def test_describe_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_describe_namespace_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_describe_namespace_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["describe", "doesnotexist"])
     assert result.exit_code == 1
@@ -167,8 +170,8 @@ def test_describe_namespace_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_describe_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_describe_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["describe", "default.foo"])
     assert result.exit_code == 0
@@ -199,8 +202,8 @@ Properties            read.split.target.size  134217728
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_describe_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_describe_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["describe", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -208,8 +211,8 @@ def test_describe_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_schema():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_schema(_):
     runner = CliRunner()
     result = runner.invoke(run, ["schema", "default.foo"])
     assert result.exit_code == 0
@@ -223,8 +226,8 @@ z  long
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_schema_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_schema_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["schema", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -232,8 +235,8 @@ def test_schema_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_spec():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_spec(_):
     runner = CliRunner()
     result = runner.invoke(run, ["spec", "default.foo"])
     assert result.exit_code == 0
@@ -247,8 +250,8 @@ def test_spec():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_spec_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_spec_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["spec", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -256,8 +259,8 @@ def test_spec_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_uuid():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_uuid(_):
     runner = CliRunner()
     result = runner.invoke(run, ["uuid", "default.foo"])
     assert result.exit_code == 0
@@ -265,8 +268,8 @@ def test_uuid():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_uuid_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_uuid_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["uuid", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -274,8 +277,8 @@ def test_uuid_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_location():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_location(_):
     runner = CliRunner()
     result = runner.invoke(run, ["location", "default.foo"])
     assert result.exit_code == 0
@@ -283,8 +286,8 @@ def test_location():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_location_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_location_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["location", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -292,8 +295,8 @@ def test_location_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_drop_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_drop_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["drop", "table", "default.foo"])
     assert result.exit_code == 0
@@ -301,8 +304,8 @@ def test_drop_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_drop_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_drop_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["drop", "table", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -310,8 +313,8 @@ def test_drop_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_drop_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_drop_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["drop", "namespace", "default"])
     assert result.exit_code == 0
@@ -319,8 +322,8 @@ def test_drop_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_drop_namespace_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_drop_namespace_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["drop", "namespace", "doesnotexit"])
     assert result.exit_code == 1
@@ -328,8 +331,8 @@ def test_drop_namespace_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_rename_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_rename_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["rename", "default.foo", "default.bar"])
     assert result.exit_code == 0
@@ -337,8 +340,8 @@ def test_rename_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_rename_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_rename_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["rename", "default.doesnotexit", "default.bar"])
     assert result.exit_code == 1
@@ -346,8 +349,8 @@ def test_rename_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "default.foo"])
     assert result.exit_code == 0
@@ -355,8 +358,8 @@ def test_properties_get_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_table_specific_property():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_table_specific_property(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "default.foo", "read.split.target.size"])
     assert result.exit_code == 0
@@ -364,8 +367,8 @@ def test_properties_get_table_specific_property():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_table_specific_property_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_table_specific_property_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "default.foo", "doesnotexist"])
     assert result.exit_code == 1
@@ -373,8 +376,8 @@ def test_properties_get_table_specific_property_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "doesnotexist"])
     assert result.exit_code == 1
@@ -382,8 +385,8 @@ def test_properties_get_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "default"])
     assert result.exit_code == 0
@@ -391,8 +394,8 @@ def test_properties_get_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_get_namespace_specific_property():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_get_namespace_specific_property(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "get", "default", "location"])
     assert result.exit_code == 0
@@ -400,8 +403,8 @@ def test_properties_get_namespace_specific_property():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_set_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_set_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "set", "namespace", "default", "location", "s3://new_location"])
     assert result.exit_code == 0
@@ -409,8 +412,8 @@ def test_properties_set_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_set_namespace_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_set_namespace_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "set", "namespace", "doesnotexist", "location", "s3://new_location"])
     assert result.exit_code == 1
@@ -418,8 +421,8 @@ def test_properties_set_namespace_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_set_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_set_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "set", "table", "default.foo", "location", "s3://new_location"])
     assert result.exit_code == 1
@@ -427,8 +430,8 @@ def test_properties_set_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_set_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_set_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "set", "table", "default.doesnotexist", "location", "s3://new_location"])
     assert result.exit_code == 1
@@ -436,8 +439,8 @@ def test_properties_set_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_remove_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_remove_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "remove", "namespace", "default", "location"])
     assert result.exit_code == 0
@@ -445,8 +448,8 @@ def test_properties_remove_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_remove_namespace_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_remove_namespace_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "remove", "namespace", "doesnotexist", "location"])
     assert result.exit_code == 1
@@ -454,8 +457,8 @@ def test_properties_remove_namespace_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_remove_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_remove_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "remove", "table", "default.foo", "read.split.target.size"])
     assert result.exit_code == 1
@@ -463,8 +466,8 @@ def test_properties_remove_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_remove_table_property_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_remove_table_property_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "remove", "table", "default.foo", "doesnotexist"])
     assert result.exit_code == 1
@@ -472,8 +475,8 @@ def test_properties_remove_table_property_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_properties_remove_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_properties_remove_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["properties", "remove", "table", "default.doesnotexist", "location"])
     assert result.exit_code == 1
@@ -481,8 +484,8 @@ def test_properties_remove_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_list_root():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_list_root(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "list"])
     assert result.exit_code == 0
@@ -490,8 +493,8 @@ def test_json_list_root():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_list_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_list_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "list", "default"])
     assert result.exit_code == 0
@@ -499,8 +502,8 @@ def test_json_list_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_describe_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_describe_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "describe", "default"])
     assert result.exit_code == 0
@@ -508,8 +511,8 @@ def test_json_describe_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_describe_namespace_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_describe_namespace_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "describe", "doesnotexist"])
     assert result.exit_code == 1
@@ -517,8 +520,8 @@ def test_json_describe_namespace_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_describe_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_describe_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "describe", "default.foo"])
     assert result.exit_code == 0
@@ -529,8 +532,8 @@ def test_json_describe_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_describe_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_describe_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "describe", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -540,8 +543,8 @@ def test_json_describe_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_schema():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_schema(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "schema", "default.foo"])
     assert result.exit_code == 0
@@ -552,8 +555,8 @@ def test_json_schema():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_schema_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_schema_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "schema", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -561,8 +564,8 @@ def test_json_schema_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_spec():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_spec(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "spec", "default.foo"])
     assert result.exit_code == 0
@@ -573,8 +576,8 @@ def test_json_spec():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_spec_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_spec_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "spec", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -582,8 +585,8 @@ def test_json_spec_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_uuid():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_uuid(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "uuid", "default.foo"])
     assert result.exit_code == 0
@@ -591,8 +594,8 @@ def test_json_uuid():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_uuid_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_uuid_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "uuid", "default.doesnotexit"])
     assert result.exit_code == 1
@@ -600,8 +603,8 @@ def test_json_uuid_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_location():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_location(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "location", "default.foo"])
     assert result.exit_code == 0
@@ -609,8 +612,8 @@ def test_json_location():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_location_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_location_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "location", "default.doesnotexist"])
     assert result.exit_code == 1
@@ -618,8 +621,8 @@ def test_json_location_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_drop_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_drop_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "drop", "table", "default.foo"])
     assert result.exit_code == 0
@@ -627,8 +630,8 @@ def test_json_drop_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_drop_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_drop_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "drop", "table", "default.doesnotexist"])
     assert result.exit_code == 1
@@ -636,8 +639,8 @@ def test_json_drop_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_drop_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_drop_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "drop", "namespace", "default"])
     assert result.exit_code == 0
@@ -645,8 +648,8 @@ def test_json_drop_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_drop_namespace_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_drop_namespace_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "drop", "namespace", "doesnotexist"])
     assert result.exit_code == 1
@@ -654,8 +657,8 @@ def test_json_drop_namespace_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_rename_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_rename_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "rename", "default.foo", "default.bar"])
     assert result.exit_code == 0
@@ -663,8 +666,8 @@ def test_json_rename_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_rename_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_rename_table_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "rename", "default.doesnotexit", "default.bar"])
     assert result.exit_code == 1
@@ -672,8 +675,8 @@ def test_json_rename_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "default.foo"])
     assert result.exit_code == 0
@@ -681,8 +684,8 @@ def test_json_properties_get_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_table_specific_property():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_table_specific_property(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "default.foo", "read.split.target.size"])
     assert result.exit_code == 0
@@ -690,8 +693,8 @@ def test_json_properties_get_table_specific_property():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_table_specific_property_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_table_specific_property_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "default.foo", "doesnotexist"])
     assert result.exit_code == 1
@@ -702,8 +705,8 @@ def test_json_properties_get_table_specific_property_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "doesnotexist"])
     assert result.exit_code == 1
@@ -711,8 +714,8 @@ def test_json_properties_get_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "default"])
     assert result.exit_code == 0
@@ -720,8 +723,8 @@ def test_json_properties_get_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_get_namespace_specific_property():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_get_namespace_specific_property(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "get", "default", "location"])
     assert result.exit_code == 0
@@ -729,8 +732,8 @@ def test_json_properties_get_namespace_specific_property():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_set_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_set_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "set", "namespace", "default", "location", "s3://new_location"])
     assert result.exit_code == 0
@@ -738,8 +741,8 @@ def test_json_properties_set_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_set_namespace_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_set_namespace_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(
         run, ["--output=json", "properties", "set", "namespace", "doesnotexist", "location", "s3://new_location"]
@@ -749,8 +752,8 @@ def test_json_properties_set_namespace_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_set_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_set_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "set", "table", "default.foo", "location", "s3://new_location"])
     assert result.exit_code == 1
@@ -758,8 +761,8 @@ def test_json_properties_set_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_set_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_set_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(
         run, ["--output=json", "properties", "set", "table", "default.doesnotexist", "location", "s3://new_location"]
@@ -769,8 +772,8 @@ def test_json_properties_set_table_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_remove_namespace():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_remove_namespace(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "remove", "namespace", "default", "location"])
     assert result.exit_code == 0
@@ -778,8 +781,8 @@ def test_json_properties_remove_namespace():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_remove_namespace_that_doesnt_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_remove_namespace_that_doesnt_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "remove", "namespace", "doesnotexist", "location"])
     assert result.exit_code == 1
@@ -787,8 +790,8 @@ def test_json_properties_remove_namespace_that_doesnt_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_remove_table():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_remove_table(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.foo", "read.split.target.size"])
     assert result.exit_code == 1
@@ -796,8 +799,8 @@ def test_json_properties_remove_table():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_remove_table_property_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_remove_table_property_does_not_exists(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.foo", "doesnotexist"])
     assert result.exit_code == 1
@@ -808,8 +811,8 @@ def test_json_properties_remove_table_property_does_not_exist():
 
 
 @mock.patch.dict(os.environ, MOCK_ENVIRONMENT)
-@mock.patch("pyiceberg.cli.console.SUPPORTED_CATALOGS", MOCK_CATALOGS)
-def test_json_properties_remove_table_does_not_exist():
+@mock.patch("pyiceberg.cli.console.load_catalog", return_value=MOCK_CATALOG)
+def test_json_properties_remove_table_does_not_exist(_):
     runner = CliRunner()
     result = runner.invoke(run, ["--output=json", "properties", "remove", "table", "default.doesnotexist", "location"])
     print(result)
diff --git a/python/tests/conftest.py b/python/tests/conftest.py
index 089443cead..b4d2e005fe 100644
--- a/python/tests/conftest.py
+++ b/python/tests/conftest.py
@@ -289,7 +289,7 @@ def example_table_metadata_v2() -> Dict[str, Any]:
 
 @pytest.fixture
 def catalog() -> InMemoryCatalog:
-    return InMemoryCatalog("test.in.memory.catalog", {"test.key": "test.value"})
+    return InMemoryCatalog("test.in.memory.catalog", **{"test.key": "test.value"})
 
 
 manifest_entry_records = [
diff --git a/python/tests/utils/test_config.py b/python/tests/utils/test_config.py
new file mode 100644
index 0000000000..e4a8970876
--- /dev/null
+++ b/python/tests/utils/test_config.py
@@ -0,0 +1,54 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+import os
+from unittest import mock
+
+import yaml
+
+from pyiceberg.utils.config import Config, _lowercase_dictionary_keys
+
+EXAMPLE_ENV = {"PYICEBERG_CATALOG__PRODUCTION__URI": "https://service.io/api"}
+
+
+def test_config():
+    """To check if all the file lookups go well without any mocking"""
+    assert Config()
+
+
+@mock.patch.dict(os.environ, EXAMPLE_ENV)
+def test_from_environment_variables():
+    assert Config().get_catalog_config("production") == {"uri": "https://service.io/api"}
+
+
+@mock.patch.dict(os.environ, EXAMPLE_ENV)
+def test_from_environment_variables_uppercase():
+    assert Config().get_catalog_config("PRODUCTION") == {"uri": "https://service.io/api"}
+
+
+def test_from_configuration_files(tmp_path_factory):
+    config_path = str(tmp_path_factory.mktemp("config"))
+    with open(f"{config_path}/.pyiceberg.yaml", "w", encoding="utf-8") as file:
+        yaml.dump({"catalog": {"production": {"uri": "https://service.io/api"}}}, file)
+
+    os.environ["PYICEBERG_HOME"] = config_path
+    assert Config().get_catalog_config("production") == {"uri": "https://service.io/api"}
+
+
+def test_lowercase_dictionary_keys():
+    uppercase_keys = {"UPPER": {"NESTED_UPPER": {"YES"}}}
+    expected = {"upper": {"nested_upper": {"YES"}}}
+    assert _lowercase_dictionary_keys(uppercase_keys) == expected