You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2021/04/06 21:15:08 UTC

[airflow] 01/01: Fix make_kwargs_callable import for http provider (out of bands)

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

potiuk pushed a commit to branch out-of-bands-backport-providers
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 0d2ec4262bc60218ea4595bb6a694283f68f28f4
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Apr 6 22:20:56 2021 +0200

    Fix make_kwargs_callable import for http provider (out of bands)
    
    Fixes failing import problem of the http backport
    provider with Airflow 1.10.* series.
    
    A problem was introduced in #11922 which cause the http provider
    to stop working (local import was not caught at the review time
    and as local import it has not been caught by the test harness).
    
    Since the http provider is defunct and is very popular, we
    decided to release an out-of-band release of the http provider
    even if backport providers reached end-of-life.
    
    This PR copies implementation of make_kwargs_callable into http
    backport provider to deal with the incompatibility.
    
    Fixes: #15198
---
 .../http/BACKPORT_PROVIDER_CHANGES_2021.04.10.md   |  9 +++++
 airflow/providers/http/BACKPORT_PROVIDER_README.md | 11 +++++-
 airflow/providers/http/__init__.py                 | 46 ++++++++++++++++++++++
 airflow/providers/http/operators/http.py           |  3 +-
 airflow/providers/http/sensors/http.py             |  3 +-
 scripts/ci/libraries/_initialization.sh            |  2 +-
 setup.py                                           |  2 +-
 7 files changed, 69 insertions(+), 7 deletions(-)

