You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by jh...@apache.org on 2021/05/07 02:39:26 UTC

[airflow] 01/02: Add API to query for providers (#12468)

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

jhtimmins pushed a commit to branch pr/13394
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit acc96844340d2897bcecddd507aef1f445bb7ab4
Author: Nicola Gemo <ni...@hotmail.it>
AuthorDate: Mon Dec 28 20:01:39 2020 +0100

    Add API to query for providers (#12468)
---
 .../api_connexion/endpoints/provider_endpoint.py   |  47 ++++++++
 airflow/api_connexion/openapi/v1.yaml              |  45 ++++++++
 airflow/api_connexion/schemas/provider_schema.py   |  46 ++++++++
 airflow/security/permissions.py                    |   1 +
 airflow/www/security.py                            |   1 +
 docs/apache-airflow/security/access-control.rst    |   1 +
 .../endpoints/test_provider_endpoint.py            | 127 +++++++++++++++++++++
 7 files changed, 268 insertions(+)

diff --git a/airflow/api_connexion/endpoints/provider_endpoint.py b/airflow/api_connexion/endpoints/provider_endpoint.py
new file mode 100644
index 0000000..844dcd3
--- /dev/null
+++ b/airflow/api_connexion/endpoints/provider_endpoint.py
@@ -0,0 +1,47 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import re
+from typing import Dict, List
+
+from airflow.api_connexion import security
+from airflow.api_connexion.schemas.provider_schema import ProviderCollection, provider_collection_schema
+from airflow.providers_manager import ProviderInfo, ProvidersManager
+from airflow.security import permissions
+
+
+def _remove_rst_syntax(value: str) -> str:
+    return re.sub("[`_<>]", "", value.strip(" \n."))
+
+
+def _provider_mapper(provider: ProviderInfo) -> Dict:
+    return {
+        "package_name": provider[1]["package-name"],
+        "description": _remove_rst_syntax(provider[1]["description"]),
+        "version": provider[0],
+    }
+
+
+@security.requires_access([(permissions.ACTION_CAN_READ, permissions.RESOURCE_PROVIDER)])
+def get_providers():
+    """Get providers"""
+    providers_info: List[ProviderInfo] = list(ProvidersManager().providers.values())
+    providers = [_provider_mapper(d) for d in providers_info]
+    total_entries = len(providers)
+    return provider_collection_schema.dump(
+        ProviderCollection(providers=providers, total_entries=total_entries)
+    )
diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml
index 2fd58ea..720411c 100644
--- a/airflow/api_connexion/openapi/v1.yaml
+++ b/airflow/api_connexion/openapi/v1.yaml
@@ -133,6 +133,7 @@ info:
     |-|-|
     | v2.0 | Initial release |
     | v2.0.2    | Added /plugins endpoint |
+    | v2.1 | New providers endpoint |
 
     # Trying the API
 
@@ -849,6 +850,26 @@ paths:
         '404':
           $ref: '#/components/responses/NotFound'
 
+  /providers:
+    get:
+      summary: List providers
+      x-openapi-router-controller: airflow.api_connexion.endpoints.provider_endpoint
+      operationId: get_providers
+      tags: [Provider]
+      responses:
+        '200':
+          description: List of providers.
+          content:
+            application/json:
+              schema:
+                allOf:
+                  - $ref: '#/components/schemas/ProviderCollection'
+                  - $ref: '#/components/schemas/CollectionInfo'
+        '401':
+          $ref: '#/components/responses/Unauthenticated'
+        '403':
+          $ref: '#/components/responses/PermissionDenied'
+
   /dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances:
     parameters:
       - $ref: '#/components/parameters/DAGID'
@@ -2016,6 +2037,29 @@ components:
         - $ref: '#/components/schemas/CollectionInfo'
 
 
+    Provider:
+      description: The provider
+      type: object
+      properties:
+        package_name:
+          type: string
+          description: The package name of the provider.
+        description:
+          type: string
+          description: The description of the provider.
+        version:
+          type: string
+          description: The version of the provider.
+
+    ProviderCollection:
+      type: object
+      properties:
+        providers:
+          type: array
+          items:
+            $ref: '#/components/schemas/Provider'
+
+
     SLAMiss:
       type: object
       properties:
@@ -3390,6 +3434,7 @@ tags:
   - name: ImportError
   - name: Monitoring
   - name: Pool
+  - name: Provider
   - name: TaskInstance
   - name: Variable
   - name: XCom
diff --git a/airflow/api_connexion/schemas/provider_schema.py b/airflow/api_connexion/schemas/provider_schema.py
new file mode 100644
index 0000000..df0ddd5
--- /dev/null
+++ b/airflow/api_connexion/schemas/provider_schema.py
@@ -0,0 +1,46 @@
+# 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 typing import List, NamedTuple
+
+from marshmallow import Schema, fields
+
+
+class ProviderSchema(Schema):
+    """Provider schema"""
+
+    package_name = fields.String(required=True)
+    description = fields.String(required=True)
+    version = fields.String(required=True)
+
+
+class ProviderCollection(NamedTuple):
+    """List of Providers"""
+
+    providers: List[ProviderSchema]
+    total_entries: int
+
+
+class ProviderCollectionSchema(Schema):
+    """Provider Collection schema"""
+
+    providers = fields.List(fields.Nested(ProviderSchema))
+    total_entries = fields.Int()
+
+
+provider_collection_schema = ProviderCollectionSchema()
+provider_schema = ProviderSchema()
diff --git a/airflow/security/permissions.py b/airflow/security/permissions.py
index f4337eb..115f25a 100644
--- a/airflow/security/permissions.py
+++ b/airflow/security/permissions.py
@@ -38,6 +38,7 @@ RESOURCE_PERMISSION = "Permissions"
 RESOURCE_PERMISSION_VIEW = "Permission Views"  # Refers to a Perm <-> View mapping, not an MVC View.
 RESOURCE_POOL = "Pools"
 RESOURCE_PLUGIN = "Plugins"
+RESOURCE_PROVIDER = "Providers"
 RESOURCE_ROLE = "Roles"
 RESOURCE_SLA_MISS = "SLA Misses"
 RESOURCE_TASK_INSTANCE = "Task Instances"
diff --git a/airflow/www/security.py b/airflow/www/security.py
index d1bf9bc..0cb7b40 100644
--- a/airflow/www/security.py
+++ b/airflow/www/security.py
@@ -124,6 +124,7 @@ class AirflowSecurityManager(SecurityManager, LoggingMixin):  # pylint: disable=
         (permissions.ACTION_CAN_READ, permissions.RESOURCE_POOL),
         (permissions.ACTION_CAN_EDIT, permissions.RESOURCE_POOL),
         (permissions.ACTION_CAN_DELETE, permissions.RESOURCE_POOL),
+        (permissions.ACTION_CAN_READ, permissions.RESOURCE_PROVIDER),
         (permissions.ACTION_CAN_CREATE, permissions.RESOURCE_VARIABLE),
         (permissions.ACTION_CAN_READ, permissions.RESOURCE_VARIABLE),
         (permissions.ACTION_CAN_EDIT, permissions.RESOURCE_VARIABLE),
diff --git a/docs/apache-airflow/security/access-control.rst b/docs/apache-airflow/security/access-control.rst
index 38f537e..2615725 100644
--- a/docs/apache-airflow/security/access-control.rst
+++ b/docs/apache-airflow/security/access-control.rst
@@ -168,6 +168,7 @@ Endpoint
 /pools/{pool_name}                                                                 DELETE Pool.can_delete                                                   Op
 /pools/{pool_name}                                                                 GET    Pool.can_read                                                     Op
 /pools/{pool_name}                                                                 PATCH  Pool.can_edit                                                     Op
+/providers                                                                         GET    Provider.can_read                                                 Op
 /dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances                                  GET    DAGs.can_read, DAG Runs.can_read, Task Instances.can_read         Viewer
 /dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances/{task_id}                        GET    DAGs.can_read, DAG Runs.can_read, Task Instances.can_read         Viewer
 /dags/{dag_id}/dagRuns/{dag_run_id}/taskInstances/{task_id}/links                  GET    DAGs.can_read, DAG Runs.can_read, Task Instances.can_read         Viewer
diff --git a/tests/api_connexion/endpoints/test_provider_endpoint.py b/tests/api_connexion/endpoints/test_provider_endpoint.py
new file mode 100644
index 0000000..25c2618
--- /dev/null
+++ b/tests/api_connexion/endpoints/test_provider_endpoint.py
@@ -0,0 +1,127 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+from collections import OrderedDict
+from unittest import mock
+
+from airflow.security import permissions
+from airflow.www import app
+from tests.test_utils.api_connexion_utils import create_user, delete_user
+from tests.test_utils.config import conf_vars
+
+MOCK_PROVIDERS = OrderedDict(
+    [
+        (
+            'apache-airflow-providers-amazon',
+            (
+                '1.0.0',
+                {
+                    'package-name': 'apache-airflow-providers-amazon',
+                    'name': 'Amazon',
+                    'description': '`Amazon Web Services (AWS) <https://aws.amazon.com/>`__.\n',
+                    'versions': ['1.0.0'],
+                },
+            ),
+        ),
+        (
+            'apache-airflow-providers-apache-cassandra',
+            (
+                '1.0.0',
+                {
+                    'package-name': 'apache-airflow-providers-apache-cassandra',
+                    'name': 'Apache Cassandra',
+                    'description': '`Apache Cassandra <http://cassandra.apache.org/>`__.\n',
+                    'versions': ['1.0.0'],
+                },
+            ),
+        ),
+    ]
+)
+
+
+class TestBaseProviderEndpoint(unittest.TestCase):
+    @classmethod
+    def setUpClass(cls) -> None:
+        with conf_vars({("api", "auth_backend"): "tests.test_utils.remote_user_api_auth_backend"}):
+            cls.app = app.create_app(testing=True)  # type:ignore
+        create_user(
+            cls.app,  # type:ignore
+            username="test",
+            role_name="Test",
+            permissions=[(permissions.ACTION_CAN_READ, permissions.RESOURCE_PROVIDER)],  # type: ignore
+        )
+        create_user(cls.app, username="test_no_permissions", role_name="TestNoPermissions")  # type: ignore
+
+        cls.client = None
+
+    @classmethod
+    def tearDownClass(cls) -> None:
+        delete_user(cls.app, username="test")  # type: ignore
+        delete_user(cls.app, username="test_no_permissions")  # type: ignore
+
+    def setUp(self) -> None:
+        self.client = self.app.test_client()  # type:ignore
+
+
+class TestGetProviders(TestBaseProviderEndpoint):
+    @mock.patch(
+        "airflow.providers_manager.ProvidersManager.providers",
+        new_callable=mock.PropertyMock,
+        return_value={},
+    )
+    def test_response_200_empty_list(self, mock_providers):
+        response = self.client.get("/api/v1/providers", environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 200
+        self.assertEqual({"providers": [], "total_entries": 0}, response.json)
+
+    @mock.patch(
+        "airflow.providers_manager.ProvidersManager.providers",
+        new_callable=mock.PropertyMock,
+        return_value=MOCK_PROVIDERS,
+    )
+    def test_response_200(self, mock_providers):
+        response = self.client.get("/api/v1/providers", environ_overrides={'REMOTE_USER': "test"})
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                'providers': [
+                    {
+                        'description': 'Amazon Web Services (AWS) https://aws.amazon.com/',
+                        'package_name': 'apache-airflow-providers-amazon',
+                        'version': '1.0.0',
+                    },
+                    {
+                        'description': 'Apache Cassandra http://cassandra.apache.org/',
+                        'package_name': 'apache-airflow-providers-apache-cassandra',
+                        'version': '1.0.0',
+                    },
+                ],
+                'total_entries': 2,
+            },
+            response.json,
+        )
+
+    def test_should_raises_401_unauthenticated(self):
+        response = self.client.get("/api/v1/providers")
+        assert response.status_code == 401
+
+    def test_should_raise_403_forbidden(self):
+        response = self.client.get(
+            "/api/v1/providers", environ_overrides={'REMOTE_USER': "test_no_permissions"}
+        )
+        assert response.status_code == 403