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/08/10 11:26:20 UTC

[GitHub] [iceberg] Fokko opened a new pull request, #5488: Python: Add configuration

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

   This PR will add the option to read from a configuration file and environment variables. This can be used to read the catalog configuration from a file instead of having to pass it through the CLI or Python.
   
   Looked at different Python projects:
   
   - https://docs.dask.org/en/stable/configuration.html
   - https://nvidia.github.io/spark-rapids/docs/configs.html
   - https://pandas.pydata.org/pandas-docs/stable/user_guide/options.html
   - https://docs.python.org/3/library/configparser.html
   
   Most of them have (a variation) of dot notation config.
   
   Python itself comes with a `configparser`: https://docs.python.org/3/library/configparser.html
   
   But this one uses sections, that isn't compatible with the dot notation config `pyiceberg.catalog.uri=thrift://`. Also, the Java implementation uses dotted config, so that's also nice.
   
   Python ships out of the box with a yaml parser. The current implementation looks like this:
   
   ```yaml
   
   catalog:
      production:
         uri: thrift://prod:9083
      rest-dev:
         uri: http://server.io
         credential: sometoken
   ```
   
   You can also override the config using environment variables:
   ```sh
   PYICEBERG__CATALOG_PRODUCTION_URI=thrift://dev:9083
   ```


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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:

Review Comment:
   Yes, the documentation was misleading. Thanks for pointing out the null values. I didn't really consider that because that would require anyone to explicitly set a `None` value in yaml:
   ```yaml
   catalog:
      prod:
        uri: null
   ```
   I've changed this, and also returned a new 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] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")

Review Comment:
   Airflow is slightly different because they rely on sections, but I agree 👍🏻 



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:

Review Comment:
   Alright, I agree. I'm also fine with keeping it even simpler, omitting the `$PYICEBERG_HOME` for now and just only checking the home directory. This way we can postpone this decision.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,139 @@
+# 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]:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            return file_config_lowercase
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        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 element not in _config:
+                    _config[element] = {}
+                if len(path) == 0:

Review Comment:
   Minor: if last element is missing it will set the config to `{}` before setting it to `config_value`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}

Review Comment:
   Can we create this once and make it a constant?



##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()

Review Comment:
   Should this be all caps since it is a constant?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:
+            configuration = Config().get_catalog_config(name)

Review Comment:
   I took at look at the method, it's looking really good to me. Don't worry about `py-impl` -- I think the current approach of using the URI to decide is better. We can iterate from there.
   
   For `FileIO`, you may need to change the IO based on the table you're using. A lot of people use the `HadoopFileIO` in Java, which supports many different storage schemes. But other use cases may need to choose between S3 and GCS per table. It's the responsibility of the catalog to pass a reasonable `FileIO` instance in when creating the table in Java. That's commonly the catalog's `FileIO`.
   
   For Python, we can do things a little differently. I think it makes sense to combine information from the table and the environment to come up with the right `FileIO`. The table has storage and a policy for where to create new data files (`LocationProvider` in Java). I think that boils down to a storage scheme, like `s3`. The environment then determines what can be used to handle that scheme. In Python, pyarrow could be used, by there's also a S3 file system from fsspec.
   
   In the short term, I think we should detect which file systems are available and use one that can handle the scheme for a table, where the scheme is taken from the table's location. Instantiating just the `PyArrowFileIO` works for now though.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")
+
+    if name:
+        env = _env_config.get_catalog_config(name)
+        conf = merge_config(env or {}, properties)

Review Comment:
   I merged the `uri` into the `properties`



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -210,8 +199,8 @@ def headers(self) -> Properties:
             "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"):

Review Comment:
   I'd rather not change the signature to `Optional[str]` because then we have to do yet another `None` check 🤔 



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/typedef.py:
##########
@@ -30,3 +30,4 @@ def update(self, *args: Any, **kwargs: Any) -> None:  # type: ignore
 
 Identifier = Tuple[str, ...]
 Properties = Dict[str, str]
