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

Airflow 2.0.0 manual run causes scheduled run to skip #13434

Closed
salimeryigit opened this issue Jan 2, 2021 · 12 comments · Fixed by #13963
Closed

Airflow 2.0.0 manual run causes scheduled run to skip #13434

salimeryigit opened this issue Jan 2, 2021 · 12 comments · Fixed by #13963
Assignees
Labels
affected_version:2.0 Issues Reported for 2.0 area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug priority:high High priority bug that should be patched quickly but does not require immediate new release
Milestone

Comments

@salimeryigit
Copy link

Apache Airflow version: 2.0.0

Kubernetes version (if you are using kubernetes) (use kubectl version): N/A

Environment:

  • Cloud provider or hardware configuration: local/aws
  • OS (e.g. from /etc/os-release): Ubuntu 18.04.5 LTS
  • Kernel (e.g. uname -a): 5.4.0-1032-aws
  • Install tools: pip
  • Others:

What happened:
I did a fresh Airflow 2.0.0 install. With this version, when I manually trigger a DAG, Airflow skips the next scheduled run.

What you expected to happen:
Manual runs do not interfere with the scheduled runs prior to Airflow 2.

How to reproduce it:
Create a simple hourly DAG. After enabling it and the initial run, run it manually. It shall skip the next hour. Below is an example, where the manual run with execution time of 08:17 causes the scheduled run with execution time of 08:00 to skip.
image

@salimeryigit salimeryigit added the kind:bug This is a clearly a bug label Jan 2, 2021
@boring-cyborg
Copy link

boring-cyborg bot commented Jan 2, 2021

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

@potiuk potiuk added this to the Airflow 2.0.1 milestone Jan 2, 2021
@turbaszek
Copy link
Member

Possibly related to #13407

@turbaszek turbaszek added the area:Scheduler including HA (high availability) scheduler label Jan 2, 2021
@gbonazzoli
Copy link

I don't know if my problem is related to this issue, but I want to underline a change of behavior in airflow 2.0.0 in how the max_active_runs directive works.

I have the following DAG definition:

with DAG('VMWARE_BACKUP',
          description = 'VMWARE_BACKUP',
          tags=['vmware'],
          schedule_interval = None,
          start_date = datetime(2019, 5, 29, tzinfo=local_tz),
          default_args = default_args,
          max_active_runs = 1,   # maximum number of active runs for this DAG
          concurrency = 1,       # Added with Airflow 2.0.0
          catchup = False
    ) as dag:

Before Airflow 2.0.0 only one DAG was allowed to run.

