You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/06/13 07:10:59 UTC

[GitHub] [airflow] houqp opened a new pull request #9273: implement api v1 for variables

houqp opened a new pull request #9273:
URL: https://github.com/apache/airflow/pull/9273


   close #9115 and #8133 .
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Target Github ISSUE in description if exists
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+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 [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r441892187



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,142 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var2"},
+                {"key": "var3"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=1&offset=2", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,
+        }),
+    ])
+    def test_should_get_list_variables(self, query, expected):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get(query)
+        assert response.status_code == 200
+        assert response.json == expected
+
+    def test_should_honor_100_limit_default(self):
+        for i in range(101):
+            Variable.set(f"var{i}", i)
         response = self.client.get("/api/v1/variables")
         assert response.status_code == 200
+        assert response.json["total_entries"] == 100
+        assert len(response.json["variables"]) == 100
 
 
 class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
-        assert response.status_code == 200
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {

Review comment:
       That's strange, assert always give me nested diff on errors:
   
   ```
           assert response.status_code == 400
   >       assert response.json == {
               "title": "Invalid post body",
               "status": 401,
               "type": "about:blank",
               "detail": "key from request body doesn't match uri parameter",
           }
   E       AssertionError: assert {'detail': "key from request body doesn't match uri parameter",\n 'status': 400,\n 'title': 'Invalid post body',\n 'type': 'about:blank'} == {'detail': "key from request body doesn't match uri parameter",\n 'status': 401,\n 'title': 'Invalid post body',\n 'type': 'about:blank'}
   E         Common items:
   E         {'detail': "key from request body doesn't match uri parameter",
   E          'title': 'Invalid post body',
   E          'type': 'about:blank'}
   E         Differing items:
   E         {'status': 400} != {'status': 401}
   E         Full diff:
   E           {
   E            'detail': "key from request body doesn't match uri parameter",
   E         -  'status': 401,
   E         ?              ^
   E         +  'status': 400,
   E         ?              ^
   E            'title': 'Invalid post body',
   E            'type': 'about:blank',
   E           }
   
   tests/api_connexion/endpoints/test_variable_endpoint.py:133: AssertionError
   ```
   
   Do you mind sharing how you ran tests? I usually just invoke `py.test  tests/api_connexion/endpoints/test_variable_endpoint.py` to run tests.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439760024



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       in openapi spec, detail is defined as a string, if we just return err.messages here, it will be serialized as an object.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439738186



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:

Review comment:
       ```suggestion
   def get_variables(session) -> Response:
   ```
   limit and offset are query parameters. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439773758



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,

Review comment:
       ```suggestion
               "variables": [
                   {"key": "var2"},
                   {"key": "var3"}
               ],
               "total_entries": 2,
   ```
   The total data is 3, the limit is 2 and offset is 1. What do you think?
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439759820



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())

Review comment:
       `marshmallow` calls json decode in `loads`. I was expecting it to throw a `ValidationError` error for invalid json payload, but looks like that's not the case. I have changed it back to `load(request.json)` 
   
   for `UpdateMask`, it doesn't make much sense for variable endpoints because there is only one field (`value`) to update. I can't think of a use-case where a user would want to mask `value` field when updating a variable. I think we should just remove this param from openapi spec, what do you think @mik-laj @ephraimbuddy ?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439763873



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()

Review comment:
       Thanks for pointing this out +1




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439737773



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 