+Configuration = Dict[str, "Configuration"]  # type: ignore

Review Comment:
   Still looking for someone to implement mypy support for this: https://github.com/python/mypy/issues/731



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value

Review Comment:
   I don't think that we need to loop over both left and right contents. If you're updating left using right, then you can do everything in this loop: if the key already exists, overwrite or merge, otherwise insert.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:

Review Comment:
   `merge_config`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")
+                set_property(config, parts, config_value)
+
+        return config
+
+    def get_catalog_config(self, catalog_name: str) -> RecursiveDict:
+        if CATALOG in self.config:
+            catalog_name_lower = catalog_name.lower()
+            catalogs = self.config[CATALOG]
+            assert isinstance(catalogs, dict), "Configuration path catalogs needs to be an object"

Review Comment:
   When is it appropriate to use assert vs throwing an exception?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")
+
+    if name:
+        env = _env_config.get_catalog_config(name)
+        conf = merge_config(env or {}, properties)

Review Comment:
   Since `uri` was passed separately, I think we need to add URI to `conf` or else it won't be passed.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   In the first instance, I would a name to be mandatory. I changed this because of the configuration. If you don't have a name, it won't load the config environment variable `PYICEBERG_CATALOG__DEFAULT__URI`, and fall back to `PYICEBERG_URI` in the CLI. I can imagine that we want to change this, but I'm not fully settled on this one either.
   
   What I don't really like is that the `PYICEBERG_URI` and `PYICEBERG_CREDENTIAL` are REST specific, and I wouldn't like to open a door to having another way of setting config for other catalog implementations in the future (for Glue PYICEBERG_AWS_ACCOUNT_ID for example).



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:

Review Comment:
   Can we pass `uri` as a property instead of explicitly? Not all catalogs use a URI. Dynamo, for example, just uses a table 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:

Review Comment:
   Sounds great!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:

Review Comment:
   I like it 👍🏻 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,139 @@
+# 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]:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            return file_config_lowercase
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        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 element not in _config:
+                    _config[element] = {}
+                if len(path) == 0:

Review Comment:
   Nice one, thanks!



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration
+
+    def __init__(self):
+        config = self._from_configuration_files({})
+        config = merge_dicts(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files(config: Configuration) -> Configuration:
+        """Traverses directories searching for configuration files
+
+        Takes preference over configuration that's close to the user
+        """
+
+        for path in PATHS:
+            if os.path.isfile(path):
+                with open(path, encoding="utf-8") as f:
+                    try:
+                        file_config = yaml.safe_load(f)
+                        file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                        config = merge_dicts(config, file_config_lowercase)
+                    except yaml.YAMLError:
+                        logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @staticmethod
+    def _from_environment_variables(config: Configuration) -> Configuration:

Review Comment:
   I think this is too complicated. If we want to support environment variables, we should make them independent and configure only a single catalog. I don't think there's a use case for configuring multiple catalogs through a ton of environment variables.
   
   I think we should carefully define what environment variables do and have a static set of them. I like `PYICEBERG_URI` and `PYICEBERG_CREDENTIAL`. I think those should set up a default catalog. If you don't specify a catalog, those are used. If you do specify a catalog, we should ignore them. That way we don't need to worry about conflicts, merging, or all this complexity of parsing environment variable names.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration
+
+    def __init__(self):
+        config = self._from_configuration_files({})
+        config = merge_dicts(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files(config: Configuration) -> Configuration:
+        """Traverses directories searching for configuration files
+
+        Takes preference over configuration that's close to the user
+        """
+
+        for path in PATHS:
+            if os.path.isfile(path):
+                with open(path, encoding="utf-8") as f:
+                    try:
+                        file_config = yaml.safe_load(f)
+                        file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                        config = merge_dicts(config, file_config_lowercase)
+                    except yaml.YAMLError:
+                        logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @staticmethod
+    def _from_environment_variables(config: Configuration) -> Configuration:

Review Comment:
   I do like this a lot. I got this idea from Dask as well: https://docs.dask.org/en/stable/configuration.html#environment-variables. But also tool like Airflow rely heavily on this: https://airflow.apache.org/docs/apache-airflow/stable/howto/set-config.html This is not only for configuring Airflow itself, but also the connections: https://airflow.apache.org/docs/apache-airflow/stable/howto/connection.html#storing-connections-in-environment-variables
   
   When the `pyiceberg` CLI gets used in orchestrators such as Airflow or even as a k8s cron, environment variables are often a convenient and powerful way of setting the credentials. 
   
   > I think those should set up a default catalog. If you don't specify a catalog, those are used. If you do specify a catalog, we should ignore them
   
   I agree with that, I'll take that into consideration in the CLI PR.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")
+                set_property(config, parts, config_value)
+
+        return config
+
+    def get_catalog_config(self, catalog_name: str) -> RecursiveDict:
+        if CATALOG in self.config:
+            catalog_name_lower = catalog_name.lower()
+            catalogs = self.config[CATALOG]
+            assert isinstance(catalogs, dict), "Configuration path catalogs needs to be an object"

Review Comment:
   Yes, good call. Assertions are usually for developers, they can also be disabled when using a flag on Python `-O`. Exceptions are geared towards users. I prefer to use assertions within tests, and exceptions in user code. I changed this to an exception.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -210,8 +199,8 @@ def headers(self) -> Properties:
             "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"):

Review Comment:
   Would it be better to allow passing a default? That way we don't need to bypass the `property` method for cases where it is optional.



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

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

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


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


[GitHub] [iceberg] Fokko commented on pull request #5488: Python: Add configuration

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

   @rdblue Eventual consistent Github. It is the commit just above your comment: https://github.com/apache/iceberg/pull/5488/commits/627e94bd844e7e9a3b4ff59c5906bbc747919dde


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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:

Review Comment:
   Yes, the documentation was misleading. Thanks for pointing out the null values. I didn't really consider that because that would require anyone to explicitly set an empty value in yaml:
   ```yaml
   catalogs:
      prod:
        uri: null
   ```
   I've changed this, and also returned a new dict.



##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value

Review Comment:
   You are right, I've removed it.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration

Review Comment:
   I had the same, but was still thinking of something better :)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):

Review Comment:
   👍🏻  As soon as Python starts supporting tail recursion, I'm going to change this back :)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")

Review Comment:
   Both Dask and Airflow separate variable parts with a double underscore and convert a single underscore to dash. I think we should follow the same convention if we're mimicing that feature.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")

Review Comment:
   I don't think we should warn if something isn't installed. The right place to do this is when a `thrift://` URI is passed in. If we see one and Hive isn't installed, we can have a nice 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   URI isn't specific to REST. JDBC and Hive also use URI. Credential is REST-specific though.
   
   For the environment variables, we can always remove them and add them back when we find a good way to manage them. We just need to deconflict the existing ones (`PYICEBERG_URI`, etc.) and the new-style ones `PYICEBERG__CATALOG__NAME__URI` right?
   
   I think we have two options:
   1. `PYICEBERG_URI` configures an anonymous catalog with a random name that can only be used if you don't specify `--catalog`
   2. Remove `PYICEBERG_URI` and only support `PYICEBERG__*` properties
   
   I don't really mind either way. If we're going to support the `PYICEBERG__*` properties in general then it probably makes sense to get rid of the existing ones.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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

   This looks good to me other than needing to fix the config file loading. Thanks, @Fokko!


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

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

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


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


[GitHub] [iceberg] Fokko commented on pull request #5488: Python: Add configuration

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

   @rdblue Changed the config file loading. Also pushed two more improvements:
   - Refactored the lazy loading of the catalogs a bit. In the future when we have more catalogs, we want to be mindful to not load everything. So generalized this a bit which makes it more extensible.
   - I'veintroduce a `type` property in the config to explicitly load a catalog. This is the same as in Java `spark.sql.catalog.hive_prod.type = hive`. For the CLI we still infer this from the properties. It tries to determine the type as we did before.


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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -210,8 +199,8 @@ def headers(self) -> Properties:
             "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"):

