Skip to content

Commit

Permalink
Bugfix: Don't try to create a duplicate Dag Run in Scheduler (#13920)
Browse files Browse the repository at this point in the history
closes #13685

When the Scheduler is restarted or killed after creating Dag Run in `Scheduler._create_dag_runs` but
before `Scheduler.self._update_dag_next_dagruns`, the Scheduler falls in a loop because it will not try
to create the Dag Run again in the Scheduler Loop. However, as the DagRun already exists it will fail
with:

```
Traceback (most recent call last):
  File "/Users/kaxilnaik/opt/anaconda3/lib/python3.7/site-packages/sqlalchemy/engine/base.py", line 1277, in _execute_context
    cursor, statement, parameters, context
  File "/Users/kaxilnaik/opt/anaconda3/lib/python3.7/site-packages/sqlalchemy/engine/default.py", line 593, in do_execute
    cursor.execute(statement, parameters)
psycopg2.errors.UniqueViolation: duplicate key value violates unique constraint "dag_run_dag_id_run_id_key"
DETAIL:  Key (dag_id, run_id)=(scenario1_case2_02, scheduled__2021-01-25T00:00:00+00:00) already exists.
```
  • Loading branch information
kaxil authored Feb 1, 2021
1 parent 05fbeb1 commit 594069e
Show file tree
Hide file tree
Showing 2 changed files with 94 additions and 11 deletions.
45 changes: 34 additions & 11 deletions airflow/jobs/scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
from typing import Any, Callable, DefaultDict, Dict, Iterable, List, Optional, Set, Tuple

from setproctitle import setproctitle
from sqlalchemy import and_, func, not_, or_
from sqlalchemy import and_, func, not_, or_, tuple_
from sqlalchemy.exc import OperationalError
from sqlalchemy.orm import load_only, selectinload
from sqlalchemy.orm.session import Session, make_transient
Expand Down Expand Up @@ -1563,6 +1563,20 @@ def _create_dag_runs(self, dag_models: Iterable[DagModel], session: Session) ->
Unconditionally create a DAG run for the given DAG, and update the dag_model's fields to control
if/when the next DAGRun should be created
"""
# Bulk Fetch DagRuns with dag_id and execution_date same
# as DagModel.dag_id and DagModel.next_dagrun
# This list is used to verify if the DagRun already exist so that we don't attempt to create
# duplicate dag runs
active_dagruns = (
session.query(DagRun.dag_id, DagRun.execution_date)
.filter(
tuple_(DagRun.dag_id, DagRun.execution_date).in_(
[(dm.dag_id, dm.next_dagrun) for dm in dag_models]
)
)
.all()
)

for dag_model in dag_models:
try:
dag = self.dagbag.get_dag(dag_model.dag_id, session=session)
Expand All @@ -1571,16 +1585,25 @@ def _create_dag_runs(self, dag_models: Iterable[DagModel], session: Session) ->
continue

dag_hash = self.dagbag.dags_hash.get(dag.dag_id)
dag.create_dagrun(
run_type=DagRunType.SCHEDULED,
execution_date=dag_model.next_dagrun,
start_date=timezone.utcnow(),
state=State.RUNNING,
external_trigger=False,
session=session,
dag_hash=dag_hash,
creating_job_id=self.id,
)
# Explicitly check if the DagRun already exists. This is an edge case
# where a Dag Run is created but `DagModel.next_dagrun` and `DagModel.next_dagrun_create_after`
# are not updated.
# We opted to check DagRun existence instead
# of catching an Integrity error and rolling back the session i.e
# we need to run self._update_dag_next_dagruns if the Dag Run already exists or if we
# create a new one. This is so that in the next Scheduling loop we try to create new runs
# instead of falling in a loop of Integrity Error.
if (dag.dag_id, dag_model.next_dagrun) not in active_dagruns:
dag.create_dagrun(
run_type=DagRunType.SCHEDULED,
execution_date=dag_model.next_dagrun,
start_date=timezone.utcnow(),
state=State.RUNNING,
external_trigger=False,
session=session,
dag_hash=dag_hash,
creating_job_id=self.id,
)

self._update_dag_next_dagruns(dag_models, session)

Expand Down
60 changes: 60 additions & 0 deletions tests/jobs/test_scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -3695,6 +3695,66 @@ def test_bulk_write_to_db_external_trigger_dont_skip_scheduled_run(self):
dag_model = session.query(DagModel).get(dag.dag_id)
assert dag_model.next_dagrun == DEFAULT_DATE + timedelta(minutes=1)

def test_scheduler_create_dag_runs_check_existing_run(self):
"""
Test that if a dag run exists, scheduler._create_dag_runs does not raise an error.
And if a Dag Run does not exist it creates next Dag Run. In both cases the Scheduler
sets next execution date as DagModel.next_dagrun
"""
dag = DAG(
dag_id='test_scheduler_create_dag_runs_check_existing_run',
start_date=DEFAULT_DATE,
schedule_interval=timedelta(days=1),
)

DummyOperator(
task_id='dummy',
dag=dag,
)

session = settings.Session()
assert dag.get_last_dagrun(session) is None

dagbag = DagBag(
dag_folder=os.devnull,
include_examples=False,
read_dags_from_db=False,
)
dagbag.bag_dag(dag=dag, root_dag=dag)

# Create DagModel
DAG.bulk_write_to_db(dagbag.dags.values())
dag_model = DagModel.get_dagmodel(dag.dag_id)

# Assert dag_model.next_dagrun is set correctly
assert dag_model.next_dagrun == DEFAULT_DATE

dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag))

dagrun = dag.create_dagrun(
run_type=DagRunType.SCHEDULED,
execution_date=dag_model.next_dagrun,
start_date=timezone.utcnow(),
state=State.RUNNING,
external_trigger=False,
session=session,
creating_job_id=2,
)
session.flush()

assert dag.get_last_dagrun(session) == dagrun

scheduler = SchedulerJob(subdir=os.devnull, executor=self.null_exec)
scheduler.dagbag = dagbag
scheduler.processor_agent = mock.MagicMock()

# Test that this does not raise any error
scheduler._create_dag_runs([dag_model], session)

# Assert dag_model.next_dagrun is set correctly to next execution date
assert dag_model.next_dagrun == DEFAULT_DATE + timedelta(days=1)
session.rollback()

def test_do_schedule_max_active_runs_upstream_failed(self):
"""
Test that tasks in upstream failed don't count as actively running.
Expand Down

0 comments on commit 594069e

Please sign in to comment.