Review comment:
       ```suggestion
   from connexion import NoContent
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r440946856



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,133 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")
+        print(response.data)
         assert response.status_code == 204
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
 
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.get("/api/v1/variables")
+    def test_should_get_list_variables(self):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get("/api/v1/variables?limit=2&offset=0")
         assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var1", "value": "1"},
+                {"key": "var2", "value": "foo"},
+            ],
+            "total_entries": 2,
+        }
 
+        response = self.client.get("/api/v1/variables?limit=2&offset=1")
+        assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var3", "value": "[100, 101]"},
+            ],
+            "total_entries": 1,
+        }
 
-class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
+        response = self.client.get("/api/v1/variables?limit=1&offset=2")
         assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var3", "value": "[100, 101]"},
+            ],
+            "total_entries": 1,
+        }
+
+
+class TestPatchVariable(TestVariableEndpoint):
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {
+            "key": "var1",
+            "value": "updated",
+        }
+
+    def test_should_reject_invalid_update(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var2",
+            "value": "updated",
+        })
+        assert response.status_code == 400
+        assert response.json == {
+            "title": "Invalid post body",
+            "status": 400,
+            "type": "about:blank",
+            "detail": "key from request body doesn't match uri parameter",
+        }
+
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var2",
+        })
+        assert response.json == {
+            "title": "Invalid Variable schema",
+            "status": 400,
+            "type": "about:blank",

Review comment:
       I do not have a specific plan yet, but I will try to prepare a reference documentation that will be generated in Swagger UI or Redoc and add links where possible. 
   Here is issue about docs: https://github.com/apache/airflow/issues/8143




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp removed a comment on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp removed a comment on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-645722927


   @mik-laj updated and ready for another round of review


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439723137



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,133 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")
+        print(response.data)
         assert response.status_code == 204
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
 
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.get("/api/v1/variables")
+    def test_should_get_list_variables(self):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get("/api/v1/variables?limit=2&offset=0")
         assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var1", "value": "1"},
+                {"key": "var2", "value": "foo"},
+            ],
+            "total_entries": 2,
+        }
 
+        response = self.client.get("/api/v1/variables?limit=2&offset=1")
+        assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var3", "value": "[100, 101]"},
+            ],
+            "total_entries": 1,
+        }
 
-class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
+        response = self.client.get("/api/v1/variables?limit=1&offset=2")
         assert response.status_code == 200
+        assert response.json == {
+            "variables": [
+                {"key": "var3", "value": "[100, 101]"},
+            ],
+            "total_entries": 1,
+        }
+
+
+class TestPatchVariable(TestVariableEndpoint):
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {
+            "key": "var1",
+            "value": "updated",
+        }
+
+    def test_should_reject_invalid_update(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var2",
+            "value": "updated",
+        })
+        assert response.status_code == 400
+        assert response.json == {
+            "title": "Invalid post body",
+            "status": 400,
+            "type": "about:blank",
+            "detail": "key from request body doesn't match uri parameter",
+        }
+
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var2",
+        })
+        assert response.json == {
+            "title": "Invalid Variable schema",
+            "status": 400,
+            "type": "about:blank",

Review comment:
       @mik-laj this field requires a uri for a human readable documentation, how do you envision this to be supported in airflow?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739274



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))
+
+    if var.data["key"] != variable_key:
+        raise BadRequest("Invalid post body", detail="key from request body doesn't match uri parameter")
+
+    Variable.set(var.data["key"], var.data["val"])
+    return make_response('', 204)
 
 
 def post_variables():
     """
     Create a variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())

Review comment:
       ```suggestion
           var = variable_schema.load(request.json)
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643675092


   @OmairK the api tests should have already covered serialization and deserialization code path, no? Do you have an example in mind where a refactor will cause issue that won't be caught by the api tests but can be caught by testing schema module itself?


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

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



[GitHub] [airflow] mik-laj merged pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj merged pull request #9273:
URL: https://github.com/apache/airflow/pull/9273


   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439758939



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()

Review comment:
       we should just the query param passed in by connexion through the keyword arguments here. There is no need to parse the query param again manually. connexion also takes care of injecting the 100 default value from from openapi spec for you.
   
   as for offset handling, the offset should be calculated using `offset * limit`. The connection implementation uses `query.offset(offset).limit(limit)`, which is a bug. You can port the `test_should_get_list_variables` test to connection test suite and checkout the test error.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r441916241



##########
File path: airflow/www/extensions/init_views.py
##########
@@ -92,15 +93,16 @@ def init_error_handlers(app: Flask):
     app.register_error_handler(404, views.circles)
 
 
-def init_api_connexion(app: Flask):
+def init_api_connexion(app: Flask, csrf: CSRFProtect) -> None:
     """Initialize Stable API"""
     spec_dir = path.join(ROOT_APP_DIR, 'api_connexion', 'openapi')
     connexion_app = connexion.App(__name__, specification_dir=spec_dir, skip_error_handlers=True)
     connexion_app.app = app
-    connexion_app.add_api(
+    api_bp = connexion_app.add_api(
         specification='v1.yaml', base_path='/api/v1', validate_responses=True, strict_validation=False
-    )
+    ).blueprint
     app.register_error_handler(ProblemException, connexion_app.common_error_handler)
+    csrf.exempt(api_bp)