Review Comment:
   I've added a docstring to the property function:
   ```python
   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]
   ```
   
   A token isn't required, for example, when authentication is disabled on the rest catalog.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]

Review Comment:
   Fair point, I took this from the Dask docs that do it like this: https://docs.dask.org/en/stable/configuration.html
   
   I've reduced this to the files that you've 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")
+
+    if name:
+        env = _env_config.get_catalog_config(name)
+        conf = merge_config(env or {}, properties)
+    else:
+        conf = properties
+
+    catalog: Catalog | None = None
+    for scheme, catalog_type in supported_catalogs.items():
+        if uri.startswith(scheme):
+            catalog = catalog_type(name, **conf)

Review Comment:
   This looks good to me.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:
+            configuration = Config().get_catalog_config(name)

Review Comment:
   Great suggestion, just implemented that. I left the `py-impl` out of scope; this PR is already getting far too big. 
   
   While we're at it. One question, do we keep the `FileIO` at a catalog level? You could also specify it on a table level `create_table(..., properties={'py-impl': 'pyarrow'}` right? I was browsing the Java codebase, and it looks that's it is at a catalog level (which would make things easier).



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/rest.py:
##########
@@ -210,8 +199,8 @@ def headers(self) -> Properties:
             "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"):

Review Comment:
   Should this be `self.property("token")`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")
+
+    if name:

Review Comment:
   I already changed this back, but this check was still in place. I think mypy should also have caught this since the else branch is unreachable.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}

Review Comment:
   Makes sense!



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]

Review Comment:
   What is the rationale for having so many patterns here? Can we keep it simple? I would start with `$HOME/.pyiceberg`. Maybe then add a `$PYICEBERG_HOME/.pyiceberg` to control it, but I don't see the need for `prefix` or `sys.prefix`.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"

Review Comment:
   According to Wiki it is both:
   ![image](https://user-images.githubusercontent.com/1134248/184511299-434265a3-f0de-4a18-bbf5-b39b1d7f5053.png)
   
   Then again, the yaml faq recommends `.yaml`: https://yaml.org/faq.html
   
   Interestingly enough, `dbt` uses `.yml` for their profiles https://docs.getdbt.com/dbt-cli/configure-your-profile
   
   Let me change it to `.yaml`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")
+                set_property(config, parts, config_value)
+
+        return config
+
+    def get_catalog_config(self, catalog_name: str) -> RecursiveDict:
+        if CATALOG in self.config:
+            catalog_name_lower = catalog_name.lower()
+            catalogs = self.config[CATALOG]
+            assert isinstance(catalogs, dict), "Configuration path catalogs needs to be an object"
+            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 EMPTY_DICT

Review Comment:
   Yes, good one. I've changed this by returning a `None`. I like to just return an empty `dict`, so we can merge that dict into other ones, without having to do a lot of `None` 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:

Review Comment:
   Looks like this is the only thing left to do before we can merge 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/typedef.py:
##########
@@ -30,3 +30,4 @@ def update(self, *args: Any, **kwargs: Any) -> None:  # type: ignore
 
 Identifier = Tuple[str, ...]
 Properties = Dict[str, str]
+Configuration = Dict[str, "Configuration"]  # type: ignore

Review Comment:
   Should this give an option for strings as leaf nodes?
   
   ```python
   Configuration = Dict[str, str | "Configuration"]
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"

Review Comment:
   I thought the extension for YAML was [`.yaml`](https://yaml.org/faq.html).



##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"

Review Comment:
   I thought the extension for YAML was [`.yaml`](https://yaml.org/faq.html)?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):
+            key = path[0]
+            if len(path) == 1:
+                _config[key] = config_value
+            else:
+                if key not in _config:
+                    _config[key] = {}
+                set_property(_config[key], path[1:], config_value)  # type: ignore
+
+        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("_")
+                set_property(config, parts, config_value)
+
+        return config
+
+    def get_catalog_config(self, catalog_name: str) -> RecursiveDict:
+        if CATALOG in self.config:
+            catalog_name_lower = catalog_name.lower()
+            catalogs = self.config[CATALOG]
+            assert isinstance(catalogs, dict), "Configuration path catalogs needs to be an object"
+            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 EMPTY_DICT

Review Comment:
   What about returning None? Otherwise how would you know that the catalog was not defined?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):

Review Comment:
   What about avoiding recursion here?
   
   ```python
       name = path[-1]
       current = _config
       for key in path[0:-1]:
           value = current.get(key)
           if isinstance(value, dict):
               current = value
           else:
               raise ValueError(f"Not a dict: {value}") # probably needs a better error message
       current[name] = config_value
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   Should name be required instead? When does it make sense to create a catalog with no 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")
+
+    if name:

Review Comment:
   I think name should always be required. If we are creating some default catalog, we can use a default name (or randomly generate one).



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:

Review Comment:
   Of course, updated! Keep in mind that we determine the catalog based on the `uri`, so for dynamo you should set the URI to Dynamo. We could also pass the properties to a static method on the Catalog, which then returns a boolean if the properties match 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] rdblue commented on pull request #5488: Python: Add configuration

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

   > Changed the config file loading
   
   I'm not seeing a change. Did you miss a 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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   `name` is required, so I don't think we need to worry about cases where it is `None`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration
+
+    def __init__(self):
+        config = self._from_configuration_files({})
+        config = merge_dicts(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files(config: Configuration) -> Configuration:
+        """Traverses directories searching for configuration files
+
+        Takes preference over configuration that's close to the user
+        """
+
+        for path in PATHS:

Review Comment:
   Why merge config rather than using the first config file? I think it's more normal (and easier to manage) to use a single config file. I would use `$PYICEBERG_HOME/.pyiceberg` (or maybe .conf or .yaml?) where `PYICEBERG_HOME` defaults to `$HOME`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:

Review Comment:
   Did you intend to compare left with right? This compares left with itself and will never trigger.
   
   Also, the doc says this merges the left into the right, but this is actually merging the right into the left. The left is modified and the right takes precedence. I generally prefer to create new dicts rather than modifying incoming values.
   
   We should also decide on precedence. There are existing patterns for both left and right precedence. If we think of this like a "coalesce" operation that takes the first non-null, then the left value should be used. If we think of it like "dict.update" then the right value should be used. I assumed the first, but it looks like you're intending to use the second. Let's just make sure it is clear which one we are going with.



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

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

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration

Review Comment:
   Using both `Configuration` and `Config` is a bit confusing. What about using a `RecursiveDict` instead of `Configuration`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/typedef.py:
##########
@@ -30,3 +30,4 @@ def update(self, *args: Any, **kwargs: Any) -> None:  # type: ignore
 
 Identifier = Tuple[str, ...]
 Properties = Dict[str, str]
+Configuration = Dict[str, "Configuration"]  # type: ignore

Review Comment:
   Ah, great catch! 👍🏻 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:

Review Comment:
   Yes, the documentation was misleading. Thanks for pointing out the null values. I didn't really consider that because that would require anyone to explicitly set an empty value in yaml:
   ```yaml
   catalogs:
      prod:
        uri: null
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:

Review Comment:
   I don't think there should be multiple config files that are merged. There isn't much value in doing it and it will cause a lot of confusion when behavior doesn't match the config file that the user is looking at. I think the behavior should be to check whether `PYICEBERG_HOME` is defined. If it is, then use `$PYICEBERG_HOME/pyiceberg.yaml`. If not, use `~/pyiceberg.yaml`:
   
   ```pyhton'
       directory = os.environ.get(PYICEBERG_HOME, '~')
       if directory:
           path = f"{directory}/{PYICEBERG_YML}"
           ...
   ```
   
   I think that's how most config files behave. Introducing multiple files brings in problems with precedence and knowing what the final config actually was.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   GlueCatalog wouldn't have a URI, but let's not overcomplicate things. I made it mandatory.
   
   I also removed `PYICEBERG_URI` I think it is better and less confusing in the long run to have on the way of setting the environment variables. I also refactored the properties, mostly by changing all the `__init__` methods of the catalogs:
   ```python
       def __init__(
           self,
           name: str,
           **properties: str,
       ):
   ```
   This way we don't have any specifics in the Catalog itself.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]

Review Comment:
   Fair point, I took this from the Dask docs that do it like this: https://docs.dask.org/en/stable/configuration.html



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,125 @@
+# 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
+import site
+import sys
+from typing import Any, List
+
+import yaml
+
+from pyiceberg.typedef import EMPTY_DICT, Configuration, FrozenDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+
+PATHS = [
+    f"{os.environ.get('ICEBERG_ROOT_CONFIG', '/etc/pyiceberg').rstrip('/')}/pyiceberg",
+    *[f"{prefix}/etc/pyiceberg" for prefix in site.PREFIXES],
+    f"{sys.prefix}/etc/pyiceberg",
+    "~/config/pyiceberg",
+]
+
+
+def merge_dicts(lhs: Configuration, rhs: Configuration) -> Configuration:
+    """merges left-hand side into the right-hand side"""
+    for lhs_key, lhs_value in lhs.items():
+        if lhs_key in rhs:
+            rhs_value = rhs[lhs_key]
+            if isinstance(lhs_value, dict) and isinstance(rhs_value, dict):
+                lhs[lhs_key] = merge_dicts(lhs_value, rhs_value)
+            elif lhs_value != lhs_value:
+                lhs[lhs_key] = rhs_value
+
+    # Add the keys from the rhs that are not in the lhs
+    for rhs_key, rhs_value in rhs.items():
+        if rhs_key not in lhs:
+            lhs[rhs_key] = rhs_value
+
+    return lhs
+
+
+logger = logging.getLogger(__name__)
+
+
+def _lowercase_dictionary_keys(input_dict: Configuration) -> Configuration:
+    """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: Configuration
+
+    def __init__(self):
+        config = self._from_configuration_files({})
+        config = merge_dicts(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files(config: Configuration) -> Configuration:
+        """Traverses directories searching for configuration files
+
+        Takes preference over configuration that's close to the user
+        """
+
+        for path in PATHS:
+            if os.path.isfile(path):
+                with open(path, encoding="utf-8") as f:
+                    try:
+                        file_config = yaml.safe_load(f)
+                        file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                        config = merge_dicts(config, file_config_lowercase)
+                    except yaml.YAMLError:
+                        logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @staticmethod
+    def _from_environment_variables(config: Configuration) -> Configuration:

Review Comment:
   If Dask and Airflow both have this convention then there's sufficient precedent. I'd probably add it later, but it should be fine.



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:
+            if directory:
+                path = f"{directory.rstrip('/')}/{PYICEBERG_YML}"
+                if os.path.isfile(path):
+                    with open(path, encoding="utf-8") as f:
+                        try:
+                            file_config = yaml.safe_load(f)
+                            file_config_lowercase = _lowercase_dictionary_keys(file_config)
+                            config = _merge_config(config, file_config_lowercase)
+                        except yaml.YAMLError:
+                            logger.exception("Not valid config yaml: %s", path)
+        return config
+
+    @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):

Review Comment:
   ![image](https://user-images.githubusercontent.com/1134248/184511472-6d482265-5eae-4ca3-862d-85606257a8f5.png)
   
   As soon as Python starts supporting tail recursion, I'm going to change this back :)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"

Review Comment:
   According to Wiki it is both:
   ![image](https://user-images.githubusercontent.com/1134248/184511299-434265a3-f0de-4a18-bbf5-b39b1d7f5053.png)
   
   But the yaml faq recommends `.yaml`: https://yaml.org/faq.html
   
   Interestingly enough, `dbt` uses `.yml` for their profiles https://docs.getdbt.com/dbt-cli/configure-your-profile
   
   Let me change it to `.yaml`



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:
+            configuration = Config().get_catalog_config(name)

Review Comment:
   I'd prefer not to load configuration from files every time a catalog is created. It also seems a bit strange to me to make this depend directly on config rather than passing it in.
   
   On the other hand, I can see why it would be great to be able to rely on the environment config always being used so that callers don't need to instantiate their own `Config` and merge properties themselves.
   
   How about using a factory method like this?
   
   ```python
   _env_config = Config()
   
   def load_catalog(name: str, **properties: str):
       env = _env_config.get_catalog_config(name)
       conf = merge_config(env or {}, properties)
       impl = conf['py-impl'] # not sure how we're doing this yet
       return impl(name, conf)
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}
+
+    try:
+        # In case Thrift isn't installed
+        from pyiceberg.catalog.hive import HiveCatalog
+
+        supported_catalogs["thrift"] = HiveCatalog
+    except ImportError:
+        logger.warning("Apache Hive not supported, to enable: pip install 'pyiceberg[hive]'")

