diff --git a/.github/workflows/check-providers.yml b/.github/workflows/check-providers.yml index e89d4a81faaca..b394f7927329a 100644 --- a/.github/workflows/check-providers.yml +++ b/.github/workflows/check-providers.yml @@ -108,6 +108,11 @@ jobs: run: > breeze release-management generate-issue-content-providers --only-available-in-dist --disable-progress + - name: > + Remove incompatible Python ${{ matrix.python-version }} provider packages + run: | + echo "Removing Python 3.8-incompatible provider: cloudant" + rm -vf dist/apache_airflow_providers_cloudant* - name: "Generate source constraints from CI image" shell: bash run: > diff --git a/airflow/providers/cloudant/CHANGELOG.rst b/airflow/providers/cloudant/CHANGELOG.rst index ceb4d95ed6328..a32cd2b42f2ec 100644 --- a/airflow/providers/cloudant/CHANGELOG.rst +++ b/airflow/providers/cloudant/CHANGELOG.rst @@ -27,6 +27,14 @@ Changelog --------- +4.0.0 +..... + +Breaking changes +~~~~~~~~~~~~~~~~ + +* ``Switch cloudant provider from cloudant library to ibmcloudant library (#41555)`` + 3.6.0 ..... diff --git a/airflow/providers/cloudant/cloudant_fake.py b/airflow/providers/cloudant/cloudant_fake.py new file mode 100644 index 0000000000000..91b97c72a91c8 --- /dev/null +++ b/airflow/providers/cloudant/cloudant_fake.py @@ -0,0 +1,34 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + + +class CloudantV1: + """Phony class to pass mypy when real class is not imported.""" + + def __init__(self, authenticator): + pass + + def set_service_url(self, service_url: str): + pass + + +class CouchDbSessionAuthenticator: + """Phony class to pass mypy when real class is not imported.""" + + def __init__(self, username: str, password: str): + pass diff --git a/airflow/providers/cloudant/hooks/cloudant.py b/airflow/providers/cloudant/hooks/cloudant.py index 41a2316ca89ba..b09ad6932efa6 100644 --- a/airflow/providers/cloudant/hooks/cloudant.py +++ b/airflow/providers/cloudant/hooks/cloudant.py @@ -19,13 +19,20 @@ from __future__ import annotations -from typing import Any +import sys +from typing import TYPE_CHECKING, Any -from cloudant import cloudant # type: ignore[attr-defined] +if sys.version_info < (3, 10): + from airflow.providers.cloudant.cloudant_fake import CloudantV1, CouchDbSessionAuthenticator +else: + from ibmcloudant import CloudantV1, CouchDbSessionAuthenticator from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook +if TYPE_CHECKING: + from airflow.models import Connection + class CloudantHook(BaseHook): """ @@ -45,35 +52,44 @@ class CloudantHook(BaseHook): def get_ui_field_behaviour(cls) -> dict[str, Any]: """Return custom field behaviour.""" return { - "hidden_fields": ["port", "extra"], - "relabeling": {"host": "Account", "login": "Username (or API Key)", "schema": "Database"}, + "hidden_fields": ["schema", "port", "extra"], + "relabeling": {"host": "Account", "login": "Username (or API Key)"}, } def __init__(self, cloudant_conn_id: str = default_conn_name) -> None: super().__init__() self.cloudant_conn_id = cloudant_conn_id - def get_conn(self) -> cloudant: + def get_conn(self) -> CloudantV1: """ - Open a connection to the cloudant service and close it automatically if used as context manager. + Create an authenticated service object for connection to the Cloudant service. .. note:: In the connection form: - - 'host' equals the 'Account' (optional) + - 'host' equals the 'Account' (required) - 'login' equals the 'Username (or API Key)' (required) - 'password' equals the 'Password' (required) - :return: an authorized cloudant session context manager object. + :return: a CloudantV1 service object backed by a session-based user/password authenticator. """ conn = self.get_connection(self.cloudant_conn_id) self._validate_connection(conn) - cloudant_session = cloudant(user=conn.login, passwd=conn.password, account=conn.host) + authenticator = CouchDbSessionAuthenticator(username=conn.login, password=conn.password) + service = CloudantV1(authenticator=authenticator) + service.set_service_url(f"https://{conn.host}.cloudant.com") - return cloudant_session + return service - def _validate_connection(self, conn: cloudant) -> None: - for conn_param in ["login", "password"]: + @staticmethod + def _validate_connection(conn: Connection) -> None: + missing_params = [] + for conn_param in ["host", "login", "password"]: if not getattr(conn, conn_param): - raise AirflowException(f"missing connection parameter {conn_param}") + missing_params.append(conn_param) + + if missing_params: + raise AirflowException( + f"Missing connection parameter{'s' if len(missing_params) > 1 else ''}: {', '.join(missing_params)}" + ) diff --git a/airflow/providers/cloudant/provider.yaml b/airflow/providers/cloudant/provider.yaml index 3f645d808ba92..09857936b877b 100644 --- a/airflow/providers/cloudant/provider.yaml +++ b/airflow/providers/cloudant/provider.yaml @@ -25,6 +25,7 @@ state: ready source-date-epoch: 1723969866 # note that those versions are maintained by release manager - do not update them manually versions: + - 4.0.0 - 3.6.0 - 3.5.2 - 3.5.1 @@ -46,7 +47,15 @@ versions: dependencies: - apache-airflow>=2.8.0 - - cloudant>=2.13.0 + - ibmcloudant>=0.9.1 + +excluded-python-versions: + # ibmcloudant transitively brings in urllib3 2.x, but the snowflake provider has a dependency that pins + # urllib3 to 1.x on Python 3.8 and 3.9; thus we exclude those Python versions from taking the update + # to ibmcloudant. + # See #21004, #41555, and https://github.com/snowflakedb/snowflake-connector-python/issues/2016 + - "3.8" + - "3.9" integrations: - integration-name: IBM Cloudant diff --git a/dev/breeze/src/airflow_breeze/global_constants.py b/dev/breeze/src/airflow_breeze/global_constants.py index 6f6545bf8f67d..ab191e4f1b36d 100644 --- a/dev/breeze/src/airflow_breeze/global_constants.py +++ b/dev/breeze/src/airflow_breeze/global_constants.py @@ -509,19 +509,19 @@ def get_airflow_extras(): { "python-version": "3.8", "airflow-version": "2.8.4", - "remove-providers": "fab", + "remove-providers": "cloudant fab", "run-tests": "true", }, { "python-version": "3.8", "airflow-version": "2.9.3", - "remove-providers": "", + "remove-providers": "cloudant", "run-tests": "true", }, { "python-version": "3.8", "airflow-version": "2.10.0", - "remove-providers": "", + "remove-providers": "cloudant", "run-tests": "true", }, ] diff --git a/dev/breeze/src/airflow_breeze/utils/selective_checks.py b/dev/breeze/src/airflow_breeze/utils/selective_checks.py index d432d8fb207f9..eb38675fb3aab 100644 --- a/dev/breeze/src/airflow_breeze/utils/selective_checks.py +++ b/dev/breeze/src/airflow_breeze/utils/selective_checks.py @@ -81,7 +81,6 @@ USE_PUBLIC_RUNNERS_LABEL = "use public runners" USE_SELF_HOSTED_RUNNERS_LABEL = "use self-hosted runners" - ALL_CI_SELECTIVE_TEST_TYPES = ( "API Always BranchExternalPython BranchPythonVenv " "CLI Core ExternalPython Operators Other PlainAsserts " @@ -1278,7 +1277,10 @@ def excluded_providers_as_string(self) -> str: if "excluded-python-versions" in provider_info: for python_version in provider_info["excluded-python-versions"]: providers_to_exclude[python_version].append(provider) - return json.dumps(providers_to_exclude) + sorted_providers_to_exclude = dict( + sorted(providers_to_exclude.items(), key=lambda item: int(item[0].split(".")[1])) + ) # ^ sort by Python minor version + return json.dumps(sorted_providers_to_exclude) @cached_property def testable_integrations(self) -> list[str]: diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py index 9e4476425207d..2491b6f920cf8 100644 --- a/dev/breeze/tests/test_selective_checks.py +++ b/dev/breeze/tests/test_selective_checks.py @@ -853,7 +853,9 @@ def test_excluded_providers(): ) assert_outputs_are_printed( { - "excluded-providers-as-string": json.dumps({"3.12": ["apache.beam"]}), + "excluded-providers-as-string": json.dumps( + {"3.8": ["cloudant"], "3.9": ["cloudant"], "3.12": ["apache.beam"]} + ), }, str(stderr), ) diff --git a/docs/spelling_wordlist.txt b/docs/spelling_wordlist.txt index 46b441ef3e3f6..2556113d001dc 100644 --- a/docs/spelling_wordlist.txt +++ b/docs/spelling_wordlist.txt @@ -250,8 +250,8 @@ classpaths cleartext cli clientId -Cloudant cloudant +CloudantV cloudbuild CloudBuildClient cloudml @@ -775,6 +775,7 @@ hyperparameter hyperparameters IaC iam +ibmcloudant idempotence idempotency IdP diff --git a/generated/provider_dependencies.json b/generated/provider_dependencies.json index a091b47d9d900..7e158f476af4f 100644 --- a/generated/provider_dependencies.json +++ b/generated/provider_dependencies.json @@ -352,12 +352,15 @@ "cloudant": { "deps": [ "apache-airflow>=2.8.0", - "cloudant>=2.13.0" + "ibmcloudant>=0.9.1" ], "devel-deps": [], "plugins": [], "cross-providers-deps": [], - "excluded-python-versions": [], + "excluded-python-versions": [ + "3.8", + "3.9" + ], "state": "ready" }, "cncf.kubernetes": { diff --git a/tests/always/test_project_structure.py b/tests/always/test_project_structure.py index ae59f12ffcb1b..cf87efebe3238 100644 --- a/tests/always/test_project_structure.py +++ b/tests/always/test_project_structure.py @@ -85,6 +85,7 @@ def test_providers_modules_should_have_tests(self): "tests/providers/apache/hive/plugins/test_hive.py", "tests/providers/celery/executors/test_celery_executor_utils.py", "tests/providers/celery/executors/test_default_celery.py", + "tests/providers/cloudant/test_cloudant_fake.py", "tests/providers/cncf/kubernetes/backcompat/test_backwards_compat_converters.py", "tests/providers/cncf/kubernetes/executors/test_kubernetes_executor_types.py", "tests/providers/cncf/kubernetes/executors/test_kubernetes_executor_utils.py", diff --git a/tests/providers/cloudant/hooks/test_cloudant.py b/tests/providers/cloudant/hooks/test_cloudant.py index 8e7d3cd9a760a..eec71f65e79b3 100644 --- a/tests/providers/cloudant/hooks/test_cloudant.py +++ b/tests/providers/cloudant/hooks/test_cloudant.py @@ -17,15 +17,24 @@ # under the License. from __future__ import annotations +import sys from unittest.mock import patch import pytest from airflow.exceptions import AirflowException from airflow.models import Connection -from airflow.providers.cloudant.hooks.cloudant import CloudantHook -pytestmark = pytest.mark.db_test +pytestmark = [pytest.mark.db_test] + +if sys.version_info < (3, 10): + pytestmark.append( + pytest.mark.skip( + f"Skipping {__name__} as the cloudant provider is not supported on Python 3.8 and 3.9, see #41555." + ) + ) +else: + from airflow.providers.cloudant.hooks.cloudant import CloudantHook class TestCloudantHook: @@ -34,20 +43,39 @@ def setup_method(self): @patch( "airflow.providers.cloudant.hooks.cloudant.CloudantHook.get_connection", - return_value=Connection(login="user", password="password", host="account"), + return_value=Connection(login="the_user", password="the_password", host="the_account"), ) - @patch("airflow.providers.cloudant.hooks.cloudant.cloudant") - def test_get_conn(self, mock_cloudant, mock_get_connection): + @patch("airflow.providers.cloudant.hooks.cloudant.CouchDbSessionAuthenticator") + @patch("airflow.providers.cloudant.hooks.cloudant.CloudantV1") + def test_get_conn_passes_expected_params_and_returns_cloudant_object( + self, mock_cloudant_v1, mock_session_authenticator, mock_get_connection + ): cloudant_session = self.cloudant_hook.get_conn() conn = mock_get_connection.return_value - mock_cloudant.assert_called_once_with(user=conn.login, passwd=conn.password, account=conn.host) - assert cloudant_session == mock_cloudant.return_value - @patch( - "airflow.providers.cloudant.hooks.cloudant.CloudantHook.get_connection", - return_value=Connection(login="user"), + mock_session_authenticator.assert_called_once_with(username=conn.login, password=conn.password) + mock_cloudant_v1.assert_called_once_with(authenticator=mock_session_authenticator.return_value) + + cloudant_service = mock_cloudant_v1.return_value + cloudant_service.set_service_url.assert_called_once_with(f"https://{conn.host}.cloudant.com") + + assert cloudant_session == cloudant_service + + @pytest.mark.parametrize( + "conn", + [ + Connection(), + Connection(host="acct"), + Connection(login="user"), + Connection(password="pwd"), + Connection(host="acct", login="user"), + Connection(host="acct", password="pwd"), + Connection(login="user", password="pwd"), + ], ) - def test_get_conn_invalid_connection(self, mock_get_connection): + @patch("airflow.providers.cloudant.hooks.cloudant.CloudantHook.get_connection") + def test_get_conn_invalid_connection(self, mock_get_connection, conn): + mock_get_connection.return_value = conn with pytest.raises(AirflowException): self.cloudant_hook.get_conn()