Review comment:
       ```suggestion
       app.extensions['csrf'].exempt(api_bp)
   ```
   Here is example: 
   https://github.com/apache/airflow/blob/master/airflow/www/extensions/init_views.py#L118




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439781508



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,

Review comment:
       limit is 2 means page size is 2, if we are at second page (offset=1), shouldn't it be just return the last variable?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740375



##########
File path: airflow/api_connexion/schemas/variable_schema.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 marshmallow import Schema, fields
+
+
+class VariableSchema(Schema):
+    """ Variable Schema """
+    key = fields.String(required=True)
+    value = fields.String(attribute="val", required=True)
+
+
+class VariableCollectionSchema(Schema):
+    """ Variable Collection Schema """
+    variables = fields.List(fields.Nested(VariableSchema))
+    total_entries = fields.Int()
+

Review comment:
       I would do this schema as follow:
   ```
   class VariableCollectionItemSchema(SQLAlchemySchema):
       """ A schema for variable collection item """
       class Meta:
           """ Meta """
           model = Variable
       key = auto_field(required=True)
   
   
   class VariableSchema(VariableCollectionItemSchema):
       """ Variable Schema """
       val = auto_field(dump_to='value', load_from='value', required=True)
   
   class VariableCollectionSchema(Schema):
       """ Variable Collection Schema """
       variables = fields.List(fields.Nested(VariableCollectionItemSchema))
       total_entries = fields.Int()
   
   ```
   The spec has this to say about VariableCollectionItemSchema : `Divided into two schemas for sensitive data protection`




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439721698



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,77 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(offset: Optional[int], limit: Optional[int], session) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:

Review comment:
       Turns out this is already handled by `connexion` because default value for limit is defined in api spec. I have added test to verify that.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740375



##########
File path: airflow/api_connexion/schemas/variable_schema.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 marshmallow import Schema, fields
+
+
+class VariableSchema(Schema):
+    """ Variable Schema """
+    key = fields.String(required=True)
+    value = fields.String(attribute="val", required=True)
+
+
+class VariableCollectionSchema(Schema):
+    """ Variable Collection Schema """
+    variables = fields.List(fields.Nested(VariableSchema))
+    total_entries = fields.Int()
+

