You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "ImpressionableRaccoon (via GitHub)" <gi...@apache.org> on 2023/12/27 10:43:51 UTC

[PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

ImpressionableRaccoon opened a new pull request, #36449:
URL: https://github.com/apache/airflow/pull/36449

   <!--
    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.
    -->
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   This PR added support for Yandex Cloud Lockbox secrets backend to Yandex Cloud provider 😎
   
   <!-- Please keep an empty line above the dashes. -->
   ---
   **^ Add meaningful description above**
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "ImpressionableRaccoon (via GitHub)" <gi...@apache.org>.
ImpressionableRaccoon commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1457253372


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,

Review Comment:
   wow 😱



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "boring-cyborg[bot] (via GitHub)" <gi...@apache.org>.
boring-cyborg[bot] commented on PR #36449:
URL: https://github.com/apache/airflow/pull/36449#issuecomment-1870181422

   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (ruff, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, it's a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   - Always keep your Pull Requests rebased, otherwise your build might fail due to changes not related to your commits.
   Apache Airflow is a community-driven project and together we are making it better πŸš€.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1436945310


##########
airflow/providers/yandex/CHANGELOG.rst:
##########
@@ -27,6 +27,15 @@
 Changelog
 ---------
 
+3.8.0
+.....
+
+Features
+~~~~~~~~
+
+* ``add Yandex Lockbox secrets backend``
+* ``add support for metadata service for Yandex Cloud connections``
+

Review Comment:
   You don't have to add this lines here, this would be autogenerated during new release.



##########
airflow/providers/yandex/provider.yaml:
##########
@@ -21,8 +21,9 @@ name: Yandex
 description: |
     Yandex including `Yandex.Cloud <https://cloud.yandex.com/>`__
 suspended: false
-source-date-epoch: 1703288182
+source-date-epoch: 1703673062
 versions:
+  - 3.8.0

Review Comment:
   This would be updated by a release manager



##########
airflow/providers/yandex/provider.yaml:
##########
@@ -21,8 +21,9 @@ name: Yandex
 description: |
     Yandex including `Yandex.Cloud <https://cloud.yandex.com/>`__
 suspended: false
-source-date-epoch: 1703288182
+source-date-epoch: 1703673062

Review Comment:
   Same here, it would be updated by release manager



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Piatachock (via GitHub)" <gi...@apache.org>.
Piatachock commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1452392318


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,257 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = yc_connection_id or default_conn_name
+        self._use_connection = not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path])
+
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: Connection ID
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if self._use_connection:
+            if conn_id == self.yc_connection_id:
+                return None
+
+        return self._get_secret_value(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> str | None:
+        """
+        Return value for Airflow Variable.
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        if self.variables_prefix is None:
+            return None
+
+        return self._get_secret_value(self.variables_prefix, key)
+
+    def get_config(self, key: str) -> str | None:
+        """
+        Return value for Airflow Config Key.
+
+        :param key: Config Key
+        :return: Config Value
+        """
+        if self.config_prefix is None:
+            return None
+
+        return self._get_secret_value(self.config_prefix, key)
+
+    @property

Review Comment:
   we should cache SDK, TLS channel creation on each secret request is costly



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,257 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = yc_connection_id or default_conn_name
+        self._use_connection = not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path])
+
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: Connection ID
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if self._use_connection:
+            if conn_id == self.yc_connection_id:
+                return None
+
+        return self._get_secret_value(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> str | None:
+        """
+        Return value for Airflow Variable.
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        if self.variables_prefix is None:
+            return None
+
+        return self._get_secret_value(self.variables_prefix, key)
+
+    def get_config(self, key: str) -> str | None:
+        """
+        Return value for Airflow Config Key.
+
+        :param key: Config Key
+        :return: Config Value
+        """
+        if self.config_prefix is None:
+            return None
+
+        return self._get_secret_value(self.config_prefix, key)
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        if self._use_connection:
+            self.yc_oauth_token = self._get_field("oauth")
+            self.yc_sa_key_json = self._get_field("service_account_json")
+            self.yc_sa_key_json_path = self._get_field("service_account_json_path")
+            self.folder_id = self.folder_id or self._get_field("folder_id")
+
+        credentials = get_credentials(
+            oauth_token=self.yc_oauth_token,
+            service_account_json=self.yc_sa_key_json,
+            service_account_json_path=self.yc_sa_key_json_path,
+        )
+        return yandexcloud.SDK(**credentials).client

Review Comment:
   Missed `endpoint` and `user_agent` parameters. IMO:
   * `endpoint` should be part of `SecretsBackend` interface
   * `user_agent` could be too, or could be just a static string like in Hook. Not so important



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,257 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = yc_connection_id or default_conn_name
+        self._use_connection = not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path])
+
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: Connection ID
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if self._use_connection:

Review Comment:
   nit: excessive check IMO. `if conn_id == self.yc_connection_id:` alone should work just fine when `self.yc_connection_id is None`. Just dont assign `default_conn_name` to `self.yc_connection_id` if it is not meant to be used.



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,257 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = yc_connection_id or default_conn_name

Review Comment:
   nit: 
   ```
   if not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path]):
       self.yc_connection_id = yc_connection_id or default_conn_name
   else:
       assert yc_connection_id is None, "..."
   ```
   
   This way you won't need `self._use_connection`



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,257 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = yc_connection_id or default_conn_name
+        self._use_connection = not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path])
+
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: Connection ID
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if self._use_connection:
+            if conn_id == self.yc_connection_id:
+                return None
+
+        return self._get_secret_value(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> str | None:
+        """
+        Return value for Airflow Variable.
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        if self.variables_prefix is None:
+            return None
+
+        return self._get_secret_value(self.variables_prefix, key)
+
+    def get_config(self, key: str) -> str | None:
+        """
+        Return value for Airflow Config Key.
+
+        :param key: Config Key
+        :return: Config Value
+        """
+        if self.config_prefix is None:
+            return None
+
+        return self._get_secret_value(self.config_prefix, key)
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        if self._use_connection:

Review Comment:
   Reason for this part if initialization to happen here and not in __init__ is confusing. Is Connection object not availabe when `LockboxSecretManager.__init__` is called? If that's the case, I'd add explicit comment to clarify intent of this lazy initialization.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Piatachock (via GitHub)" <gi...@apache.org>.
Piatachock commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1457025440


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   My thoughts here is:
   1. This problem is addressed in this PR
   2. I guess Deployment Manager can just choose to not use Connection in SecretManager configuration if that connection metadata is not trustworthy enough.
   
   On the other hand, ability to not duplicate creds between SecretsManager and Hook looks profitable for simple installations.



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   My thoughts here are:
   1. This problem is addressed in this PR
   2. I guess Deployment Manager can just choose to not use Connection in SecretManager configuration if that connection metadata is not trustworthy enough.
   
   On the other hand, ability to not duplicate creds between SecretsManager and Hook looks profitable for simple installations.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1456058203


##########
airflow/providers/yandex/utils/user_agent.py:
##########
@@ -0,0 +1,43 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import warnings
+
+from airflow.providers.yandex.utils.defaults import conn_type, hook_name
+
+
+def provider_user_agent() -> str | None:
+    """Construct User-Agent from Airflow core & provider package versions."""
+    from airflow import __version__ as airflow_version
+    from airflow.configuration import conf
+    from airflow.providers_manager import ProvidersManager
+
+    try:
+        manager = ProvidersManager()
+        provider_name = manager.hooks[conn_type].package_name  # type: ignore[union-attr]
+        provider = manager.providers[provider_name]
+        return " ".join(
+            (
+                conf.get("yandex", "sdk_user_agent_prefix", fallback=""),
+                f"apache-airflow/{airflow_version}",
+                f"{provider_name}/{provider.version}",
+            )
+        ).strip()
+    except KeyError:
+        warnings.warn(f"Hook '{hook_name}' info is not initialized in airflow.ProviderManager")

Review Comment:
   ```suggestion
           warnings.warn(
               f"Hook '{hook_name}' info is not initialized in airflow.ProviderManager",
               UserWarning,
               stacklevel=2,
           )
   ```
   
   See: https://github.com/apache/airflow/pull/36831



##########
airflow/providers/yandex/hooks/yandex.py:
##########
@@ -85,28 +91,6 @@ def get_connection_form_widgets(cls) -> dict[str, Any]:
             ),
         }
 
-    @classmethod
-    def provider_user_agent(cls) -> str | None:
-        """Construct User-Agent from Airflow core & provider package versions."""
-        from airflow import __version__ as airflow_version
-        from airflow.configuration import conf
-        from airflow.providers_manager import ProvidersManager
-
-        try:
-            manager = ProvidersManager()
-            provider_name = manager.hooks[cls.conn_type].package_name  # type: ignore[union-attr]
-            provider = manager.providers[provider_name]
-            return " ".join(
-                (
-                    conf.get("yandex", "sdk_user_agent_prefix", fallback=""),
-                    f"apache-airflow/{airflow_version}",
-                    f"{provider_name}/{provider.version}",
-                )
-            ).strip()
-        except KeyError:
-            warnings.warn(f"Hook '{cls.hook_name}' info is not initialized in airflow.ProviderManager")
-            return None
-

Review Comment:
   Public method. Should be deprecated first instead of remove



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,
+        folder_id: str = "",
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+        endpoint: str | None = None,
+    ):
+        super().__init__()
+
+        self.yc_oauth_token = yc_oauth_token
+        self.yc_sa_key_json = yc_sa_key_json
+        self.yc_sa_key_json_path = yc_sa_key_json_path
+        self.yc_connection_id = None
+        if not any([yc_oauth_token, yc_sa_key_json, yc_sa_key_json_path]):
+            self.yc_connection_id = yc_connection_id or default_conn_name
+        else:
+            assert (
+                yc_connection_id is None
+            ), "yc_connection_id should not be used if other credentials are specified"
+
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+        self.endpoint = endpoint
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: Connection ID
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if conn_id == self.yc_connection_id:
+            return None
+
+        return self._get_secret_value(self.connections_prefix, conn_id)
+
+    def get_variable(self, key: str) -> str | None:
+        """
+        Return value for Airflow Variable.
+
+        :param key: Variable Key
+        :return: Variable Value
+        """
+        if self.variables_prefix is None:
+            return None
+
+        return self._get_secret_value(self.variables_prefix, key)
+
+    def get_config(self, key: str) -> str | None:
+        """
+        Return value for Airflow Config Key.
+
+        :param key: Config Key
+        :return: Config Value
+        """
+        if self.config_prefix is None:
+            return None
+
+        return self._get_secret_value(self.config_prefix, key)
+
+    @cached_property
+    def _client(self):
+        """
+        Create a Yandex Cloud SDK client.
+
+        Lazy loading is used here
+        because we can't establish a Connection until all secrets backends have been initialized.
+        """
+        if self.yc_connection_id:
+            self.yc_oauth_token = self._get_field("oauth")
+            self.yc_sa_key_json = self._get_field("service_account_json")
+            self.yc_sa_key_json_path = self._get_field("service_account_json_path")
+            self.folder_id = self.folder_id or self._get_field("folder_id")
+
+        credentials = get_credentials(
+            oauth_token=self.yc_oauth_token,
+            service_account_json=self.yc_sa_key_json,
+            service_account_json_path=self.yc_sa_key_json_path,
+        )
+        sdk_config = self._get_endpoint()
+        return yandexcloud.SDK(user_agent=provider_user_agent(), **credentials, **sdk_config).client
+
+    def _get_endpoint(self) -> dict[str, str]:
+        sdk_config = {}
+
+        if self.endpoint:
+            sdk_config["endpoint"] = self.endpoint
+
+        return sdk_config
+
+    @cached_property
+    def _connection(self) -> Connection | None:
+        if not self.yc_connection_id:
+            return None
+
+        conn = Connection.get_connection_from_secrets(self.yc_connection_id)
+        logging.info("Using connection ID '%s' for task execution.", conn.conn_id)

Review Comment:
   Same here with Root logger you either could create new logger in top of the module or inherit class from `airflow.utils.log.logging_mixin.LoggingMixin`
   
   
   
   ```python
   ...
   from airflow.utils.log.logging_mixin import LoggingMixin
   ...
   
   class LockboxSecretBackend(BaseSecretsBackend, LoggingMixin):
       ...
   
       @cached_property
       def _connection(self) -> Connection | None:
           ...
           self.log.info("Using connection ID '%s' for task execution.", conn.conn_id)
   ```



##########
airflow/providers/yandex/hooks/yandex.py:
##########
@@ -117,7 +101,7 @@ def get_ui_field_behaviour(cls) -> dict[str, Any]:
 
     def __init__(
         self,
-        # Connection id is deprecated. Use yandex_conn_id instead
+        # connection_id is deprecated, use yandex_conn_id instead

Review Comment:
   I guess better to better move this information into the docstring.
   
   https://github.com/apache/airflow/blob/b1ce06ec54803be2768bb0b68db4e21dab119674/airflow/providers/yandex/hooks/yandex.py#L36-L40
   
   This would reflected into [Class API](https://airflow.apache.org/docs/apache-airflow-providers-yandex/stable/_api/airflow/providers/yandex/hooks/yandex/index.html) as well as into the end users / developers IDEs



##########
airflow/providers/yandex/utils/credentials.py:
##########
@@ -0,0 +1,96 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import json
+import logging
+from typing import Any
+
+
+def get_credentials(
+    oauth_token: str | None = None,
+    service_account_json: str | None = None,
+    service_account_json_path: str | None = None,
+) -> dict[str, Any]:
+    """
+    Return credentials JSON for Yandex Cloud SDK based on credentials.
+
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Metadata Service
+
+    :param oauth_token: OAuth Token
+    :param service_account_json: Service Account JSON key
+    :param service_account_json_path: Service Account JSON key file path
+    :return: Credentials JSON
+    """
+    if oauth_token:
+        return {"token": oauth_token}
+
+    service_account_key = get_service_account_key(
+        service_account_json=service_account_json,
+        service_account_json_path=service_account_json_path,
+    )
+    if service_account_key:
+        return {"service_account_key": service_account_key}
+
+    logging.info("using metadata service as credentials")

Review Comment:
   There is no reason to use Root logger, you might just create new one and use it instead
   
   ```python
   
   import logging
   ...
   
   log = logging.getLogger(__name__)
   
   ...
   
   
   def get_credentials(
       oauth_token: str | None = None,
       service_account_json: str | None = None,
       service_account_json_path: str | None = None,
   ) -> dict[str, Any]:
       ...
       log.info("using metadata service as credentials")
   ```



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

Review Comment:
   `hook_name` it is refers to how Connection would be named in to the UI Connection type drop list
   
   ```suggestion
   hook_name = "Yandex Cloud"
   ```
   
   ![image](https://github.com/apache/airflow/assets/3998685/888322b7-c553-423c-8871-1f2eec2fbf43)
   



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend

Review Comment:
   I guess you forgot to change the class name here
   
   ```suggestion
           backend = airflow.providers.yandex.secrets.secrets_manager.LockboxSecretBackend
   ```



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   I would rather avoid to use any Airflow Connections into the Secrets Backend.
   
   1. **Chicken and Egg problem**: Very easy to get into the situation that Backend not yet initialised
   2. **Security/Maintainability**: Authenticated UI users who has access to the Connections (Metadata) might change creds and make Deployment Managers surprised. See: [Airflow Security Model](https://airflow.apache.org/docs/apache-airflow/stable/security/security_model.html)
   



##########
airflow/providers/yandex/utils/fields.py:
##########
@@ -0,0 +1,42 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+from typing import Any
+
+
+def get_field_from_extras(extras: dict[str, Any], field_name: str, default: Any = None) -> Any:
+    """
+    Get field from extras, first checking short name, then for backcompat checking for prefixed name.
+
+    :param extras: Dictionary with extras keys
+    :param field_name: Field name to get from extras
+    :param default: Default value if field not found
+    :return: Field value or default if not found
+    """
+    backcompat_prefix = "extra__yandexcloud__"
+    if field_name.startswith("extra__"):
+        raise ValueError(
+            f"Got prefixed name {field_name}; please remove the '{backcompat_prefix}' prefix "
+            "when using this method."

Review Comment:
   ```suggestion
               "when using this function."
   ```



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,

Review Comment:
   What if make this parameter either python dictionary or string? In this case users might avoid escaping this parameter in `[secrets] backend_kwargs`
   
   https://github.com/apache/airflow/blob/b1ce06ec54803be2768bb0b68db4e21dab119674/docs/apache-airflow-providers-yandex/secrets-backends/yandex-cloud-lockbox-secret-backend.rst?plain=1#L126-L131
   
   



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend

Review Comment:
   ```suggestion
           backend = airflow.providers.yandex.secrets.secrets_manager.LockboxSecretBackend
   ```



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########


Review Comment:
   If you wanted you also might change the name of the module to more Yandex specific. I guess this file originally based on `airflow/providers/amazon/aws/secrets/secrets_manager.py` where secrets_manager is stands for AWS Secrets Manager service



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1459030435


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   > This problem is addressed in this PR
   
   That might be not such as straight forward. Secrets Manager has particular issue not only directly with connection but with other components and it is very difficult to detect in regular unit tests because during unit tests everything already initialised, however in regular usage it is not.
   
   Something what we have in the past with other Secrets Backends:
   - Try to obtain Connection to the DB from the secrets backend was raised circular import error: https://github.com/apache/airflow/pull/26784/files
   - Try to obtain Fernet key from the secrets backend was raised circular import error.
   
   This past issues not directly related to this PR, but still worthwhile to check it with this provider, I guess one of this might produce some interesting errors, or not.
   
   >I guess Deployment Manager can just choose to not use Connection in SecretManager configuration if that connection metadata is not trustworthy enough.
   
   Yeah, I've checked that couple other components also use conn_id, e.g. remote logging mechanism. So maybe this one not so critical than I thought initially
   
   > On the other hand, ability to not duplicate creds between SecretsManager and Hook looks profitable for simple installations.
   
   This is good idea to have some wrapper class / sets of function to convert provided parameters to the valid parameters for the Hook and SecretsManager, the same approach use into the Amazon Provider and I guess into the Google provider. My point here more about whether or not is a good idea to use Connection ID in Secrets Manager.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "ImpressionableRaccoon (via GitHub)" <gi...@apache.org>.
ImpressionableRaccoon commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1437044907


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify id of yandexcloud connection to connect to Yandex Lockbox with.
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        If set to None (null), requests will use default connection_id from YandexCloudBaseHook.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null), requests will use the connection folder_id.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        connection_id: str | None = None,
+        folder_id: str | None = None,
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.connection_id = connection_id
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        return self._base_hook.client
+
+    @cached_property
+    def _base_hook(self) -> YandexCloudBaseHook:
+        return YandexCloudBaseHook(
+            yandex_conn_id=self.connection_id,
+            default_folder_id=self.folder_id,
+        )
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: connection id
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if conn_id == self.connection_id:
+            return None
+
+        if not self.connection_id and conn_id == YandexCloudBaseHook.default_conn_name:

Review Comment:
   It's similar to the previous one. It's needed because the secrets backend needs a connection to connect to Lockbox. It will cause a recursion issue if the secrets manager backend tries to answer a connection request from itself.
   
   ```
   conn_id == self.connection_id
   ```
   
   This happens when the user explicitly specifies the connection_id.
   
   ```
   not self.connection_id and conn_id == YandexCloudBaseHook.default_conn_name.
   ```
   
   This happens when the user does not specify a connection_id. In this case, the Yandex Cloud SDK will use `YandexCloudBaseHook.default_conn_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: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "ImpressionableRaccoon (via GitHub)" <gi...@apache.org>.
ImpressionableRaccoon commented on PR #36449:
URL: https://github.com/apache/airflow/pull/36449#issuecomment-1904146739

   I think it's ready to merge


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1436976250


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify id of yandexcloud connection to connect to Yandex Lockbox with.
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        If set to None (null), requests will use default connection_id from YandexCloudBaseHook.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null), requests will use the connection folder_id.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        connection_id: str | None = None,
+        folder_id: str | None = None,
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.connection_id = connection_id
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        return self._base_hook.client
+
+    @cached_property
+    def _base_hook(self) -> YandexCloudBaseHook:
+        return YandexCloudBaseHook(
+            yandex_conn_id=self.connection_id,
+            default_folder_id=self.folder_id,
+        )

Review Comment:
   I think this one required to have a connection id somewhere else, e.g. Environment Variable or Airflow DB Backend first. All other providers provide ability to use it entirely without other backend. 
   
   Better to have an ability to provide credentials thought backend_config, by the same way as it done in other community providers secrets backends:
   
   **SystemsManagerParameterStoreBackend from Amazon Provider**
   https://github.com/apache/airflow/blob/71c726d52d5a8a30f59268cc175560a4244c8016/airflow/providers/amazon/aws/secrets/systems_manager.py#L64-L67
   
   **VaultBackend from Hashicorp provider**
   https://github.com/apache/airflow/blob/71c726d52d5a8a30f59268cc175560a4244c8016/airflow/providers/hashicorp/secrets/vault.py#L66-L85
   
   **CloudSecretManagerBackend from Google Provider**
   https://github.com/apache/airflow/blob/71c726d52d5a8a30f59268cc175560a4244c8016/airflow/providers/google/cloud/secrets/secret_manager.py#L75-L79
   
   **AzureKeyVaultBackend from Microsoft Azure provider**
   https://github.com/apache/airflow/blob/71c726d52d5a8a30f59268cc175560a4244c8016/airflow/providers/microsoft/azure/secrets/key_vault.py#L83-L91



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify id of yandexcloud connection to connect to Yandex Lockbox with.
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        If set to None (null), requests will use default connection_id from YandexCloudBaseHook.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null), requests will use the connection folder_id.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        connection_id: str | None = None,
+        folder_id: str | None = None,
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.connection_id = connection_id
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        return self._base_hook.client
+
+    @cached_property
+    def _base_hook(self) -> YandexCloudBaseHook:
+        return YandexCloudBaseHook(
+            yandex_conn_id=self.connection_id,
+            default_folder_id=self.folder_id,
+        )
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: connection id
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if conn_id == self.connection_id:
+            return None
+
+        if not self.connection_id and conn_id == YandexCloudBaseHook.default_conn_name:

Review Comment:
   Not clear about this condition: `conn_id == YandexCloudBaseHook.default_conn_name`, thy it is required.



##########
tests/providers/yandex/secrets/test_secrets_manager.py:
##########
@@ -0,0 +1,343 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+from __future__ import annotations
+
+import json
+from unittest.mock import Mock, patch
+
+import pytest
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+
+try:
+    import yandex
+    import yandexcloud
+
+    from airflow.providers.yandex.secrets.secrets_manager import SecretsManagerBackend
+except ImportError:
+    yandex = None
+    yandexcloud = None

Review Comment:
   I think this block is redundant, maybe it exists from the past when yandex provider do not preinstall or have some issue with ARM image in the past (difficult to guess).
   
   You could get rid of try-except block as well as  `pytest.mark.skipif` bellow
   
   ```suggestion
   import yandex
   import yandexcloud
   from airflow.providers.yandex.secrets.secrets_manager import SecretsManagerBackend
   ```



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):

Review Comment:
   Also expected to have a documentation in the provider documentation. By the same way as it created into the:
   - [Amazon Provider](https://airflow.apache.org/docs/apache-airflow-providers-amazon/stable/secrets-backends/index.html)
   - [Google Provider](https://airflow.apache.org/docs/apache-airflow-providers-google/stable/secrets-backends/google-cloud-secret-manager-backend.html)
   - [Hashicorp Provider](https://airflow.apache.org/docs/apache-airflow-providers-hashicorp/stable/secrets-backends/hashicorp-vault.html)
   - [Microsoft Azure Provider ](https://airflow.apache.org/docs/apache-airflow-providers-microsoft-azure/stable/secrets-backends/azure-key-vault.html)



##########
airflow/providers/yandex/__init__.py:
##########
@@ -27,7 +27,7 @@
 
 __all__ = ["__version__"]
 
-__version__ = "3.7.1"
+__version__ = "3.8.0"

Review Comment:
   This also should not be touched until provider released



##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):

Review Comment:
   Maybe better to have less generic name, what about `LockboxSecretsManagerBackend` or `YandexSecretsManagerBackend` or `YandexLockboxSecretsManagerBackend`



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Piatachock (via GitHub)" <gi...@apache.org>.
Piatachock commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1457026974


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########


Review Comment:
   It's  not necessary IMHO, since `yandex` is already mentioned in module path, and you won't need to distinguish between different secrets manager implementation in one place, since you normally only use one at a time.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "boring-cyborg[bot] (via GitHub)" <gi...@apache.org>.
boring-cyborg[bot] commented on PR #36449:
URL: https://github.com/apache/airflow/pull/36449#issuecomment-1909937318

   Awesome work, congrats on your first merged pull request! You are invited to check our [Issue Tracker](https://github.com/apache/airflow/issues) for additional contributions.
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1459063566


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   I think 2. Is not really problematic. The user who can edit connections is highly priviledged already and can access (read/write) any connection credentials. In this case though it gives a little different capabilities - you could redirect all secret queries to a different place for example. Looking at Confidentiality, Integrity and Availability:
   
   * I think it does not impact Confidentiality (because such change will not "leak" any Airflow secrets to users who have no access to them - more than what they already have).   
   * It has a little potential to Impact Integrity - but no more than for example individually changing connections to point to different places for example and breaking integrity of the workflows - Connection UI Editing user already has those capabilities
   * It has a potentnial to impact Availability - but no more than what the Connection UI Editing user already has by individually modifying connections.
   
   So - I do not think it elevates privilledges of such user. 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1459062085


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########


Review Comment:
   "There are only two hard things in Computer Science: cache invalidation and naming things."
   
   And this is second 🀣 .
   
   My comment more was more a question rather than actual suggestion. Usual the module naming stand for the naming of service or software. So if you think that naming is fine, then I have no objection



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1456603272


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,280 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+import logging
+from functools import cached_property
+from typing import Any
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+import yandexcloud
+
+from airflow.models import Connection
+from airflow.providers.yandex.utils.credentials import get_credentials
+from airflow.providers.yandex.utils.defaults import default_conn_name
+from airflow.providers.yandex.utils.fields import get_field_from_extras
+from airflow.providers.yandex.utils.user_agent import provider_user_agent
+from airflow.secrets import BaseSecretsBackend
+
+
+class LockboxSecretBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables or Configs from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"yc_connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify credentials or id of yandexcloud connection to connect to Yandex Lockbox with.
+    Credentials will be used with this priority:
+
+    * OAuth Token
+    * Service Account JSON file
+    * Service Account JSON
+    * Yandex Cloud Connection
+
+    If no credentials specified, default connection id will be used.
+
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param yc_oauth_token: Specifies the user account OAuth token to connect to Yandex Lockbox with.
+        Looks like ``y3_xxxxx``.
+    :param yc_sa_key_json: Specifies the service account auth JSON.
+        Looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_sa_key_json_path: Specifies the service account auth JSON file path.
+        Looks like ``/home/airflow/authorized_key.json``.
+        File content looks like ``{"id": "...", "service_account_id": "...", "private_key": "..."}``.
+    :param yc_connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null in JSON), requests will use the connection folder_id if specified.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null in JSON), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null in JSON), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    :param endpoint: Specifies an API endpoint.
+        Leave blank to use default.
+    """
+
+    def __init__(
+        self,
+        yc_oauth_token: str | None = None,
+        yc_sa_key_json: str | None = None,
+        yc_sa_key_json_path: str | None = None,
+        yc_connection_id: str | None = None,

Review Comment:
   @potiuk @eladkal
   I want to validate my concerns, I don’t want to force to rewrite half of the logic if it just my paranoia πŸ™„ 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1437070445


##########
airflow/providers/yandex/hooks/yandex.py:
##########
@@ -167,10 +168,8 @@ def _get_credentials(self) -> dict[str, Any]:
         if service_account_key:
             return {"service_account_key": service_account_key}
 
-        raise AirflowException(
-            "No credentials are found in connection. Specify either service account "
-            "authentication JSON or user OAuth token in Yandex.Cloud connection"
-        )
+        logging.info("using metadata service as credentials")

Review Comment:
   [`BaseHook`](https://github.com/apache/airflow/blob/2bcd450e84426fd678b3fa2e4a15757af234e98a/airflow/hooks/base.py#L35) based on [`LoggingMixin`](https://github.com/apache/airflow/blob/1e1adc569f43494aabf3712b651956636c04df7f/airflow/utils/log/logging_mixin.py#L67), so you could call `self.log.<logger-method>` instead of create separate logger / use root logger
   
   ```suggestion
           self.log.info("using metadata service as credentials")
   ```



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

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


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "Taragolis (via GitHub)" <gi...@apache.org>.
Taragolis commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1459066788


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########
@@ -0,0 +1,209 @@
+# 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.
+"""Objects relating to sourcing secrets from Yandex Cloud Lockbox."""
+from __future__ import annotations
+
+from functools import cached_property
+
+import yandex.cloud.lockbox.v1.payload_pb2 as payload_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2 as payload_service_pb
+import yandex.cloud.lockbox.v1.payload_service_pb2_grpc as payload_service_pb_grpc
+import yandex.cloud.lockbox.v1.secret_pb2 as secret_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2 as secret_service_pb
+import yandex.cloud.lockbox.v1.secret_service_pb2_grpc as secret_service_pb_grpc
+
+from airflow.providers.yandex.hooks.yandex import YandexCloudBaseHook
+from airflow.secrets import BaseSecretsBackend
+
+
+class SecretsManagerBackend(BaseSecretsBackend):
+    """
+    Retrieves Connection or Variables from Yandex Lockbox.
+
+    Configurable via ``airflow.cfg`` like so:
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connections_prefix": "airflow/connections"}
+
+    For example, when ``{"connections_prefix": "airflow/connections"}`` is set, if a secret is defined with
+    the path ``airflow/connections/smtp_default``, the connection with conn_id ``smtp_default`` would be
+    accessible.
+
+    When ``{"variables_prefix": "airflow/variables"}`` is set, if a secret is defined with
+    the path ``airflow/variables/hello``, the variable with the name ``hello`` would be accessible.
+
+    When ``{"config_prefix": "airflow/config"}`` is set, if a secret is defined with
+    the path ``airflow/config/sql_alchemy_conn``, the config with key ``sql_alchemy_conn`` would be
+    accessible.
+
+    When the prefix is empty, keys will use the Lockbox Secrets without any prefix.
+
+    .. code-block:: ini
+
+        [secrets]
+        backend = airflow.providers.yandex.secrets.secrets_manager.SecretsManagerBackend
+        backend_kwargs = {"connection_id": "<connection_ID>", "folder_id": "<folder_ID>"}
+
+    You need to specify id of yandexcloud connection to connect to Yandex Lockbox with.
+    Also, you need to specify the Yandex Cloud folder ID to search for Yandex Lockbox secrets in.
+
+    :param connection_id: Specifies the connection ID to connect to Yandex Lockbox with.
+        If set to None (null), requests will use default connection_id from YandexCloudBaseHook.
+        Default: "yandexcloud_default"
+    :param folder_id: Specifies the folder ID to search for Yandex Lockbox secrets in.
+        If set to None (null), requests will use the connection folder_id.
+    :param connections_prefix: Specifies the prefix of the secret to read to get Connections.
+        If set to None (null), requests for connections will not be sent to Yandex Lockbox.
+        Default: "airflow/connections"
+    :param variables_prefix: Specifies the prefix of the secret to read to get Variables.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/variables"
+    :param config_prefix: Specifies the prefix of the secret to read to get Configurations.
+        If set to None (null), requests for variables will not be sent to Yandex Lockbox.
+        Default: "airflow/config"
+    :param sep: Specifies the separator used to concatenate secret_prefix and secret_id.
+        Default: "/"
+    """
+
+    def __init__(
+        self,
+        connection_id: str | None = None,
+        folder_id: str | None = None,
+        connections_prefix: str | None = "airflow/connections",
+        variables_prefix: str | None = "airflow/variables",
+        config_prefix: str | None = "airflow/config",
+        sep: str = "/",
+    ):
+        super().__init__()
+
+        self.connection_id = connection_id
+        self.folder_id = folder_id
+        self.connections_prefix = connections_prefix.rstrip(sep) if connections_prefix is not None else None
+        self.variables_prefix = variables_prefix.rstrip(sep) if variables_prefix is not None else None
+        self.config_prefix = config_prefix.rstrip(sep) if config_prefix is not None else None
+        self.sep = sep
+
+    @property
+    def _client(self):
+        """Create a Yandex Cloud SDK client."""
+        return self._base_hook.client
+
+    @cached_property
+    def _base_hook(self) -> YandexCloudBaseHook:
+        return YandexCloudBaseHook(
+            yandex_conn_id=self.connection_id,
+            default_folder_id=self.folder_id,
+        )
+
+    def get_conn_value(self, conn_id: str) -> str | None:
+        """
+        Retrieve from Secrets Backend a string value representing the Connection object.
+
+        :param conn_id: connection id
+        :return: Connection Value
+        """
+        if self.connections_prefix is None:
+            return None
+
+        if conn_id == self.connection_id:
+            return None
+
+        if not self.connection_id and conn_id == YandexCloudBaseHook.default_conn_name:

Review Comment:
   Resolve in favor of https://github.com/apache/airflow/pull/36449#discussion_r1456111043 



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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


Re: [PR] feat: add Yandex Cloud Lockbox secrets backend [airflow]

Posted by "ImpressionableRaccoon (via GitHub)" <gi...@apache.org>.
ImpressionableRaccoon commented on code in PR #36449:
URL: https://github.com/apache/airflow/pull/36449#discussion_r1459122951


##########
airflow/providers/yandex/secrets/secrets_manager.py:
##########


Review Comment:
   renamed to `airflow/providers/yandex/secrets/lockbox.py` πŸ™ˆ
   because this is for Yandex Lockbox service



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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