Review Comment:
   Updated 👍🏻 



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

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

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


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


[GitHub] [iceberg] Fokko commented on a diff in pull request #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -25,6 +26,41 @@
 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.utils.config import Config, merge_config
+
+logger = logging.getLogger(__name__)
+
+_env_config = Config()
+
+
+def load_catalog(name: str, uri: str, **properties: str | None) -> Catalog:
+    from pyiceberg.catalog.rest import RestCatalog
+
+    supported_catalogs: dict[str, type[Catalog]] = {"http": RestCatalog}

Review Comment:
   Unfortunately not, because that would introduce a circular import, therefore we have to import it in the method itself.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/catalog/__init__.py:
##########
@@ -53,7 +54,15 @@ class Catalog(ABC):
 
     def __init__(self, name: str | None, **properties: str):
         self.name = name
-        self.properties = properties
+
+        if name is not None:

Review Comment:
   In the first instance, I would a name to be mandatory. I changed this because of the configuration. If you don't have a name, it won't load the config environment variable `PYICEBERG_CATALOG__DEFAULT__URI`, and fall back to `PYICEBERG_URI` in the CLI. I can imagine that we want to change this, but I'm not fully settled on this one either.
   
   What I don't really like is that the `PYICEBERG_URI` and `PYICEBERG_CREDENTIAL` are REST specific, and I wouldn't like to open a door to have another way of setting config for other catalog implementation in the future (for Glue PYICEBERG_ACCOUNT_ID for example).



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


##########
python/pyiceberg/utils/config.py:
##########
@@ -0,0 +1,130 @@
+# 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 EMPTY_DICT, FrozenDict, RecursiveDict
+
+PYICEBERG = "pyiceberg__"
+CATALOG = "catalog"
+HOME = "HOME"
+PYICEBERG_HOME = "PYICEBERG_HOME"
+PYICEBERG_YML = ".pyiceberg.yml"
+
+
+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
+
+
+logger = logging.getLogger(__name__)
+
+
+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()
+        config = _merge_config(config, self._from_environment_variables(config))
+        self.config = FrozenDict(**config)
+
+    @staticmethod
+    def _from_configuration_files() -> RecursiveDict:
+        """Looks up configuration files
+
+        Takes preference over RecursiveDict that's close to the user
+        """
+        config: RecursiveDict = {}
+        for directory in [os.environ.get(HOME), os.environ.get(PYICEBERG_HOME)]:

Review Comment:
   I've updated this to raise the exception if there is a yaml error and removed the loop:
   ```python
   @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
       # 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
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #5488: Python: Add configuration

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


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

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

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


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