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/16 12:03:18 UTC

[GitHub] [airflow] OmairK opened a new pull request #9329: [WIP] Pool CRUD Endpoints

OmairK opened a new pull request #9329:
URL: https://github.com/apache/airflow/pull/9329


   Closes #9114 
   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [ ] Description above provides context of the change
   - [ ] Unit tests coverage for changes (not needed for documentation changes)
   - [ ] Target Github ISSUE in description if exists
   - [ ] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [ ] Relevant documentation is updated including usage instructions.
   - [ ] 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] OmairK commented on a change in pull request #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -136,7 +136,225 @@ def test_response_404(self):
             {
                 "detail": None,
                 "status": 404,
-                "title": "Pool not found",
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):
+        del name
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):

Review comment:
       > Can you add a test where name is missing
   
   Here is the [test](https://github.com/apache/airflow/pull/9329/files#diff-83587b0e0b4bb59a06b380179eb8e914R270-R285)
   But removing `name` and `slots` from the required fields in spec doesn't makes sense to me. Patching selected fields should be handled by the update_mask meaning the request json should always have all updatable fields.  




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   We must add more restriction related to default branch.
   -  DEFAULT_POOL_NAME should not be possible to delete.
   -  name of DEFAULT_POOL_NAME should not be possible to change.
   


----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +55,52 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool = session.query(Pool).filter(Pool.pool == pool_name).first()
+    if not pool:
+        raise NotFound(f"Pool with name:'{pool_name}' not found")
 
+    patch_body = pool_schema.load(request.json).data
+    if update_mask:
+        update_mask = [i.strip() for i in update_mask]
+        _patch_body = {}
+        try:
+            update_mask = [
+                pool_schema.declared_fields[field].attribute
+                if pool_schema.declared_fields[field].attribute
+                else field
+                for field in update_mask
+            ]
+        except KeyError as err:
+            raise BadRequest(f"Invalid field: {err.args[0]} in update mask")
+        _patch_body = {field: patch_body[field] for field in update_mask}
+        patch_body = _patch_body
 
-def post_pool():
+    for key, value in patch_body.items():
+        setattr(pool, key, value)
+    session.commit()
+    return pool_schema.dump(pool)
+
+
+@provide_session
+def post_pool(session):
     """
-    Create aa pool
+    Create a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    post_body = pool_schema.load(request.json, session=session).data
+    pool_instance = session.query(Pool).filter(Pool.pool == post_body["pool"]).first()

Review comment:
       Thanks, [fixed](https://github.com/apache/airflow/pull/9329/files#diff-89eaafe05e80053aa3cbf1e0610ca09dR102-R106).




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -14,61 +14,91 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from flask import request
+from flask import Response, request
 from sqlalchemy import func
 
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, NotFound
 from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name: str, session) -> Response:
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    if session.query(Pool).filter(Pool.pool == pool_name).delete() == 0:
+        raise NotFound("Pool not found")
+    return Response(status=204)
 
 
 @provide_session
-def get_pool(pool_name, session):
+def get_pool(pool_name, session) -> Response:

Review comment:
       ```suggestion
   def get_pool(pool_name, session):
   ```
   It seems to me that this is not true. mypy doesn't raise error because it got lost in types, but we return the dictionary from this function.




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   > I am thinking 400. What do you suggest?
   
   400 BadRequest


----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        response = self.client.post("api/v1/pools", json={
+            "name": "test_pool_a",
+            "slots": 3
+        })
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+    
+    @provide_session
+    def test_response_409(self, session):

Review comment:
       Could you add this error code to the specification?




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -14,20 +14,28 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from connexion import NoContent
 from flask import request
 
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, NotFound
 from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name, session):
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool_id = pool_name
+    query = session.query(Pool)
+    obj = query.filter(Pool.pool == pool_id).one_or_none()
+    if obj is None:
+        raise NotFound("Pool not found")
+    session.delete(obj)

Review comment:
       Neat :smile: ! 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] OmairK commented on a change in pull request #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):

Review comment:
       Here is the change `da4cf5b`




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        response = self.client.post("api/v1/pools", json={
+            "name": "test_pool_a",
+            "slots": 3
+        })
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+    
+    @provide_session
+    def test_response_409(self, session):

Review comment:
       Here is the change `da4cf5b`




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == "default_pool":

Review comment:
       Thanks, fixed ee31051




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -14,61 +14,91 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from flask import request
+from flask import Response, request
 from sqlalchemy import func
 
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, NotFound
 from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name: str, session) -> Response:
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    if session.query(Pool).filter(Pool.pool == pool_name).delete() == 0:
+        raise NotFound("Pool not found")
+    return Response(status=204)
 
 
 @provide_session
-def get_pool(pool_name, session):
+def get_pool(pool_name, session) -> Response:
     """
     Get a pool
     """
     pool_id = pool_name
     query = session.query(Pool)
     obj = query.filter(Pool.pool == pool_id).one_or_none()
-
     if obj is None:
         raise NotFound("Pool not found")
     return pool_schema.dump(obj)
 
 
 @provide_session
-def get_pools(session):
+def get_pools(session) -> Response:

Review comment:
       ```suggestion
   def get_pools(session):
   ```

##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -14,61 +14,91 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from flask import request
+from flask import Response, request
 from sqlalchemy import func
 
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, NotFound
 from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name: str, session) -> Response:
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    if session.query(Pool).filter(Pool.pool == pool_name).delete() == 0:
+        raise NotFound("Pool not found")
+    return Response(status=204)
 
 
 @provide_session
-def get_pool(pool_name, session):
+def get_pool(pool_name, session) -> Response:
     """
     Get a pool
     """
     pool_id = pool_name
     query = session.query(Pool)
     obj = query.filter(Pool.pool == pool_id).one_or_none()
-
     if obj is None:
         raise NotFound("Pool not found")
     return pool_schema.dump(obj)
 
 
 @provide_session
-def get_pools(session):
+def get_pools(session) -> Response:
     """
     Get all pools
     """
     offset = request.args.get(parameters.page_offset, 0)
     limit = min(int(request.args.get(parameters.page_limit, 100)), 100)
 
     total_entries = session.query(func.count(Pool.id)).scalar()
-    query = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit)
+    query = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
 
-    pools = query.all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=query, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None) -> Response:

Review comment:
       ```suggestion
   def patch_pool(pool_name, session, update_mask=None):
   ```




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -136,7 +136,225 @@ def test_response_404(self):
             {
                 "detail": None,
                 "status": 404,
-                "title": "Pool not found",
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):
+        del name
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):

Review comment:
       > Name is already on the path, the normal thing is that it shouldn't be on the payload.
   That's what most users would expect
   
   Name is the lookup field to retrieve the instance but it's also a patchable field according to the [spec](https://redocly.github.io/redoc/?url=https://raw.githubusercontent.com/OmairK/airflow/healt-endpoint-8144/airflow/api_connexion/openapi/v1.yaml#operation/airflow.api_connexion.endpoints.pool_endpoint.patch_pool).
    




----------------------------------------------------------------
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 edited a comment on pull request #9329: Pool CRUD Endpoints

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #9329:
URL: https://github.com/apache/airflow/pull/9329#issuecomment-647176347


   We must add more restriction related to default branch.
   -  DEFAULT_POOL_NAME should not be possible to delete.
   -  name of pool with name= DEFAULT_POOL_NAME should not be possible to change.
   


----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == Pool.DEFAULT_POOL_NAME and request.json["name"] != Pool.DEFAULT_POOL_NAME:

Review comment:
       Yes I changed it to a required field as it should always be present in the body. 




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -16,18 +16,32 @@
 # under the License.
 from flask import request
 
+from connexion import NoContent
+
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
-from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
+from airflow.api_connexion.exceptions import NotFound, AlreadyExists
+from airflow.api_connexion.schemas.pool_schema import (
+    PoolCollection,
+    pool_collection_schema,
+    pool_schema,
+)
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name, session):
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool_id = pool_name
+    query = session.query(Pool)
+    obj = query.filter(Pool.pool == pool_id).one_or_none()
+
+    if obj is None:

Review comment:
       ```suggestion
       obj = session.query(Pool).filter(Pool.pool == pool_name).one_or_none()
       if obj is None:
   ```




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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


   It would be good to have strict validation for the schema. Strict validation raises validation error and to add it, just add strict arg to pool schema. `PoolSchema(strict=True)`. 


----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +55,52 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool = session.query(Pool).filter(Pool.pool == pool_name).first()
+    if not pool:
+        raise NotFound(f"Pool with name:'{pool_name}' not found")
 
+    patch_body = pool_schema.load(request.json).data
+    if update_mask:
+        update_mask = [i.strip() for i in update_mask]
+        _patch_body = {}
+        try:
+            update_mask = [
+                pool_schema.declared_fields[field].attribute
+                if pool_schema.declared_fields[field].attribute
+                else field
+                for field in update_mask
+            ]
+        except KeyError as err:
+            raise BadRequest(f"Invalid field: {err.args[0]} in update mask")
+        _patch_body = {field: patch_body[field] for field in update_mask}
+        patch_body = _patch_body
 
-def post_pool():
+    for key, value in patch_body.items():
+        setattr(pool, key, value)
+    session.commit()
+    return pool_schema.dump(pool)
+
+
+@provide_session
+def post_pool(session):
     """
-    Create aa pool
+    Create a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    post_body = pool_schema.load(request.json, session=session).data
+    pool_instance = session.query(Pool).filter(Pool.pool == post_body["pool"]).first()

Review comment:
       you can combine these two queries into one by relying on the unique pool name index in the database. just add the pool object as is without the check, then return already exists error if you get a unique index constraint violation exception .




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,164 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):  # pylint: disable=unused-argument
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        pool = Pool(pool="test_pool", slots=3)
+        session.add(pool)
+        session.commit()
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "occupied_slots": 0,
+                "queued_slots": 0,
+                "name": "test_pool_a",
+                "open_slots": 3,
+                "running_slots": 0,
+                "slots": 3,
+            },
+            response.json,
+        )
+
+    def test_response_404(self):
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 404
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPoolWithUpdateMask(TestBasePoolEndpoints):
+    @parameterized.expand(
+        [
+            (
+                "api/v1/pools/test_pool?update_mask=name, slots",

Review comment:
       Can you add a test that tries to update the read-only field e.g. occupied_slots?




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -14,20 +14,28 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+from connexion import NoContent
 from flask import request
 
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
+from airflow.api_connexion.exceptions import AlreadyExists, NotFound
 from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name, session):
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool_id = pool_name
+    query = session.query(Pool)
+    obj = query.filter(Pool.pool == pool_id).one_or_none()
+    if obj is None:
+        raise NotFound("Pool not found")
+    session.delete(obj)

Review comment:
       you can optimize this by running delete query with provided pool_id, then check for deleted row count to decide whether NotFound exception should be raised or not. This way, you can combine two queries into one.




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == Pool.DEFAULT_POOL_NAME and request.json["name"] != Pool.DEFAULT_POOL_NAME:

Review comment:
       Is name  field required in the body now?




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == Pool.DEFAULT_POOL_NAME and request.json["name"] != Pool.DEFAULT_POOL_NAME:

Review comment:
       I think that we cannot set any fields as required at this endpoint.
   Both requests below should be considered as valid. I see no reason for the clent to pass the name twice (once in the URL, once in the body)
   ```
   PATCH /api/v1/pools/default_pool?update_mask=slots
   {
       "sluts": 80
   }
   ```
   ```
   PATCH /api/v1/pools/awesome_pool?update_mask=name
   {
       "name": "my_new_awesome_pool"
   }
   ```
   




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):

Review comment:
       Can you also add a test that check handling for 400?




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -136,7 +136,225 @@ def test_response_404(self):
             {
                 "detail": None,
                 "status": 404,
-                "title": "Pool not found",
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):
+        del name
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):

