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 2021/11/26 08:47:15 UTC

[GitHub] [airflow] frapa-az opened a new issue #19836: DAGRun API endpoint returns status 500 for some run states

frapa-az opened a new issue #19836:
URL: https://github.com/apache/airflow/issues/19836


   ### Apache Airflow version
   
   2.0.2
   
   ### Operating System
   
   Unknown
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other Docker-based deployment
   
   ### Deployment details
   
   _No response_
   
   ### What happened
   
   I am using the REST API. 
   
   When I make a request to the [DAGRun endpoint](https://airflow.apache.org/docs/apache-airflow/stable/stable-rest-api-ref.html#operation/get_dag_runs) and one DAGRun is in state skipped, the request fails with 500 internal server error:
   
   ```text
   {
       "detail": "'skipped' is not one of ['success', 'running', 'failed']\n\nFailed validating 'enum' in schema['allOf'][0]['properties']['dag_runs']['items']['properties']['state']:\n    {'description': 'DAG State.',\n     'enum': ['success', 'running', 'failed'],\n     'readOnly': True,\n     'type': 'string',\n     'x-scope': ['',\n                 '#/components/schemas/DAGRunCollection',\n                 '#/components/schemas/DAGRun']}\n\nOn instance['dag_runs'][1]['state']:\n    'skipped'",
       "status": 500,
       "title": "Response body does not conform to specification",
       "type": "https://airflow.apache.org/docs/2.0.2/stable-rest-api-ref.html#section/Errors/Unknown"
   }
   ```
   
   This seemingly happens because the state of the run is skipped, and not one of those three.
   
   ### What you expected to happen
   
   I would expect `state=skipped` to be returned by the API.
   
   ### How to reproduce
   
   _No response_
   
   ### Anything else
   
   I have no access and do not know how AirFlow was set-up in our current system.
   
   ### Are you willing to submit PR?
   
   - [ ] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981815421


   @uranusjr what do you mean by subDags? I do not unfortunately know the details because we have an abstraction layer on top, but I think there's actually a single DAG that runs all pipelines, creating tasks dinamically based on a json bspec. Does that help?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981815421


   @uranusjr what do you mean by subDags? I do not unfortunately know the details because we have an abstraction layer on top, but I think there's actually a single DAG that runs all pipelines, creating tasks dinamically based on a json spec. Does that help?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981745458


   @ephraimbuddy Thanks for answering. I manage to reproduce it by simply pausing the DAG (not the DAG run). I do not know exactly if this is timing dependent, but for me it seems rather easy to reproduce 
   
   <img width="854" alt="image" src="https://user-images.githubusercontent.com/68114433/143895776-e93240ab-b9a0-4b9d-a2a4-152827820f03.png">
   
   I have a few in the last few days, so it's not impossible:
   
   <img width="803" alt="image" src="https://user-images.githubusercontent.com/68114433/143896114-e05c8749-a7f4-49a6-8c98-29cac2974f58.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981831469


   @uranusjr Wow, thanks for finding out. I can ask the team responsible for this if they give me more details to ensure that this is really the case. Most probably I'll come back tomorrow with an answer.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-982852567


   FWIW, we can pretty trivially validate state values on assignment (both on `DagRun` and `TaskInstance`, even though the latter one isn’t the problem here). Feel free to create a pull request for this. I created #19898 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-979806150


   WTH, a DAG run is not supposed to be skipped. There’s a serious bug somewhere.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981745458


   @ephraimbuddy Thanks for answering. I manage to reproduce it by simply pausing the DAG (not the DAG run). I do not know exactly if this is timing dependent, but for me it seems rather easy to reproduce. Usually I pause just after submission.
   
   <img width="854" alt="image" src="https://user-images.githubusercontent.com/68114433/143895776-e93240ab-b9a0-4b9d-a2a4-152827820f03.png">
   
   I have a few in the last few days, so it must not be impossible:
   
   <img width="803" alt="image" src="https://user-images.githubusercontent.com/68114433/143896114-e05c8749-a7f4-49a6-8c98-29cac2974f58.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981828727


   > I think there's actually a single DAG that runs all pipelines,
   
   Ah!
   
   SubDAGs are DAGs that belong to another DAG, created with SubDagOperator: https://www.astronomer.io/guides/subdags
   
   I’m guessing your abstraction layer is implemented exactly with this—the top-level DAG creates a task for each of the DAGs you create through the layer, and those DAGs (except the top one) are actually subDAGs from Airflow’s perspective.
   
   From my code search, there is one single possible code path that would set a DAG run’s state to skipped. If a SubDagOperator task is skipped, the task scheduler would set its state to skipped (obviously), and *the operator would also set its controlling subDAG’s state to `SKIPPED`*.
   
   This is the exact function that does it:
   
   https://github.com/apache/airflow/blob/c4e8959d141512226a994baeea74d5c7e643c598/airflow/api/common/experimental/mark_tasks.py#L195-L210
   
   This function is only called by `get_subdag_runs`, which is only called by `set_state` (all in the same module), which is used to set a task instance’s state.
   
   A DAG run can only be `QUEUED`, `RUNNING`, `SUCCESS`, or `FAILED`. I think setting the state to `FAILED` makes the most sense in this case?
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981831469


   @uranusjr Wow, thanks for finding out! I can ask the team responsible for this if they give me more details to ensure that this is really the case. Most probably I'll come back tomorrow with an answer.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] boring-cyborg[bot] commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-979792205


   Thanks for opening your first issue here! Be sure to follow the issue template!
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-982712747


   > However, maybe it would be good to add some validation to the states before saving them to avoid that they go to such inconsistent states? I find it a little strange that AirFlow accepts this to be saved into the database, but then validates it when returning it from the API.
   
   Currently you are fully in power of what you do with Airflow code, but power also means responsibility. This is the basic premise of Airlfow "open DB access" mode that Airflow currently supports. There is nothig to prevent any DB operations from the code point of view. You'd have to implement complex Databse triggers or tap into SQLAlchemy logic to prevent this (but the latter could be bypassed).  Airflow supports "raw" DB operation. Use those powers wisely.
   
   And this one is not very disruptive opration. Currently in Airlfow you can literally delete whole database from callback if you want. So preventing this case is like trying to plug a small hole with your finger where half of the bottom of your boat is missing. 
   
   However, we are already working on Multitenancy implementation and specifically it includes DB-Isolation mode, where you won't be able to directly execute DB operations, but you will have some API methods to call (including set_task state on your task). 
   
   Until there it makes litle sense to do anything with it.
   
   See the notes here: (https://docs.google.com/document/d/19d0jQeARnWm8VTVc_qSIEDldQ81acRk0KuhVwAd96wo/edit). 
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981831469


   @uranusjr Wow, thanks for finding out! I can ask the team responsible for this, if they give me more details to ensure that this is really the case. Most probably I'll come back tomorrow with an answer.
   
   I guess setting it to `FAILED` makes the most sense out of the options.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk closed issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #19836:
URL: https://github.com/apache/airflow/issues/19836


   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981831469


   @uranusjr Wow, thanks for finding out! I can ask the team responsible for this, if they give me more details to ensure that this is really the case. Most probably I'll come back tomorrow with an answer.
   
   I guess setting it to `FAILED` makes the most sense among those options.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981831469


   @uranusjr Wow, thanks for finding out! I can ask the team responsible for this if they give me more details to ensure that this is really the case. Most probably I'll come back tomorrow with an answer.
   
   I guess setting it to `FAILED` makes the most sense out of the options.


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] ephraimbuddy commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-979822717


   @frapa-az How can we reproduce DagRun to be in `skipped` state?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981745458


   @ephraimbuddy Thanks for answering. I manage to reproduce it by simply pausing the DAG (not the DAG run). I do not know exactly if this is timing dependent, but for me it seems rather easy to reproduce 
   
   <img width="854" alt="image" src="https://user-images.githubusercontent.com/68114433/143895776-e93240ab-b9a0-4b9d-a2a4-152827820f03.png">
   
   I have a few in the last few days, so it must not be impossible:
   
   <img width="803" alt="image" src="https://user-images.githubusercontent.com/68114433/143896114-e05c8749-a7f4-49a6-8c98-29cac2974f58.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-982465714


   I digged the code. We're not using Sub-DAGs, but we do something really weird with callbacks. So after all, this is our misuse of AirFlow and not a bug in AirFlow itself. Here the relevant snippet (I believe):
   
   ```python
   def on_failure_callback(context):
       if isinstance(context['exception'], TaskCanceledException):
           session = settings.Session()
           context['task_instance'].set_state(State.SKIPPED, session=session)
           context['dag_run'].state = State.SKIPPED
           session.merge(context['dag_run'])
           session.commit()
       
   task.on_failure_callback = on_failure_callback
   ```
   
   However, maybe it would be good to add some validation to the states before saving them to avoid that they go to such inconsistent states? I find it a little strange that AirFlow accepts this to be saved into the database, but then validates it when returning it from the API.
   
   


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-982873627


   > FWIW, we can pretty trivially validate state values on assignment (both on `DagRun` and `TaskInstance`, even though the latter one isn’t the problem here). Feel free to create a pull request for this. I created #19898 for this.
   
   Ah right!


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981761715


   Just curious, are any of your DAGs happen to be subDAGs?


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981745458


   @ephraimbuddy Thanks for answering. I manage to reproduce it by simply pausing the DAG (not the DAG run). I do not know exactly if this is timing dependent, but for me it seems rather easy to reproduce. Usually I pause just after the run was scheduled.
   
   <img width="854" alt="image" src="https://user-images.githubusercontent.com/68114433/143895776-e93240ab-b9a0-4b9d-a2a4-152827820f03.png">
   
   I have a few in the last few days, so it must not be impossible:
   
   <img width="803" alt="image" src="https://user-images.githubusercontent.com/68114433/143896114-e05c8749-a7f4-49a6-8c98-29cac2974f58.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] frapa-az edited a comment on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
frapa-az edited a comment on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981745458


   @ephraimbuddy Thanks for answering!
   
   I manage to reproduce it by simply pausing the DAG (not the DAG run). I do not know exactly if this is timing dependent, but for me it seems rather easy to reproduce. Usually I pause just after the run was scheduled.
   
   <img width="854" alt="image" src="https://user-images.githubusercontent.com/68114433/143895776-e93240ab-b9a0-4b9d-a2a4-152827820f03.png">
   
   I have a few in the last few days, so it must not be impossible:
   
   <img width="803" alt="image" src="https://user-images.githubusercontent.com/68114433/143896114-e05c8749-a7f4-49a6-8c98-29cac2974f58.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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
uranusjr commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-981835456


   When you do, it’d be most awesome if you could also tell them we appreciate them pushing how Airflow works to the limit that this obscure bug can surface 😛


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on issue #19836: DAGRun API endpoint returns status 500 for some run states

Posted by GitBox <gi...@apache.org>.
potiuk commented on issue #19836:
URL: https://github.com/apache/airflow/issues/19836#issuecomment-982715688


   Closing this one for now then


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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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