diff --git a/airflow/providers/http/BACKPORT_PROVIDER_CHANGES_2021.04.10.md b/airflow/providers/http/BACKPORT_PROVIDER_CHANGES_2021.04.10.md
new file mode 100644
index 0000000..719e51c
--- /dev/null
+++ b/airflow/providers/http/BACKPORT_PROVIDER_CHANGES_2021.04.10.md
@@ -0,0 +1,9 @@
+
+
+### Release 2021.4.10
+
+| Commit                                                                                         | Committed   | Subject                                                   |
+|:-----------------------------------------------------------------------------------------------|:------------|:----------------------------------------------------------|
+| [3df23d724](https://github.com/apache/airflow/commit/3df23d7243f426f416f4f226d17d7c094c50aa3e) | 2021-04-06  | `Fix make_kwargs_callable import (out of bands)`          |
+| [68e4c4dcb](https://github.com/apache/airflow/commit/68e4c4dcb0416eb51a7011a3bb040f1e23d7bba8) | 2021-03-20  | `Remove Backport Providers (#14886)`                      |
+| [589d6dec9](https://github.com/apache/airflow/commit/589d6dec922565897785bcbc5ac6bb3b973d7f5d) | 2021-02-27  | `Prepare to release the next wave of providers: (#14487)` |
diff --git a/airflow/providers/http/BACKPORT_PROVIDER_README.md b/airflow/providers/http/BACKPORT_PROVIDER_README.md
index 58c8e45..b2c5a5a 100644
--- a/airflow/providers/http/BACKPORT_PROVIDER_README.md
+++ b/airflow/providers/http/BACKPORT_PROVIDER_README.md
@@ -20,7 +20,7 @@
 
 # Package apache-airflow-backport-providers-http
 
-Release: 2021.3.3
+Release: 2021.4.10
 
 **Table of contents**
 
@@ -34,6 +34,7 @@ Release: 2021.3.3
     - [Hooks](#hooks)
         - [Moved hooks](#moved-hooks)
 - [Releases](#releases)
+    - [Release 2021.4.10](#release-2021410)
     - [Release 2021.3.3](#release-202133)
     - [Release 2021.2.5](#release-202125)
     - [Release 2020.11.23](#release-20201123)
@@ -107,6 +108,14 @@ in [Naming conventions for provider packages](https://github.com/apache/airflow/
 
 ## Releases
 
+### Release 2021.4.10
+
+| Commit                                                                                         | Committed   | Subject                                                   |
+|:-----------------------------------------------------------------------------------------------|:------------|:----------------------------------------------------------|
+| [68e4c4dcb](https://github.com/apache/airflow/commit/68e4c4dcb0416eb51a7011a3bb040f1e23d7bba8) | 2021-03-20  | `Remove Backport Providers (#14886)`                      |
+| [589d6dec9](https://github.com/apache/airflow/commit/589d6dec922565897785bcbc5ac6bb3b973d7f5d) | 2021-02-27  | `Prepare to release the next wave of providers: (#14487)` |
+
+
 ### Release 2021.3.3
 
 | Commit                                                                                         | Committed   | Subject                                                               |
diff --git a/airflow/providers/http/__init__.py b/airflow/providers/http/__init__.py
index 217e5db..1c982ed 100644
--- a/airflow/providers/http/__init__.py
+++ b/airflow/providers/http/__init__.py
@@ -15,3 +15,49 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+#
+from typing import Callable, Dict, List, Tuple, Union
+
+
+def determine_kwargs(func: Callable, args: Union[Tuple, List], kwargs: Dict) -> Dict:
+    """
+    Inspect the signature of a given callable to determine which arguments in kwargs need
+    to be passed to the callable.
+    :param func: The callable that you want to invoke
+    :param args: The positional arguments that needs to be passed to the callable, so we
+        know how many to skip.
+    :param kwargs: The keyword arguments that need to be filtered before passing to the callable.
+    :return: A dictionary which contains the keyword arguments that are compatible with the callable.
+    """
+    import inspect
+    import itertools
+
+    signature = inspect.signature(func)
+    has_kwargs = any(p.kind == p.VAR_KEYWORD for p in signature.parameters.values())
+
+    for name in itertools.islice(signature.parameters.keys(), len(args)):
+        # Check if args conflict with names in kwargs
+        if name in kwargs:
+            raise ValueError(f"The key {name} in args is part of kwargs and therefore reserved.")
+
+    if has_kwargs:
+        # If the callable has a **kwargs argument, it's ready to accept all the kwargs.
+        return kwargs
+
+    # If the callable has no **kwargs argument, it only wants the arguments it requested.
+    return {key: kwargs[key] for key in signature.parameters if key in kwargs}
+
+
+def make_kwargs_callable(func: Callable) -> Callable:
+    """
+    Make a new callable that can accept any number of positional or keyword arguments
+    but only forwards those required by the given callable func.
+    """
+    import functools
+
+    @functools.wraps(func)
+    def kwargs_func(*args, **kwargs):
+        kwargs = determine_kwargs(func, args, kwargs)
+        return func(*args, **kwargs)
+
+    return kwargs_func
diff --git a/airflow/providers/http/operators/http.py b/airflow/providers/http/operators/http.py
index 9e028a4..fb7de2d 100644
--- a/airflow/providers/http/operators/http.py
+++ b/airflow/providers/http/operators/http.py
@@ -19,6 +19,7 @@ from typing import Any, Callable, Dict, Optional
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
+from airflow.providers.http import make_kwargs_callable
 from airflow.providers.http.hooks.http import HttpHook
 from airflow.utils.decorators import apply_defaults
 
@@ -99,8 +100,6 @@ class SimpleHttpOperator(BaseOperator):
             raise AirflowException("'xcom_push' was deprecated, use 'BaseOperator.do_xcom_push' instead")
 
     def execute(self, context: Dict[str, Any]) -> Any:
-        from airflow.utils.operator_helpers import make_kwargs_callable
-
         http = HttpHook(self.method, http_conn_id=self.http_conn_id)
 
         self.log.info("Calling HTTP method")
diff --git a/airflow/providers/http/sensors/http.py b/airflow/providers/http/sensors/http.py
index 4ad6c12..e00c2ce 100644
--- a/airflow/providers/http/sensors/http.py
+++ b/airflow/providers/http/sensors/http.py
@@ -18,6 +18,7 @@
 from typing import Any, Callable, Dict, Optional
 
 from airflow.exceptions import AirflowException
+from airflow.providers.http import make_kwargs_callable
 from airflow.providers.http.hooks.http import HttpHook
 from airflow.sensors.base import BaseSensorOperator
 from airflow.utils.decorators import apply_defaults
@@ -97,8 +98,6 @@ class HttpSensor(BaseSensorOperator):
         self.hook = HttpHook(method=method, http_conn_id=http_conn_id)
 
     def poke(self, context: Dict[Any, Any]) -> bool:
-        from airflow.utils.operator_helpers import make_kwargs_callable
-
         self.log.info('Poking: %s', self.endpoint)
         try:
             response = self.hook.run(
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index d08d674..1d4ac07 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -157,7 +157,7 @@ function initialization::initialize_base_variables() {
 function initialization::initialize_branch_variables() {
     # Default branch used - this will be different in different branches
     export DEFAULT_BRANCH=${DEFAULT_BRANCH="master"}
-    export DEFAULT_CONSTRAINTS_BRANCH=${DEFAULT_CONSTRAINTS_BRANCH="constraints-master"}
+    export DEFAULT_CONSTRAINTS_BRANCH=${DEFAULT_CONSTRAINTS_BRANCH="constraints-2.0.0"}
     readonly DEFAULT_BRANCH
     readonly DEFAULT_CONSTRAINTS_BRANCH
 
diff --git a/setup.py b/setup.py
index 332d43c..7d2180f 100644
--- a/setup.py
+++ b/setup.py
@@ -298,7 +298,7 @@ google = [
     'google-cloud-language>=1.1.1,<2.0.0',
     'google-cloud-logging>=2.1.1,<3.0.0',
     'google-cloud-memcache>=0.2.0',
-    'google-cloud-monitoring>=2.0.0,<3.0.0',
+    'google-cloud-monitoring>=2.0.0,<2.1.0',
     'google-cloud-os-login>=2.0.0,<3.0.0',
     'google-cloud-pubsub>=2.0.0,<3.0.0',
     'google-cloud-redis>=2.0.0,<3.0.0',