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

Make next_dagrun_info take a data interval #18088

Merged

Conversation

uranusjr
Copy link
Member

@uranusjr uranusjr commented Sep 8, 2021

From #17552 (comment)

This modifies the Timetable.next_dagrun_info() to take the previous run's data interval instead of logical date. DAG.next_dagrun_info() is modified to accept datetime | DataInterval | None for compatibility, but the datetime form is deprecated. When receiving a datetime, a compatibility method DAG.infer_automated_data_interval() is called to reverse-infer the data interval from the logical date, which should work for all DAGs prior to AIP-39 implementation.

All code paths that need DAG.infer_automated_data_interval() become deprecated:

  • DAG.following_schedule()
  • DAG.next_dagrun_info() with a datetime argument
  • DAG.create_dagrun() without passing in a data interval
  • DagModel.calculate_dagrun_date_fields() with a datetime argument

All existing usages of the newly deprecated code paths are rewritten to avoid triggering the deprecation warning, the most significant one being DAG.following_schedule(). Most of its usages are to calculate the end of the interval, replaced by data_interval.end instead (except those already deprecated, which continue to use DAG.following_schedule() to avoid any unintended change).

For clarity, Timetable.infer_data_interval() is renamed to Timetable.infer_manual_data_interval().

Hopefully this is the last big thing we need to do before 2.2.

@uranusjr uranusjr added the AIP-39 Timetables label Sep 8, 2021
@uranusjr uranusjr requested a review from ashb September 8, 2021 14:08
@boring-cyborg boring-cyborg bot added area:API Airflow's REST/HTTP API area:CLI area:core-operators Operators, Sensors and hooks within Core Airflow area:providers area:Scheduler Scheduler or dag parsing Issues area:webserver Webserver related Issues kind:documentation provider:google Google (including GCP) related issues labels Sep 8, 2021
airflow/models/dag.py Outdated Show resolved Hide resolved
airflow/models/dag.py Outdated Show resolved Hide resolved
airflow/sensors/time_delta.py Outdated Show resolved Hide resolved
Copy link
Member

@ashb ashb left a comment

Choose a reason for hiding this comment

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

LGTM -- I've left a few small comments

@github-actions github-actions bot added the full tests needed We need to run full set of tests for this PR to merge label Sep 8, 2021
@github-actions
Copy link

github-actions bot commented Sep 8, 2021

The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.

@uranusjr uranusjr force-pushed the aip-39-timetable-receives-data-interval branch 5 times, most recently from e91a750 to 7161ad9 Compare September 9, 2021 13:28
This modifies the Timetable.next_dagrun_info() to take the previous
run's data interval instead of logical date. DAG.next_dagrun_info() is
modified to accept 'datetime | DataInterval | None' for compatibility,
but the datetime form is deprecated. When receiving a datetime, a
compatibility method DAG.infer_automated_data_interval() is called to
reverse-infer the data interval from the logical date, which should work
for all DAGs prior to AIP-39 implementation.

All code paths that need DAG.infer_automated_data_interval() become
deprecated:

* DAG.following_schedule()
* DAG.next_dagrun_info() with a datetime argument
* DAG.create_dagrun() without passing in a data interval
* DagModel.calculate_dagrun_date_fields() with a datetime argument

All existing usages of the newly deprecated code paths are rewritten to
avoid triggering the deprecation warning, the most significant one being
DAG.following_schedule(). Most of its usages are to calculate the end of
the interval, replaced by data_interval.end instead (except those
already deprecated, which continue to use DAG.following_schedule() to
avoid any unintended change).

For clarity, Timetable.infer_data_interval() is renamed to
Timetable.infer_manual_data_interval().
@uranusjr uranusjr force-pushed the aip-39-timetable-receives-data-interval branch from 7161ad9 to f6f6766 Compare September 9, 2021 14:07
@uranusjr uranusjr marked this pull request as ready for review September 9, 2021 14:58
@uranusjr
Copy link
Member Author

uranusjr commented Sep 9, 2021

Looks ready!

@uranusjr uranusjr merged commit 2fd3f27 into apache:main Sep 9, 2021
@uranusjr uranusjr deleted the aip-39-timetable-receives-data-interval branch September 9, 2021 21:27
@uranusjr uranusjr mentioned this pull request Sep 13, 2021
Comment on lines +649 to +652
This method is used to bridge runs created prior to AIP-39
implementation, which do not have an explicit data interval. Therefore,
this method only considers ``schedule_interval`` values valid prior to
Airflow 2.2.
Copy link
Contributor

Choose a reason for hiding this comment

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

@uranusjr I stumbled onto this comment. It's a little confusing.

In particular this:

this method only considers ``schedule_interval`` values valid prior to Airflow 2.2.

Could you possibly clarify what it means?

Copy link
Member Author

Choose a reason for hiding this comment

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

In 2.2, dag.schedule_interval can be "backfilled" from a custom timetable by calling timetable.summary, which is only used for UI display. This function does not account for those values, and only considers cases where dag.schedule_interval is an actual valid value supplied by the user, the only possible way to set a DAG's schedule prior to AIP-39.

The reason to this is that this function is only supposed to be used to "infer" a data interval for DAG runs created prior to 2.2. The AIP-39 implementation adds two extra columns data_interval_start and data_interval_end on DagRun, but for performance reasons, we don't backfill those values to existing DagRun instances, so runs created prior to the 2.2 upgrade would have both fields set to None. When we need to access the data interval for such a DAG run (mostly just for UI display), this function generates it dynamically from schedule_interval. Since it's impossible for a DAG run created prior to AIP-39 implementation to have a schedule backed by DAG.timetable, this function does not take the possibility of a user-supplied timetable into account, and only implements logic for cases that can be represented by schedule_interval before Airflow 2.2.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
AIP-39 Timetables area:API Airflow's REST/HTTP API area:CLI area:core-operators Operators, Sensors and hooks within Core Airflow area:providers area:Scheduler Scheduler or dag parsing Issues area:webserver Webserver related Issues full tests needed We need to run full set of tests for this PR to merge kind:documentation provider:google Google (including GCP) related issues
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants