Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add cli command for 'airflow dags reserialize` #19471

Merged
merged 4 commits into from
Nov 29, 2021

Conversation

collinmcnulty
Copy link
Contributor

Dag serialization is currently out of the hands of the user. Whenever dag reserialization is required, I run this python script:

from airflow.models.serialized_dag import SerializedDagModel
from airflow.settings import Session
session = Session()
session.query(SerializedDagModel).delete()
session.commit()

This PR makes running that script as simple as airflow dags reserialize.

closes: #19432

@potiuk
Copy link
Member

potiuk commented Nov 8, 2021

Doc build is failing. I think it would also be good to add this info to the troubleshooting section in the upcoming : upgrading page: #19453

"from the DagBag folder. This can be helpful if your serialized DAGs get out of sync with the "
"version of Airflow that you are running."
),
func=lazy_load_command('airflow.cli.commands.dag_command.dag_reserialize'),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this needs to be thought through.

  1. It should be nested under airflow dags reserialize
  2. For actual reserialize we should serialize after deleting too rather than waiting for dag parsing process I think
  3. Needs some unit tests

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I thought I did nest it there? That was my intention. Am I misreading something?
  2. I was thinking that it would be better to keep all the dag parsing logs together, but that's not a very strongly held opinion.
  3. Will do.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(2) As the name is reserialize -- it will only reserialize if the scheduler is running. If the scheduler is not running for whatever reason, the users will have a wrong impression that reserialize does actually deserialize instead of just "clear"/delete

@dstandish
Copy link
Contributor

dstandish commented Nov 8, 2021

@collinmcnulty can you share some context re under what circumstances you find that this needs to be done? perhaps there is a change we should make to the dag parsing process that makes it so we don't need to jump in there manually

@collinmcnulty
Copy link
Contributor Author

@dstandish The biggest was part of the 2.2 upgrade where the Param change was supposed to be backwards compatible, but wasn't strictly. I've also seen cases where a Dag Import error didn't show up because there was a valid but outdated, serialized version of the dag. Not sure if the latter case is still present in main. I can check.

@collinmcnulty
Copy link
Contributor Author

Docs are failing because they do not like the word "reserialize".

@dstandish
Copy link
Contributor

there's a spelling_wordlist.txt you should be able to modify

@dstandish
Copy link
Contributor

The biggest was part of the 2.2 upgrade where the Param change was supposed to be backwards compatible, but wasn't strictly. I've also seen cases where a Dag Import error didn't show up because there was a valid but outdated, serialized version of the dag. Not sure if the latter case is still present in main. I can check.

interesting i wonder if perhaps it would be good to have it automatically reserialize at the end of an upgrade. i'm not sure with what frequency this is done currently, but i would have assumed it would be done with each parsing interval like every 5 minutes or something 🤔

@collinmcnulty
Copy link
Contributor Author

@dstandish It will reserialize if it successfully parses. But if you break a DAG, then it will keep the old serialized version. See more discussion over longer term fixes in #19367

@dstandish
Copy link
Contributor

@dstandish It will reserialize if it successfully parses. But if you break a DAG, then it will keep the old serialized version. See more discussion over longer term fixes in #19367

thanks, will take a look over there. but if the dag is broken, reserializing wouldn't work anyway would it?

@collinmcnulty
Copy link
Contributor Author

It won't "work" but it will surface the Import error and will stop trying to schedule a broken dag

@kaxil
Copy link
Member

kaxil commented Nov 10, 2021

#19367 / #18120 (WIP) will hopefully fix the underlying issue

But a CLI command to show serialized representation, actually serialize from a dag file or clear -- all would be very handy

@potiuk
Copy link
Member

potiuk commented Nov 11, 2021

#19367 / #18120 (WIP) will hopefully fix the underlying issue

But a CLI command to show serialized representation, actually serialize from a dag file or clear -- all would be very handy

Agree. That's a good idea to add more tooling here - not only reserializing but outputing serialized dags nicely formatted and coloured using pygments - that woudl be reallly nice.

@potiuk
Copy link
Member

potiuk commented Nov 24, 2021

will you rebase/continue that @collinmcnulty ?

@collinmcnulty
Copy link
Contributor Author

Yes I have rebased (back from vacation). I was not planning on adding any more features to this PR. Adding the display of the serialized dags with pygments is something I would have to study how to do. I was thinking it might be worthwhile to review this PR as is and open a separate issue to add the additional tooling. If you disagree, I can look at adding this tooling, but it will take me a little longer.

@github-actions
Copy link

The PR is likely OK to be merged with just subset of tests for default Python and Database versions without running the full matrix of tests, because it does not modify the core of Airflow. If the committers decide that the full tests matrix is needed, they will add the label 'full tests needed'. Then you should rebase to the latest main or amend the last commit of the PR, and push it with --force-with-lease.

@github-actions github-actions bot added the okay to merge It's ok to merge this PR as it does not require more tests label Nov 29, 2021
@potiuk potiuk merged commit c4e8959 into apache:main Nov 29, 2021
dillonjohnson pushed a commit to dillonjohnson/airflow that referenced this pull request Dec 1, 2021
@EricGao888
Copy link
Member

EricGao888 commented Dec 7, 2021

Our team are using airflow with Celery Executor. For example, we have a cluster of 1 header machine and 2 worker machines. We deploy webserver and scheduler on header and have the workers run on worker machines. If I run airflow dags reserialize on header machine, will the two workers get updated? Will there be any consistent issues? Should I run airflow dags reserialize on both header and workers? Thanks.

@uranusjr
Copy link
Member

uranusjr commented Dec 7, 2021

Workers don’t need DAG information, only the scheduler does.

@jedcunningham jedcunningham added this to the Airflow 2.3.0 milestone Dec 7, 2021
@EricGao888
Copy link
Member

Workers don’t need DAG information, only the scheduler does.

Thanks for the reply. Another question is will the python script from airflow.models.serialized_dag import SerializedDagModel from airflow.settings import Session session = Session() session.query(SerializedDagModel).delete() session.commit() work for airflow 1.10.12?
image
I found the some info from https://airflow.apache.org/docs/apache-airflow/stable/dag-serialization.html
It seems in airflow 1.10.12, the scheduler does not use Serialized DAGS. Does it mean the sync py script in #19471 does not work for airflow 1.10.12? I did a quick test and didn’t get the dags synced in airflow 1.10.12. Did I miss any important information?

@potiuk
Copy link
Member

potiuk commented Dec 8, 2021

Did I miss any important information?

I think you missed the information that Airlfow 1.10 (not even 1.10.12 but any 1.10 version) reached end of life in June 2021 and will not receive any updates (not even critical fixes) any more. We do not produce any code that is targetted for 1.10 for about half a year already.

See here for example: https://github.com/apache/airflow#version-life-cycle

Please migrate to Airflow 2 as soon as possible

@EricGao888
Copy link
Member

Did I miss any important information?

I think you missed the information that Airlfow 1.10 (not even 1.10.12 but any 1.10 version) reached end of life in June 2021 and will not receive any updates (not even critical fixes) any more. We do not produce any code that is targetted for 1.10 for about half a year already.

See here for example: https://github.com/apache/airflow#version-life-cycle

Please migrate to Airflow 2 as soon as possible

Thanks for the reply. Will migrate to Airflow 2 ASAP.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:CLI okay to merge It's ok to merge this PR as it does not require more tests type:new-feature Changelog: New Features
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Trigger Reserialization on Demand
7 participants