kaxil commented on code in PR #64076:
URL: https://github.com/apache/airflow/pull/64076#discussion_r2983719757


##########
airflow-core/src/airflow/jobs/scheduler_job_runner.py:
##########
@@ -2726,6 +2726,7 @@ def adopt_or_reset_orphaned_tasks(self, session: Session 
= NEW_SESSION) -> int:
                         .values(state=JobState.FAILED)
                     )
                     num_failed: int = getattr(result, "rowcount", 0)
+                    session.commit()  # Release any lock caused by flagging 
tasks

Review Comment:
   This commit introduces two production concerns:
   
   **1. Breaks retry atomicity.** This line is inside `run_with_db_retries` → 
`try ... except OperationalError: session.rollback(); raise` (lines 2718-2805). 
If the TI query, executor call, or flush fails with `OperationalError` after 
this commit, `session.rollback()` only undoes changes made *after* the commit — 
the Job UPDATE is permanent. The retry re-runs the block, but `num_failed` was 
already checked and `Stats.incr` (line 2740) already fired on the first 
attempt, so metrics double-count.
   
   **2. Does not address the reported symptom.** In the Slack thread, the 
report is *"workers cannot emit heartbeat via API because API server is blocked 
on a locked TI table."* This commit only releases `job` table locks. The 
`task_instance` row locks from `FOR UPDATE SKIP LOCKED` (line 2754) are still 
held through `executor.try_adopt_task_instances()` — which for CeleryExecutor 
makes synchronous network calls to the Celery backend 
(`bulk_state_fetcher.get_many()`). Those TI locks are what block the Execution 
API heartbeat endpoint (`SELECT ... FOR UPDATE` on the same TI rows). The Job 
lock release helps scheduler-to-scheduler contention but not the worker 
heartbeat problem described in the thread.



-- 
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]

Reply via email to