Review comment:
       I would do this schema as follow:
   ```
   class VariableCollectionItemSchema(SQLAlchemySchema):
       """ A schema for variable collection item """
       class Meta:
           """ Meta """
           model = Variable
       key = auto_field(required=True)
   
   
   class VariableSchema(VariableCollectionItemSchema):
       """ Variable Schema """
       val = auto_field(dump_to='value', load_from='value', required=True)
   
   class VariableCollectionSchema(Schema):
       """ Variable Collection Schema """
       variables = fields.List(fields.Nested(VariableSchema))
       total_entries = fields.Int()
   
   ```
   The spec has this to say about VariableCollectionItemSchema : `Divided into two schemas for sensitive data protection




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643805306


   > I think, writing tests improve code coverage?
   
   @ephraimbuddy No not necessarily, my understanding of code coverage is based on [this](https://coverage.readthedocs.io/en/v4.5.x/howitworks.html). IMO it wouldn't matter much in this case, both serialization and de-serialization is already covered in the CRUD end points.
   


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439758475



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:

Review comment:
       query params are passed in as keyword arguments by connexion, see: https://github.com/zalando/connexion/blob/a250e5a59f32ce9d52754f5f7c1827f9abcf0eaf/connexion/decorators/parameter.py#L102




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740498



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()

Review comment:
       ```suggestion
           clear_db_connections()
   ```
   Any need for this?




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

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



[GitHub] [airflow] mik-laj commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643587896


   > I think the key in request body is redundant, should we update the api spec to remove it?
   
   The structure of this request body results from the fact that it is based on the Variable schema.
   ```yaml
       VariableCollectionItem:
         # Divided into two schemas for sensitive data protection
         type: object
         properties:
           key:
             type: string
   
       VariableCollection:
         type: object
         properties:
           variables:
             type: array
             items:
               $ref: '#/components/schemas/VariableCollectionItem'
   
       Variable:
         allOf:
           - $ref: '#/components/schemas/VariableCollectionItem'
           - type: object
             properties:
               value:
                 type: string
   ```
   The same schema is used to create as well as edit elements, so both fields must exist. We can create a new scheem that will not contain this one field, but it will complicate the API.
   
   This field should be immutable, which is well described in the AIP.
   
   > When a service receives an immutable field in an update request (or similar), even if included in the update mask, the service should ignore the field if the value matches, but should error with INVALID_ARGUMENT if a change is requested.
   
   https://google.aip.dev/203
   
   Unfortunately, OpenAPI has no support for immutable fields, so we have some confusion in the specification definition. I think we can add an additional description to explain this.
   
   What do you think about it?


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

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



[GitHub] [airflow] houqp commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643811896


   > I think, writing tests improve code coverage?
   
   As @OmairK already mentioned, duplicated tests that test the same code path won't increase code coverage. In fact it will make future refactoring harder because you will have to change more than one tests whenever you introduce a behavior change.
   
   If a code path can be covered by end to end api test, that should be preferred. It's better to test on expected behavior of the system instead of implementation details.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439757664



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)

Review comment:
       The method has return type annotated as `Response`, so it's better to return a Response object for type safety. The NoContent object in connexion is an unnecessary abstraction IMO, it's just defined as `object()`, which has the same effect as `''` when used in the tuple as response pattern. But it will result in an exception when passed to `make_response` function. That's why in Flask it's more common to return `''`.
   
   I have simplified this code further to just return `Response(status=204)` to avoid calling `make_response`.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r440949309



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)

Review comment:
       I agree. That would be helpful.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439760054



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))
+
+    if var.data["key"] != variable_key:
+        raise BadRequest("Invalid post body", detail="key from request body doesn't match uri parameter")
+
+    Variable.set(var.data["key"], var.data["val"])
+    return make_response('', 204)
 
 
 def post_variables():
     """
     Create a variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       see my previous comment with regards to openapi spec.

##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -16,9 +16,10 @@
 # under the License.
 import unittest
 
-import pytest
-
+from airflow.models import Variable
+from airflow.utils.session import create_session
 from airflow.www import app
+from tests.test_utils.db import clear_db_connections

Review comment:
       good catch, fixed




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439782251



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,

Review comment:
       I see where the confusion is coming from. In the API spec, the parameter is named as `PageOffset`, which sounds like offset is the page number, not the absolute element count. But the description is set to `The number of items to skip before starting to collect the result set`.
   
   I will change it to match the description in API spec.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740498



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()

Review comment:
       ```suggestion
           clear_db_connections()
   ```
   Any need for this?




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

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r440965043



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,142 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var2"},
+                {"key": "var3"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=1&offset=2", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,
+        }),
+    ])
+    def test_should_get_list_variables(self, query, expected):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get(query)
+        assert response.status_code == 200
+        assert response.json == expected
+
+    def test_should_honor_100_limit_default(self):
+        for i in range(101):
+            Variable.set(f"var{i}", i)
         response = self.client.get("/api/v1/variables")
         assert response.status_code == 200
+        assert response.json["total_entries"] == 100
+        assert len(response.json["variables"]) == 100
 
 
 class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
-        assert response.status_code == 200
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {

Review comment:
       Using self.assertEquals is helpful because it provides clearer error messages, so it's easier to find an error, but I don't require it as per the mailing list voting. I only inform.
   assert statement
   ![image (1)](https://user-images.githubusercontent.com/12058428/84798264-04c6a200-affb-11ea-8212-2c0dc0470a03.png)
   self.assertEquals
   ![image (2)](https://user-images.githubusercontent.com/12058428/84798293-10b26400-affb-11ea-8182-f4b52a354d47.png)
   ![image (3)](https://user-images.githubusercontent.com/12058428/84798300-13ad5480-affb-11ea-9256-ec370f16f0ee.png)
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643803609


   > @OmairK the api tests should have already covered serialization and deserialization code path, no? Do you have an example in mind where a refactor will cause issue that won't be caught by the api tests but can be caught by testing schema module itself?
   
   Yes, I think you are right :)  Initially we were dealing with only read endpoints so we had to explicitly test the de-serialization, which you are already covering in create/update endpoints. 


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439766984



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       It will get serialized as a json object. You can verify this yourself by checking out my branch and remove the `str` call to see how the test fails.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439781865



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,

Review comment:
       The way I understand this is that the total number of pages it should return is 2. If we have [1,2,3,4,5] items, it should return only 2 items. Now if we offset by 1, then it starts counting from 2 above and return maximum of 2 items. This is how I think I understand it. Would appreciate if you can explain with the above list. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439717133



##########
File path: airflow/www/app.py
##########
@@ -249,13 +249,14 @@ def init_api_connexion(app: Flask):
             spec_dir = path.join(ROOT_APP_DIR, 'api_connexion', 'openapi')
             connexion_app = connexion.App(__name__, specification_dir=spec_dir, skip_error_handlers=True)
             connexion_app.app = app
-            connexion_app.add_api(
+            api_bp = connexion_app.add_api(
                 specification='v1.yaml',
                 base_path='/api/v1',
                 validate_responses=True,
                 strict_validation=False
-            )
+            ).blueprint
             app.register_error_handler(ProblemException, connexion_app.common_error_handler)
+            csrf.exempt(api_bp)

Review comment:
       prevent CSRF check for all API write operations.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439764054



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       No. It will not serialize as an object. However, It's ok if you use str




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439741040



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")
+        print(response.data)
         assert response.status_code == 204
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
 
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    def test_should_get_list_variables(self):

Review comment:
       Would be nice to use parameterized for this test. Check here:
   https://github.com/apache/airflow/blob/master/tests/api_connexion/endpoints/test_connection_endpoint.py#L134




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643594204


   > When a service receives an immutable field in an update request (or similar), even if included in the update mask, the service should ignore the field if the value matches, but should error with INVALID_ARGUMENT if a change is requested.
   
   Cool, my implementation already matches this behavior.
   
   > Unfortunately, OpenAPI has no support for immutable fields, so we have some confusion in the specification definition. I think we can add an additional description to explain this.
   
   Yeah, I think it would be helpful to document this in the spec. How would you want this to be documented?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643582917


   @mik-laj the in openapi spec, the PATH method for variable takes the following body:
   
   ```
   PATCH /api/v1/variables/foo
   {"key": "foo", "value": "bar"}
   ```
   
   I think the key in request body is redundant, should we update the api spec to remove it?


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

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439737799



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)

Review comment:
       ```suggestion
       return NoContent, 204
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740647



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")

