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

Prevent scheduler crash when serialized dag is missing #19113

Merged
merged 1 commit into from
Oct 20, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 8 additions & 6 deletions airflow/jobs/scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -765,7 +765,12 @@ def _do_scheduling(self, session) -> int:

# Send the callbacks after we commit to ensure the context is up to date when it gets run
for dag_run, callback_to_run in callback_tuples:
self._send_dag_callbacks_to_processor(dag_run, callback_to_run)
dag = self.dagbag.get_dag(dag_run.dag_id, session=session)
if not dag:
self.log.error("DAG '%s' not found in serialized_dag table", dag_run.dag_id)
continue

self._send_dag_callbacks_to_processor(dag, callback_to_run)

# Without this, the session has an invalid view of the DB
session.expunge_all()
Expand Down Expand Up @@ -989,7 +994,7 @@ def _schedule_dag_run(
)

# Send SLA & DAG Success/Failure Callbacks to be executed
self._send_dag_callbacks_to_processor(dag_run, callback_to_execute)
self._send_dag_callbacks_to_processor(dag, callback_to_execute)

return 0

Expand Down Expand Up @@ -1025,13 +1030,10 @@ def _verify_integrity_if_dag_changed(self, dag_run: DagRun, session=None):
# Verify integrity also takes care of session.flush
dag_run.verify_integrity(session=session)

def _send_dag_callbacks_to_processor(
self, dag_run: DagRun, callback: Optional[DagCallbackRequest] = None
):
def _send_dag_callbacks_to_processor(self, dag: DAG, callback: Optional[DagCallbackRequest] = None):
if not self.processor_agent:
raise ValueError("Processor agent is not started.")

dag = dag_run.get_dag()
self._send_sla_callbacks_to_processor(dag)
if callback:
self.processor_agent.send_callback_to_execute(callback)
Expand Down
4 changes: 1 addition & 3 deletions tests/jobs/test_scheduler_job.py
Original file line number Diff line number Diff line change
Expand Up @@ -1363,7 +1363,6 @@ def test_dagrun_callbacks_are_not_added_when_callbacks_are_not_defined(self, sta
self.scheduler_job._send_dag_callbacks_to_processor.assert_called_once()
call_args = self.scheduler_job._send_dag_callbacks_to_processor.call_args[0]
assert call_args[0].dag_id == dr.dag_id
assert call_args[0].execution_date == dr.execution_date
assert call_args[1] is None

session.rollback()
Expand Down Expand Up @@ -1394,11 +1393,10 @@ def test_dagrun_callbacks_are_added_when_callbacks_are_defined(self, state, msg,
with mock.patch.object(settings, "USE_JOB_SCHEDULE", False):
self.scheduler_job._do_scheduling(session)

# Verify Callback is not set (i.e is None) when no callbacks are set on DAG
# Verify Callback is set (i.e is None) when no callbacks are set on DAG
self.scheduler_job._send_dag_callbacks_to_processor.assert_called_once()
call_args = self.scheduler_job._send_dag_callbacks_to_processor.call_args[0]
assert call_args[0].dag_id == dr.dag_id
assert call_args[0].execution_date == dr.execution_date
assert call_args[1] is not None
assert call_args[1].msg == msg
session.rollback()
Expand Down