peachchen0716 commented on code in PR #64816:
URL: https://github.com/apache/airflow/pull/64816#discussion_r3069079608


##########
airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py:
##########
@@ -172,6 +172,20 @@ def ti_run(
     if ti.next_kwargs:
         data.pop("start_date")
         log.debug("Removed start_date from update as task is resuming from 
deferral")
+    elif "start_date" in data:
+        # Restore original start_date for rescheduled tasks. The supervisor 
always sends
+        # start_date=utcnow(), but for a sensor in reschedule mode the true 
start is when
+        # the first poke ran, recorded in TaskReschedule. Without this, every 
re-poke
+        # resets start_date and inflates dagrun.first_task_scheduling_delay.
+        first_reschedule_start_date = session.scalar(
+            select(TaskReschedule.start_date)
+            .where(TaskReschedule.ti_id == task_instance_id)
+            .order_by(TaskReschedule.id.asc())

Review Comment:
   The change is meant to use the first try's start time, not the latest try, 
as the TI start time as copilot suggests.
   The proposed order by also doesn't use proper index, since start_date is not 
indexed.



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