Review comment:
       Would be cool to add a query that checks that a variable was added to the db




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439773896



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()

Review comment:
       `offset * limit` is not working correctly. I didn't see any issue when I used `query.offset(offset).limit(limit)`, I suggest you add a test that `query.offset(offset).limit(limit)` cannot pass.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739174



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       ```suggestion
           raise BadRequest("Invalid Variable schema", detail=err.messages)
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439719672



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,77 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(offset: Optional[int], limit: Optional[int], session) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:

Review comment:
       I think we should set a default pagination limit. In other endpoints, it is 100 items (We can add the configuration parameter later). Otherwise, the client may mistakenly request a lot of data that he does not need.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643674269


   Nitpicking here, but explicitly testing the out the [schema's](https://github.com/apache/airflow/blob/287932c276dae6bce242051a40cff3680c238626/tests/api_connexion/schemas/test_error_schema.py) serialization and de-serialization could be helpful once we start refactoring the code. @houqp, what do you think ? 


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-643758547


   > @OmairK the api tests should have already covered serialization and deserialization code path, no? Do you have an example in mind where a refactor will cause issue that won't be caught by the api tests but can be caught by testing schema module itself?
   
   I think, writing tests improve code coverage?


----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439716728



##########
File path: airflow/api_connexion/openapi/v1.yaml
##########
@@ -2023,7 +2023,7 @@ components:
         status:
           type: number
           description: The HTTP status code generated by the API server for this occurrence of the problem.
-        details:
+        detail:

Review comment:
       update field name to confirm with RFC spec




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439761744



##########
File path: airflow/api_connexion/schemas/variable_schema.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 marshmallow import Schema, fields
+
+
+class VariableSchema(Schema):
+    """ Variable Schema """
+    key = fields.String(required=True)
+    value = fields.String(attribute="val", required=True)
+
+
+class VariableCollectionSchema(Schema):
+    """ Variable Collection Schema """
+    variables = fields.List(fields.Nested(VariableSchema))
+    total_entries = fields.Int()
+

Review comment:
       good catch




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

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r441918574



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,83 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import List, Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    if Variable.delete(variable_key) == 0:
+        raise NotFound("Variable not found")
+    return Response(status=204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)

Review comment:
       ```suggestion
       query = session.query(Variable).order_by(Variable.id)
   ```
   This will allow continuity of items between pages. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r440955054



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())

Review comment:
       In my opinion the update_mask field should remain. We have only one field for now, but in the future we can add other fields. When we decide to keep this field, we will be able to change the structure of this object without introducing any breaking-change. In addition, this will allow us to maintain the consistency in the API. So I think every PATCH endpoint should have a update mask.
   
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439738004



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)

Review comment:
       I suggest we check the existence of this key and raise NotFound exception if key is not found.
   What do you think?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739395



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()

Review comment:
       ```suggestion
           clear_db_variables()
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439757966



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)