Review comment:
       I actually mean to exclude name on your patch method test. That is, to have the payload go with only slots. {"slots":3}. 
   Name is already on the path, the normal thing is that it shouldn't be on the payload.
   That's what most users would expect




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   


----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -136,7 +136,225 @@ def test_response_404(self):
             {
                 "detail": None,
                 "status": 404,
-                "title": "Pool not found",
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool with name:'invalid_pool' not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):
+        del name
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):

Review comment:
       Can you add a test where name is missing. The change in spec will not allow it. I just had same problem. This is how I solved it https://github.com/apache/airflow/commit/1a4efd35d37bae28735af470d7846ff3b0bce6f6




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,164 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):  # pylint: disable=unused-argument
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        pool = Pool(pool="test_pool", slots=3)
+        session.add(pool)
+        session.commit()
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "occupied_slots": 0,
+                "queued_slots": 0,
+                "name": "test_pool_a",
+                "open_slots": 3,
+                "running_slots": 0,
+                "slots": 3,
+            },
+            response.json,
+        )
+
+    def test_response_404(self):
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 404
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPoolWithUpdateMask(TestBasePoolEndpoints):
+    @parameterized.expand(
+        [
+            (
+                "api/v1/pools/test_pool?update_mask=name, slots",

Review comment:
       > Can you add a test that tries to update the read-only field e.g. occupied_slots?
   
   Here is the [change](https://github.com/apache/airflow/pull/9329/files#diff-83587b0e0b4bb59a06b380179eb8e914R318-R329)




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -16,18 +16,32 @@
 # under the License.
 from flask import request
 
+from connexion import NoContent
+
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
-from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
+from airflow.api_connexion.exceptions import NotFound, AlreadyExists
+from airflow.api_connexion.schemas.pool_schema import (
+    PoolCollection,
+    pool_collection_schema,
+    pool_schema,
+)
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name, session):
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool_id = pool_name
+    query = session.query(Pool)
+    obj = query.filter(Pool.pool == pool_id).one_or_none()
+
+    if obj is None:

Review comment:
       ```suggestion
       obj = session.query(Pool).filter(Pool.pool == pool_name).one_or_none()
       if obj is None:
   ```
   I feel the code is cleaner like above...




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   
   > ```
   > PATCH /api/v1/pools/default_pool?update_mask=name,slots
   > {
   >     "name": "default_pool",
   >     "slots": 80
   > }
   > ```
   > 
   > Will this request be handled correctly? The name has not been changed because it is the same all the time.
   
   @mik-laj 
   Thanks, fixed it, here is the change `ee31051`


----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -16,18 +16,32 @@
 # under the License.
 from flask import request
 
+from connexion import NoContent
+
 from airflow.api_connexion import parameters
-from airflow.api_connexion.exceptions import NotFound
-from airflow.api_connexion.schemas.pool_schema import PoolCollection, pool_collection_schema, pool_schema
+from airflow.api_connexion.exceptions import NotFound, AlreadyExists
+from airflow.api_connexion.schemas.pool_schema import (
+    PoolCollection,
+    pool_collection_schema,
+    pool_schema,
+)
 from airflow.models.pool import Pool
 from airflow.utils.session import provide_session
 
 
-def delete_pool():
+@provide_session
+def delete_pool(pool_name, session):
     """
     Delete a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    pool_id = pool_name
+    query = session.query(Pool)
+    obj = query.filter(Pool.pool == pool_id).one_or_none()
+
+    if obj is None:

Review comment:
       Thanks here is the change 00bbe23




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   @OmairK 
   ```
   PATCH /api/v1/pools/default_pool?update_mask=name,slots
   {
       "name": "default_pool",
       "sluts: 80
   }
   ```
   Will this request be handled correctly?  The name has not been changed because it is the same all the time.
   


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

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



[GitHub] [airflow] OmairK commented on a change in pull request #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        response = self.client.post("api/v1/pools", json={
+            "name": "test_pool_a",
+            "slots": 3
+        })
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+    
+    @provide_session
+    def test_response_409(self, session):

Review comment:
       Here is the change ~da4cf5b~ e591251 




----------------------------------------------------------------
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 #9329: [WIP] Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,69 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    @provide_session
+    def test_response_404(self, session):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+class TestPostPool(TestBasePoolEndpoints):

Review comment:
       Here is the change ~da4cf5b~ e591251 




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,164 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):  # pylint: disable=unused-argument
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        pool = Pool(pool="test_pool", slots=3)
+        session.add(pool)
+        session.commit()
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "occupied_slots": 0,
+                "queued_slots": 0,
+                "name": "test_pool_a",
+                "open_slots": 3,
+                "running_slots": 0,
+                "slots": 3,
+            },
+            response.json,
+        )
+
+    def test_response_404(self):
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 404
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPoolWithUpdateMask(TestBasePoolEndpoints):
+    @parameterized.expand(
+        [
+            (
+                "api/v1/pools/test_pool?update_mask=name, slots",

Review comment:
       > Also a test where there's a typo in the fields in update mask?
   
   Here is [change](https://github.com/apache/airflow/pull/9329/files#diff-83587b0e0b4bb59a06b380179eb8e914R330-R341)




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == "default_pool":

Review comment:
       ```suggestion
       if pool_name == Pool.DEFAULT_POOL_NAME:
   ```




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

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



[GitHub] [airflow] OmairK commented on a change in pull request #9329: Pool CRUD Endpoints

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



##########
File path: airflow/api_connexion/endpoints/pool_endpoint.py
##########
@@ -53,18 +59,59 @@ def get_pools(session):
 
     total_entries = session.query(func.count(Pool.id)).scalar()
     pools = session.query(Pool).order_by(Pool.id).offset(offset).limit(limit).all()
-    return pool_collection_schema.dump(PoolCollection(pools=pools, total_entries=total_entries)).data
+    return pool_collection_schema.dump(
+        PoolCollection(pools=pools, total_entries=total_entries)
+    ).data
 
 
-def patch_pool():
+@provide_session
+def patch_pool(pool_name, session, update_mask=None):
     """
     Update a pool
     """
-    raise NotImplementedError("Not implemented yet.")
+    # Only slots can be modified in 'default_pool'
+    if pool_name == Pool.DEFAULT_POOL_NAME and request.json["name"] != Pool.DEFAULT_POOL_NAME:

Review comment:
       Thanks fixed it `4802a80 `




----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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


   > We must add more restriction related to default branch.
   > 
   >     * DEFAULT_POOL_NAME should not be possible to delete.
   > 
   >     * name of pool with name= DEFAULT_POOL_NAME should not be possible to change.
   
   Fixed `0f748af`
   


----------------------------------------------------------------
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 #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,164 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):  # pylint: disable=unused-argument
+        response = self.client.post("api/v1/pools", json=request_json)
+        assert response.status_code == 400
+        self.assertEqual(
+            {
+                "detail": f"'{missing_field}' is a required property",
+                "status": 400,
+                "title": "Bad Request",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_200(self, session):
+        pool = Pool(pool="test_pool", slots=3)
+        session.add(pool)
+        session.commit()
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "occupied_slots": 0,
+                "queued_slots": 0,
+                "name": "test_pool_a",
+                "open_slots": 3,
+                "running_slots": 0,
+                "slots": 3,
+            },
+            response.json,
+        )
+
+    def test_response_404(self):
+        response = self.client.patch(
+            "api/v1/pools/test_pool", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 404
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 404,
+                "title": "Pool not found",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+
+class TestPatchPoolWithUpdateMask(TestBasePoolEndpoints):
+    @parameterized.expand(
+        [
+            (
+                "api/v1/pools/test_pool?update_mask=name, slots",

Review comment:
       Also a test where there's a typo in the fields in 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] OmairK commented on pull request #9329: Pool CRUD Endpoints

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


   > We must add more restriction related to default branch.
   > 
   >     * DEFAULT_POOL_NAME should not be possible to delete.
   > 
   >     * name of pool with name= DEFAULT_POOL_NAME should not be possible to change.
   
   What error code should be raised when the user tries to modify/delete the default_pool? 
   I am thinking `400`. 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] mik-laj commented on a change in pull request #9329: Pool CRUD Endpoints

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



##########
File path: tests/api_connexion/endpoints/test_pool_endpoint.py
##########
@@ -141,3 +141,164 @@ def test_response_404(self):
             },
             response.json,
         )
+
+
+class TestDeletePool(TestBasePoolEndpoints):
+    @provide_session
+    def test_response_204(self, session):
+        pool_name = "test_pool"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+
+        response = self.client.delete(f"api/v1/pools/{pool_name}")
+        assert response.status_code == 204
+        # Check if the pool is deleted from the db
+        response = self.client.get(f"api/v1/pools/{pool_name}")
+        self.assertEqual(response.status_code, 404)
+
+    def test_response_404(self):
+        response = self.client.delete("api/v1/pools/invalid_pool")
+        self.assertEqual(response.status_code, 404)
+
+
+class TestPostPool(TestBasePoolEndpoints):
+    def test_response_200(self):
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 200
+        self.assertEqual(
+            {
+                "name": "test_pool_a",
+                "slots": 3,
+                "occupied_slots": 0,
+                "running_slots": 0,
+                "queued_slots": 0,
+                "open_slots": 3,
+            },
+            response.json,
+        )
+
+    @provide_session
+    def test_response_409(self, session):
+        pool_name = "test_pool_a"
+        pool_instance = Pool(pool=pool_name, slots=3)
+        session.add(pool_instance)
+        session.commit()
+        response = self.client.post(
+            "api/v1/pools", json={"name": "test_pool_a", "slots": 3}
+        )
+        assert response.status_code == 409
+        self.assertEqual(
+            {
+                "detail": None,
+                "status": 409,
+                "title": f"Pool: {pool_name} already exists",
+                "type": "about:blank",
+            },
+            response.json,
+        )
+
+    @parameterized.expand(
+        [
+            ("for missing pool name", {"slots": 3}, "name"),
+            ("for missing slots", {"name": "invalid_pool"}, "slots"),
+        ]
+    )
+    def test_response_400(self, name, request_json, missing_field):  # pylint: disable=unused-argument

Review comment:
       ```suggestion
       def test_response_400(self, name, request_json, missing_field):
           del name
   ```




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

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