dstandish commented on code in PR #43520:
URL: https://github.com/apache/airflow/pull/43520#discussion_r1835163879
##########
providers/src/airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py:
##########
@@ -443,6 +446,23 @@ def delete_pod(self, pod_name: str, namespace: str) ->
None:
if str(e.status) != "404":
raise
+ def patch_pod_delete_stuck(self, *, pod_name: str, namespace: str):
+ """Add a "done" annotation to ensure we don't continually adopt
pods."""
Review Comment:
fixed
##########
airflow/jobs/scheduler_job_runner.py:
##########
@@ -1796,28 +1808,95 @@ def _fail_tasks_stuck_in_queued(self, session: Session
= NEW_SESSION) -> None:
)
).all()
+ num_allowed_retries = conf.getint("scheduler",
"num_stuck_in_queued_retries")
for executor, stuck_tis in
self._executor_to_tis(tasks_stuck_in_queued).items():
- try:
- cleaned_up_task_instances =
set(executor.cleanup_stuck_queued_tasks(tis=stuck_tis))
- for ti in stuck_tis:
- if repr(ti) in cleaned_up_task_instances:
- self.log.warning(
- "Marking task instance %s stuck in queued as
failed. "
- "If the task instance has available retries, it
will be retried.",
- ti,
+ tis: Iterable[TaskInstance] = []
+ with suppress(NotImplementedError):
+ # BaseExecutor has "abstract" method
`cleanup_stuck_queued_tasks`
+ # We are tolerant of implementers not implementing it.
+ tis = executor.cleanup_stuck_queued_tasks(tis=stuck_tis)
+ for ti in tis:
+ if not isinstance(ti, TaskInstance):
+ # todo: when can we remove this?
+ # this is for backcompat. the pre-2.10.4 version of the
interface
+ # expected a string return val.
+ self.log.warning(
+ "Task instance %s stuck in queued. May be set to
failed.",
+ ti,
+ )
+ continue
+
+ self.log.warning("Task stuck in queued and may be requeued.
task_id=%s", ti.key)
+
+ num_times_stuck = self._get_num_times_stuck_in_queued(ti,
session)
+ if num_times_stuck < num_allowed_retries:
+ session.add(
+ Log(
+ event=TASK_STUCK_IN_QUEUED_RESCHEDULE_EVENT,
+ task_instance=ti.key,
+ extra=(
+ f"Task was in queued state for longer than
{self._task_queued_timeout} "
+ "seconds; task state will be set back to
scheduled."
+ ),
)
- session.add(
- Log(
- event="stuck in queued",
- task_instance=ti.key,
- extra=(
- "Task will be marked as failed. If the
task instance has "
- "available retries, it will be retried."
- ),
- )
+ )
+ with suppress(KeyError):
+ executor.running.remove(ti.key)
+ self._reschedule_stuck_task(ti)
+ else:
+ self.log.warning(
+ "Task requeue attempts exceeded max; marking failed.
task_instance=%s", ti
+ )
+ session.add(
+ Log(
+ event="stuck in queued tries exceeded",
+ task_instance=ti.key,
+ extra=(
+ f"Task was requeued more than
{num_allowed_retries} times "
+ "and will be failed."
+ ),
)
- except NotImplementedError:
- self.log.debug("Executor doesn't support cleanup of stuck
queued tasks. Skipping.")
+ )
+ executor.fail(ti.key)
+
+ @provide_session
+ def _reschedule_stuck_task(self, ti, session=NEW_SESSION):
+ session.execute(
+ update(TI)
+ .where(TI.filter_for_tis([ti]))
+ .values(
+ state=TaskInstanceState.SCHEDULED,
+ queued_dttm=None,
+ )
+ .execution_options(synchronize_session=False)
+ )
+
+ @provide_session
+ def _get_num_times_stuck_in_queued(self, ti: TaskInstance, session:
Session = NEW_SESSION) -> int:
+ """
+ Check the Log table to see how many times a taskinstance has been
stuck in queued.
+
+ We can then use this information to determine whether to reschedule a
task or fail it.
+ """
+ return (
+ session.query(Log)
+ .where(
+ Log.task_id == ti.task_id,
+ Log.dag_id == ti.dag_id,
+ Log.run_id == ti.run_id,
+ Log.map_index == ti.map_index,
+ Log.try_number == ti.try_number,
+ Log.event == TASK_STUCK_IN_QUEUED_RESCHEDULE_EVENT,
+ )
+ .count()
+ )
+
+ @provide_session
+ def _reset_task_instance(self, ti: TaskInstance, session: Session =
NEW_SESSION):
Review Comment:
fixed
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]