Review comment:
       I initially had this in the implementation, but removed it later after noticing openapi spec did not specify a 404 response for this endpoint. I think it's better to return 404 when the key does not exist. @mik-laj thoughts on adding 404 to the openapi spec?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-644838702


   > Yeah, I think it would be helpful to document this in the spec. How would you want this to be documented?
   
   For now I think it's enough to add a note in the specification. "This field is immutable".  I will add a general description of what the invariant field means at the beginning of the documentation later.


----------------------------------------------------------------
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.

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



[GitHub] [airflow] kaxil commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r442174818



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,142 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var2"},
+                {"key": "var3"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=1&offset=2", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,
+        }),
+    ])
+    def test_should_get_list_variables(self, query, expected):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get(query)
+        assert response.status_code == 200
+        assert response.json == expected
+
+    def test_should_honor_100_limit_default(self):
+        for i in range(101):
+            Variable.set(f"var{i}", i)
         response = self.client.get("/api/v1/variables")
         assert response.status_code == 200
+        assert response.json["total_entries"] == 100
+        assert len(response.json["variables"]) == 100
 
 
 class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
-        assert response.status_code == 200
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {

Review comment:
       @mik-laj maybe check out this: https://stackoverflow.com/questions/50608443/how-do-i-get-pycharm-to-show-entire-error-diffs-from-pytest




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439781865



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,

Review comment:
       The way I understand this is that the total number of pages it should return is 2. If we have [1,2,3,4,5] items, it should return only 2 items. Now if we offset by 1, then it starts counting from 2 above and return maximum of 2 items. This is how I understand it. Would appreciate if you can explain with the above list. 




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739218



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))
+
+    if var.data["key"] != variable_key:
+        raise BadRequest("Invalid post body", detail="key from request body doesn't match uri parameter")
+
+    Variable.set(var.data["key"], var.data["val"])
+    return make_response('', 204)

Review comment:
       ```suggestion
       return NoContent, 204
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r441915601



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,142 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var2"},
+                {"key": "var3"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=1&offset=2", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,
+        }),
+    ])
+    def test_should_get_list_variables(self, query, expected):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get(query)
+        assert response.status_code == 200
+        assert response.json == expected
+
+    def test_should_honor_100_limit_default(self):
+        for i in range(101):
+            Variable.set(f"var{i}", i)
         response = self.client.get("/api/v1/variables")
         assert response.status_code == 200
+        assert response.json["total_entries"] == 100
+        assert len(response.json["variables"]) == 100
 
 
 class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
-        assert response.status_code == 200
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {

Review comment:
       I use green button in Intelij IDEA.
   <img width="316" alt="Screenshot 2020-06-18 at 03 16 39" src="https://user-images.githubusercontent.com/12058428/84966589-31211200-b112-11ea-99e1-e2bd7d57523a.png">
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439760587



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")
+        print(response.data)

Review comment:
       good catch, that line should have been deleted.
   
   Thanks for the `pdb` suggestion. I think this is more of a personal preference :) I found print to be sufficient for 99% of my day to day debugging need. It is available in all languages and is more efficient to work with compared to using a debugger. The only few times when I had to use a debugger was for debugging kernel concurrency bugs when adding print statement will incur overhead on context switches and caused the concurrency bug to disappear ;)
   Obviously my preference on debugger is heavily influenced by Linus: https://lkml.org/lkml/2000/9/6/65.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739463



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -16,9 +16,10 @@
 # under the License.
 import unittest
 
-import pytest
-
+from airflow.models import Variable
+from airflow.utils.session import create_session
 from airflow.www import app
+from tests.test_utils.db import clear_db_connections

Review comment:
       ```suggestion
   from tests.test_utils.db import clear_db_variables
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] OmairK commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
OmairK commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739395



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()

Review comment:
       ```suggestion
           clear_db_variables()
   ```
   I think you meant to clear variables.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439773896



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()

Review comment:
       `offset * limit` is not working correctly. I didn't see any issue when I used `query.offset(offset).limit(limit)`, I suggest you add a test that `query.offset(offset).limit(limit)` cannot pass. What do you suggest?




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439738738



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()

Review comment:
       We need to get limit and offset from `request.args`
   An example:
   ```
   offset = request.args.get(parameters.page_offset, 0)
   limit = min(int(request.args.get(parameters.page_limit, 100)), 100)
   ```
   The parameters module should have page_offset and page_limit variables defined as  equal to 'offset' and 'limit' respectively as to be consistent with what has already been merged.
   The query should for limit and offset should be like this:
   ```
   variables = query.offset(offset).limit(limit)
   ```
   




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740797



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +30,141 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        with create_session() as session:
+            session.query(Variable).delete()
+
+    def tearDown(self) -> None:
+        clear_db_connections()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        response = self.client.delete("/api/v1/variables/delete_var1")
+        print(response.data)

Review comment:
       ```suggestion
           
   ```
   You might want to use pdb module when debuging. On that particular line, you can say:
   `import pdb; pdb.set_trace()`.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r442381705



##########
File path: tests/api_connexion/endpoints/test_variable_endpoint.py
##########
@@ -29,38 +31,142 @@ def setUpClass(cls) -> None:
 
     def setUp(self) -> None:
         self.client = self.app.test_client()  # type:ignore
+        clear_db_variables()
+
+    def tearDown(self) -> None:
+        clear_db_variables()
 
 
 class TestDeleteVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.delete("/api/v1/variables/TEST_VARIABLE_KEY")
+    def test_should_response_204(self):
+        Variable.set("delete_var1", 1)
+        # make sure variable is added
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 200
+
+        response = self.client.delete("/api/v1/variables/delete_var1")
         assert response.status_code == 204
 
+        # make sure variable is deleted
+        response = self.client.get("/api/v1/variables/delete_var1")
+        assert response.status_code == 404
+
 
 class TestGetVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
+
     def test_should_response_200(self):
+        expected_value = '{"foo": 1}'
+        Variable.set("TEST_VARIABLE_KEY", expected_value)
         response = self.client.get("/api/v1/variables/TEST_VARIABLE_KEY")
         assert response.status_code == 200
+        assert response.json == {"key": "TEST_VARIABLE_KEY", "value": expected_value}
+
+    def test_should_response_404_if_not_found(self):
+        response = self.client.get("/api/v1/variables/NONEXIST_VARIABLE_KEY")
+        assert response.status_code == 404
 
 
 class TestGetVariables(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
+    @parameterized.expand([
+        ("/api/v1/variables?limit=2&offset=0", {
+            "variables": [
+                {"key": "var1"},
+                {"key": "var2"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=2&offset=1", {
+            "variables": [
+                {"key": "var2"},
+                {"key": "var3"},
+            ],
+            "total_entries": 2,
+        }),
+        ("/api/v1/variables?limit=1&offset=2", {
+            "variables": [
+                {"key": "var3"},
+            ],
+            "total_entries": 1,
+        }),
+    ])
+    def test_should_get_list_variables(self, query, expected):
+        Variable.set("var1", 1)
+        Variable.set("var2", "foo")
+        Variable.set("var3", "[100, 101]")
+        response = self.client.get(query)
+        assert response.status_code == 200
+        assert response.json == expected
+
+    def test_should_honor_100_limit_default(self):
+        for i in range(101):
+            Variable.set(f"var{i}", i)
         response = self.client.get("/api/v1/variables")
         assert response.status_code == 200
+        assert response.json["total_entries"] == 100
+        assert len(response.json["variables"]) == 100
 
 
 class TestPatchVariable(TestVariableEndpoint):
-    @pytest.mark.skip(reason="Not implemented yet")
-    def test_should_response_200(self):
-        response = self.client.patch("/api/v1/variables/TEST_VARIABLE_KEY")
-        assert response.status_code == 200
+    def test_should_update_variable(self):
+        Variable.set("var1", "foo")
+        response = self.client.patch("/api/v1/variables/var1", json={
+            "key": "var1",
+            "value": "updated",
+        })
+        assert response.status_code == 204
+        response = self.client.get("/api/v1/variables/var1")
+        assert response.json == {

Review comment:
       Ha I see. I have not used IDEs myself, but definitely looks like something that can be easily fixed by tweaking the IDE config. Intelij IDEA is probably using some kind of custom test runner by default. Nested diffing with assert is supported natively by upstream py.test runner.




----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739323



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))
+
+    if var.data["key"] != variable_key:
+        raise BadRequest("Invalid post body", detail="key from request body doesn't match uri parameter")
+
+    Variable.set(var.data["key"], var.data["val"])
+    return make_response('', 204)
 
 
 def post_variables():
     """
     Create a variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())
+    except ValidationError as err:
+        raise BadRequest("Invalid Variable schema", detail=str(err.messages))

Review comment:
       ```suggestion
           raise BadRequest("Invalid Variable schema", detail=err.messages)
   ```




----------------------------------------------------------------
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.

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



[GitHub] [airflow] houqp commented on pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
houqp commented on pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#issuecomment-645722927


   @mik-laj updated and ready for another round of review


----------------------------------------------------------------
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.

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439739119



##########
File path: airflow/api_connexion/endpoints/variable_endpoint.py
##########
@@ -14,41 +14,76 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from typing import Optional
 
-# TODO(mik-laj): We have to implement it.
-#     Do you want to help? Please look at: https://github.com/apache/airflow/issues/8142
+from flask import Response, make_response, request
+from marshmallow import ValidationError
 
+from airflow.api_connexion.exceptions import BadRequest, NotFound
+from airflow.api_connexion.schemas.variable_schema import variable_collection_schema, variable_schema
+from airflow.models import Variable
+from airflow.utils.session import provide_session
 
-def delete_variable():
+
+def delete_variable(variable_key: str) -> Response:
     """
     Delete variable
     """
-    raise NotImplementedError("Not implemented yet.")
+    Variable.delete(variable_key)
+    return make_response('', 204)
 
 
-def get_variable():
+def get_variable(variable_key: str) -> Response:
     """
     Get a variables by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = Variable.get(variable_key)
+    except KeyError:
+        raise NotFound("Variable not found")
+    return variable_schema.dump({"key": variable_key, "val": var})
 
 
-def get_variables():
+@provide_session
+def get_variables(session, limit: Optional[int], offset: Optional[int] = None) -> Response:
     """
     Get all variable values
     """
-    raise NotImplementedError("Not implemented yet.")
+    query = session.query(Variable)
+    if limit:
+        if offset:
+            query = query.offset(offset * limit)
+        query = query.limit(limit)
+    variables = query.all()
+    return variable_collection_schema.dump({
+        "variables": variables,
+        "total_entries": len(variables),
+    })
 
 
-def patch_variable():
+def patch_variable(variable_key: str):
     """
     Update a variable by key
     """
-    raise NotImplementedError("Not implemented yet.")
+    try:
+        var = variable_schema.loads(request.get_data())

Review comment:
       ```suggestion
           var = variable_schema.load(request.json)
   ```
   The view expects a json? I suggest we use `load` instead of `loads`
   
   Also, this view expects an UpdateMask parameter in query. Would be great if you add it




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

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



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #9273: implement api v1 for variables

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #9273:
URL: https://github.com/apache/airflow/pull/9273#discussion_r439740375



##########
File path: airflow/api_connexion/schemas/variable_schema.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 marshmallow import Schema, fields
+
+
+class VariableSchema(Schema):
+    """ Variable Schema """
+    key = fields.String(required=True)
+    value = fields.String(attribute="val", required=True)
+
+
+class VariableCollectionSchema(Schema):
+    """ Variable Collection Schema """
+    variables = fields.List(fields.Nested(VariableSchema))
+    total_entries = fields.Int()
+

Review comment:
       I would do this schema as follow:
   ```
   class VariableCollectionItemSchema(SQLAlchemySchema):
       """ A schema for variable collection item """
       class Meta:
           """ Meta """
           model = Variable
       key = auto_field(required=True)
   
   
   class VariableSchema(VariableCollectionItemSchema):
       """ Variable Schema """
       val = auto_field(dump_to='value', load_from='value', required=True)
   
   class VariableCollectionSchema(Schema):
       """ Variable Collection Schema """
       variables = fields.List(fields.Nested(VariableSchema))
       total_entries = fields.Int()
   
   ```
   The spec has this to say about VariableCollectionItemSchema : `Divided into two schemas for sensitive data protection`




----------------------------------------------------------------
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.

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