With Airflow 2.0.0 (probably due to the scheduler's rewriting with the fantastic speed added in switching between tasks) at the end of one task in a DAGrun instance instead of scheduling the following one it is scheduled a task from another DAGrun Instance, as you can se from the attached screenshot.

Screen Shot 2021-01-06 at 03 57 10

I can mitigate the problem in Airflow 2.0.0 with the added concurrency = 1 so that at list the only very long task that I want to be the only one running on the system it runs alone....

It works in my use case in other circumstances it could be impracticable.

Please tell me if it is better to open a new main issue.

@vxtals
Copy link

vxtals commented Jan 14, 2021

I don't think this is really a bug but a change of behavior, IMO this should be reversed or at least allow to change it through config.

The problem is in the method bulk_write_to_db in the class DAG https://github.com/apache/airflow/blob/master/airflow/models/dag.py

    # Get the latest dag run for each existing dag as a single query (avoid n+1 query)
    most_recent_dag_runs = dict(
        session.query(DagRun.dag_id, func.max_(DagRun.execution_date))
        .filter(
            DagRun.dag_id.in_(existing_dag_ids),
            or_(
                DagRun.run_type == DagRunType.BACKFILL_JOB,
                DagRun.run_type == DagRunType.SCHEDULED,
                DagRun.external_trigger.is_(True),
            ),
        )
        .group_by(DagRun.dag_id)
        .all()
    )

When is getting from db 'most_recent_dag_runs' it includes DagRun.external_trigger.is_(True).
This most_recent_dag_runs is used later in the method to calculate the next execution, so if it finds a manually triggered execution in the current schedule interval it won't schedule the execution. By removing that line it goes back to previous versions behavior.

@salimeryigit
Copy link
Author

I agree, and to be consistent with the older versions the default behavior should be the old one IMHO (personally, I think it should be reversed).
Because of the way Airflow handles execution time, the execution time between manual and scheduled runs may cause problems. Consider a daily dag run at 00:00. If the dag runs at the scheduled time on 13 Jan, the execution date would be 12 Jan 00:00 (period close time) . If I manually trigger the dag at say 00:30, the execution date would be 13 Jan 00:30 which would cause the scheduled run with execution date 13 Jan 00:00 to skip. Depending on the use case, this can cause problems.

@astleychen
Copy link

@salimeryigit Agreed either. It's too much of an implication that a manual trigger may skip the next scheduled run. I'm also surprised at the first time I met this issue as this basic schedule function should be able to be trusted and normally run as expected. In our scenario, we may trigger the DAG times in a day and also scheduled DAG to run daily as well. It's broken after V2 upgrade. Can we elevate this issue explicitly so that users on V2 can notice this behavior change?

@vikramkoka vikramkoka added the affected_version:2.0 Issues Reported for 2.0 label Jan 18, 2021
@LanDeQuHuXi
Copy link

LanDeQuHuXi commented Jan 23, 2021

It's a quite big change of behavior, please fix or let it be configurable at least.

@jedcunningham
Copy link
Member

This also happens in Postgres with psycopg2 2.8.6.

@salimeryigit
Copy link
Author

Same DB configuration here, Postgres with psycopg2 2.8.6.

@bobfang1992
Copy link

Don't you think this is an unacceptable change? At least in its current form, we risk the next automated run will be skipped entirely if we want to manually trigger a dag run. This will certainly cause trouble. The manual run should not interfere with the scheduled runs IMHO.

@turbaszek
Copy link
Member

This will certainly cause trouble. The manual run should not interfere with the scheduled runs IMHO.

This is already causing problems and confusion. Users are surprised by this change.

@kaxil kaxil self-assigned this Jan 28, 2021
@kaxil kaxil added the priority:high High priority bug that should be patched quickly but does not require immediate new release label Jan 28, 2021
@kaxil
Copy link
Member

kaxil commented Jan 29, 2021

Will be fixed by #13963 and released in 2.0.1 (around 2nd week of Feb)

kaxil added a commit to astronomer/airflow that referenced this issue Jan 29, 2021
kaxil added a commit that referenced this issue Feb 1, 2021
kaxil added a commit that referenced this issue Feb 4, 2021
kaxil added a commit to astronomer/airflow that referenced this issue Apr 12, 2021
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 16, 2021
closes apache/airflow#13434

(cherry picked from commit de277c69e7909cf0d563bbd542166397523ebbe0)

GitOrigin-RevId: 958e25836ffa78273aa195865bc456dceee06351
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 17, 2021
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 23, 2021
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 27, 2021
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Mar 10, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 4, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 7, 2022
closes apache/airflow#13434

(cherry picked from commit de277c69e7909cf0d563bbd542166397523ebbe0)

GitOrigin-RevId: 958e25836ffa78273aa195865bc456dceee06351
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jul 9, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Aug 27, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 4, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
aglipska pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Oct 7, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Dec 7, 2022
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jan 27, 2023
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
kosteev pushed a commit to kosteev/composer-airflow-test-copybara that referenced this issue Sep 12, 2024
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
kosteev pushed a commit to kosteev/composer-airflow-test-copybara that referenced this issue Sep 12, 2024
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Sep 17, 2024
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
kosteev pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Nov 7, 2024
closes apache/airflow#13434

GitOrigin-RevId: de277c69e7909cf0d563bbd542166397523ebbe0
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.0 Issues Reported for 2.0 area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug priority:high High priority bug that should be patched quickly but does not require immediate new release
Projects
None yet
Development

Successfully merging a pull request may close this issue.