From 077c505204ccc8fb10728652ed3851003204f321 Mon Sep 17 00:00:00 2001 From: Paul Mathew Date: Wed, 20 May 2026 03:06:39 -0400 Subject: [PATCH 1/2] Enforce execution_timeout in deferrable KubernetesPodOperator Closes: #67227 Co-authored-by: Cursor --- .../providers/amazon/aws/triggers/eks.py | 49 +----- .../cncf/kubernetes/operators/pod.py | 31 +++- .../providers/cncf/kubernetes/triggers/pod.py | 81 ++++++++- .../cncf/kubernetes/operators/test_pod.py | 145 +++++++++++++++- .../unit/cncf/kubernetes/triggers/test_pod.py | 160 ++++++++++++++++++ 5 files changed, 418 insertions(+), 48 deletions(-) diff --git a/providers/amazon/src/airflow/providers/amazon/aws/triggers/eks.py b/providers/amazon/src/airflow/providers/amazon/aws/triggers/eks.py index a386f96da1b26..1fc11c6be337f 100644 --- a/providers/amazon/src/airflow/providers/amazon/aws/triggers/eks.py +++ b/providers/amazon/src/airflow/providers/amazon/aws/triggers/eks.py @@ -16,7 +16,6 @@ # under the License. from __future__ import annotations -import datetime from typing import TYPE_CHECKING, Any from botocore.exceptions import ClientError @@ -29,8 +28,6 @@ from airflow.triggers.base import TriggerEvent if TYPE_CHECKING: - from pendulum import DateTime - from airflow.providers.amazon.aws.hooks.base_aws import AwsGenericHook @@ -118,49 +115,11 @@ def __init__( eks_cluster_name: str, aws_conn_id: str | None = None, region: str | None = None, - pod_name: str, - pod_namespace: str, - trigger_start_time: datetime.datetime, - base_container_name: str, - kubernetes_conn_id: str | None = None, - connection_extras: dict | None = None, - poll_interval: float = 2, - cluster_context: str | None = None, - config_dict: dict | None = None, - in_cluster: bool | None = None, - get_logs: bool = True, - startup_timeout: int = 120, - startup_check_interval: float = 5, - schedule_timeout: int = 120, - on_finish_action: str = "delete_pod", - on_kill_action: str = "delete_pod", - termination_grace_period: int | None = None, - last_log_time: DateTime | None = None, - logging_interval: int | None = None, - trigger_kwargs: dict | None = None, + **kwargs, ): - super().__init__( - pod_name=pod_name, - pod_namespace=pod_namespace, - trigger_start_time=trigger_start_time, - base_container_name=base_container_name, - kubernetes_conn_id=kubernetes_conn_id, - connection_extras=connection_extras, - poll_interval=poll_interval, - cluster_context=cluster_context, - config_dict=config_dict, - in_cluster=in_cluster, - get_logs=get_logs, - startup_timeout=startup_timeout, - startup_check_interval=startup_check_interval, - schedule_timeout=schedule_timeout, - on_finish_action=on_finish_action, - on_kill_action=on_kill_action, - termination_grace_period=termination_grace_period, - last_log_time=last_log_time, - logging_interval=logging_interval, - trigger_kwargs=trigger_kwargs, - ) + # Forward base-trigger kwargs through ``**kwargs`` rather than + # listing each one explicitly. + super().__init__(**kwargs) self.eks_cluster_name = eks_cluster_name self._aws_conn_id = aws_conn_id self.region = region diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py index 6e935290b5345..9a5040ffc3c80 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/operators/pod.py @@ -28,6 +28,7 @@ import re import shlex import string +import time from collections.abc import Callable, Container, Iterable, Mapping, Sequence from contextlib import AbstractContextManager, suppress from enum import Enum @@ -908,6 +909,28 @@ def invoke_defer_method( trigger_start_time = datetime.datetime.now(tz=datetime.timezone.utc) + # Translate ``execution_timeout`` into an absolute deadline plumbed + # into the trigger. Anchoring on ``ti.start_date`` keeps the deadline + # stable across re-deferrals (``logging_interval`` re-entries), since + # Airflow preserves the original ``start_date`` when a task resumes + # from defer. + execution_deadline: int | None = None + defer_timeout: datetime.timedelta | None = None + if self.execution_timeout is not None and context is not None: + ti_start_date = context["ti"].start_date + execution_deadline = int(ti_start_date.timestamp() + self.execution_timeout.total_seconds()) + # ``defer.timeout`` bounds the trigger's lifetime via the + # framework's ``trigger_timeout``. Clamp to a 60s minimum buffer: + # the trigger's first-iteration deadline check fires within + # ``poll_interval`` seconds and emits the operator-handled + # ``status="timeout"`` event, which runs ``_clean()`` and deletes + # the pod. + remaining = execution_deadline - time.time() + defer_timeout = max( + datetime.timedelta(seconds=remaining), + datetime.timedelta(seconds=60), + ) + trigger = KubernetesPodTrigger( pod_name=self.pod.metadata.name, # type: ignore[union-attr] pod_namespace=self.pod.metadata.namespace, # type: ignore[union-attr] @@ -929,6 +952,7 @@ def invoke_defer_method( last_log_time=last_log_time, logging_interval=self.logging_interval, trigger_kwargs=self.trigger_kwargs, + execution_deadline=execution_deadline, ) pod_container_state = trigger.define_pod_container_state(self.pod) if self.pod else None if context and ( @@ -949,7 +973,7 @@ def invoke_defer_method( }, ) else: - self.defer(trigger=trigger, method_name="trigger_reentry") + self.defer(trigger=trigger, method_name="trigger_reentry", timeout=defer_timeout) def trigger_reentry(self, context: Context, event: dict[str, Any]) -> Any: """ @@ -1018,8 +1042,13 @@ def trigger_reentry(self, context: Context, event: dict[str, Any]) -> Any: ) self.trigger_kwargs = dict(self.trigger_kwargs or {}) self.trigger_kwargs["_redefer_count"] = redefer_count + 1 + # Re-pass ``context`` so ``invoke_defer_method`` can recompute the + # ``execution_deadline`` for this re-deferral. ``ti.start_date`` is + # preserved across resumes, so the deadline stays anchored to the + # original task start. self.invoke_defer_method( last_log_time=last_log_time, + context=context, ) # invoke_defer_method raises TaskDeferred, execution does not continue here diff --git a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/triggers/pod.py b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/triggers/pod.py index 65b1b45bb04da..bb5a983687fd9 100644 --- a/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/triggers/pod.py +++ b/providers/cncf/kubernetes/src/airflow/providers/cncf/kubernetes/triggers/pod.py @@ -19,6 +19,7 @@ import asyncio import contextlib import datetime +import time import traceback from collections.abc import AsyncIterator from enum import Enum @@ -99,6 +100,13 @@ class KubernetesPodTrigger(BaseTrigger): the operator to print latest logs. If ``None`` will wait until container done. :param last_log_time: where to resume logs from :param trigger_kwargs: additional keyword parameters to send in the event + :param execution_deadline: Optional absolute timestamp (integer Unix epoch seconds) + after which the trigger emits a ``timeout`` event so the operator can fail the + task and delete the pod. Checked at the top of ``run()``, around + ``_wait_for_pod_start()``, and on every iteration of + ``_wait_for_container_completion()`` so the deadline is enforced regardless of + which phase the pod is in. Used to enforce ``execution_timeout`` semantics for + deferred tasks. """ def __init__( @@ -123,6 +131,7 @@ def __init__( last_log_time: DateTime | None = None, logging_interval: int | None = None, trigger_kwargs: dict | None = None, + execution_deadline: int | None = None, ): super().__init__() self.pod_name = pod_name @@ -145,6 +154,7 @@ def __init__( self.on_kill_action = OnKillAction(on_kill_action) self.termination_grace_period = termination_grace_period self.trigger_kwargs = trigger_kwargs or {} + self.execution_deadline = execution_deadline self._fired_event = False self._since_time = None @@ -173,6 +183,7 @@ def serialize(self) -> tuple[str, dict[str, Any]]: "last_log_time": self.last_log_time, "logging_interval": self.logging_interval, "trigger_kwargs": self.trigger_kwargs, + "execution_deadline": self.execution_deadline, }, ) @@ -184,8 +195,26 @@ async def run(self) -> AsyncIterator[TriggerEvent]: self.pod_namespace, self.poll_interval, ) + # Fast-path the timeout when the deadline has already elapsed before + # the trigger even starts (e.g. a long-paused triggerer queue, or a + # re-defer after the deadline passed). + if self.execution_deadline is not None and time.time() >= self.execution_deadline: + self._fired_event = True + yield TriggerEvent( + { + "status": "timeout", + "namespace": self.pod_namespace, + "name": self.pod_name, + "message": ( + f"Pod {self.pod_namespace}/{self.pod_name} reached the task's " + "execution_timeout deadline before the trigger could begin polling." + ), + **self.trigger_kwargs, + } + ) + return try: - state = await self._wait_for_pod_start() + state = await self._wait_for_pod_start_within_deadline() if state == ContainerState.TERMINATED: event = TriggerEvent( { @@ -272,6 +301,35 @@ def _format_exception_description(self, exc: Exception) -> Any: description += f"\ntrigger traceback:\n{curr_traceback}" return description + async def _wait_for_pod_start_within_deadline(self) -> ContainerState: + """ + Run ``_wait_for_pod_start`` bounded by ``execution_deadline``. + + Wraps the underlying call in :func:`asyncio.wait_for` when an + ``execution_deadline`` is set so the startup phase honours + ``execution_timeout`` too — otherwise a Pending pod would not time + out until ``startup_timeout`` (default 120s) regardless of how + short the user's ``execution_timeout`` was. On timeout we raise + :class:`PodLaunchTimeoutException` so the existing handler in + :meth:`run` emits the operator's expected ``status="timeout"`` + event. + """ + if self.execution_deadline is None: + return await self._wait_for_pod_start() + remaining = self.execution_deadline - time.time() + if remaining <= 0: + raise PodLaunchTimeoutException( + f"Pod {self.pod_namespace}/{self.pod_name} reached the task's " + "execution_timeout deadline before the pod left the Pending phase." + ) + try: + return await asyncio.wait_for(self._wait_for_pod_start(), timeout=remaining) + except asyncio.TimeoutError as exc: + raise PodLaunchTimeoutException( + f"Pod {self.pod_namespace}/{self.pod_name} reached the task's " + "execution_timeout deadline while waiting for the pod to start." + ) from exc + async def _wait_for_pod_start(self) -> ContainerState: """Loops until pod phase leaves ``PENDING`` If timeout is reached, throws error.""" pod = await self._get_pod() @@ -306,6 +364,27 @@ async def _wait_for_container_completion(self) -> TriggerEvent: if self.logging_interval is not None: time_get_more_logs = time_begin + datetime.timedelta(seconds=self.logging_interval) while True: + # ``execution_deadline`` is the operator's translation of the + # task-level ``execution_timeout`` into an absolute UTC timestamp + if self.execution_deadline is not None and time.time() >= self.execution_deadline: + self.log.info( + "Execution deadline reached for pod %s/%s — emitting timeout event.", + self.pod_namespace, + self.pod_name, + ) + return TriggerEvent( + { + "status": "timeout", + "namespace": self.pod_namespace, + "name": self.pod_name, + "message": ( + f"Pod {self.pod_namespace}/{self.pod_name} reached the task's " + "execution_timeout deadline." + ), + "last_log_time": self.last_log_time, + **self.trigger_kwargs, + } + ) pod = await self._get_pod() pod_container_state = self.define_pod_container_state(pod) if pod_container_state == ContainerState.TERMINATED: diff --git a/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/operators/test_pod.py b/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/operators/test_pod.py index 8f1307881071c..70a69142f4c96 100644 --- a/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/operators/test_pod.py +++ b/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/operators/test_pod.py @@ -39,7 +39,7 @@ _optionally_suppress, ) from airflow.providers.cncf.kubernetes.secret import Secret -from airflow.providers.cncf.kubernetes.triggers.pod import KubernetesPodTrigger +from airflow.providers.cncf.kubernetes.triggers.pod import ContainerState, KubernetesPodTrigger from airflow.providers.cncf.kubernetes.utils.pod_manager import ( OnFinishAction, PodLoggingStatus, @@ -50,6 +50,7 @@ from airflow.providers.common.compat.sdk import ( XCOM_RETURN_KEY, AirflowException, + AirflowNotFoundException, AirflowSkipException, TaskDeferred, ) @@ -2655,6 +2656,148 @@ def test_async_create_pod_should_throw_exception(self, mocked_hook, mocked_clean log_message = "Trigger emitted an %s event, failing the task: %s" mocked_log.error.assert_called_once_with(log_message, status, message) + @patch(KUB_OP_PATH.format("convert_config_file_to_dict")) + @patch("airflow.providers.cncf.kubernetes.operators.pod.BaseHook.get_connection") + def test_invoke_defer_method_passes_execution_deadline_when_execution_timeout_set( + self, mocked_get_connection, mocked_convert_config + ): + """ + ``execution_timeout`` is translated into an absolute ``execution_deadline`` + plumbed through to the trigger, and ``defer.timeout`` is set to the + remaining budget so the framework's ``trigger_timeout`` bounds the + trigger's lifetime as a backstop. Anchoring on ``ti.start_date`` keeps + the deadline stable across re-deferrals. + """ + mocked_get_connection.side_effect = AirflowNotFoundException("connection not found") + + execution_timeout = datetime.timedelta(seconds=300) + k = KubernetesPodOperator( + task_id=TEST_TASK_ID, + namespace=TEST_NAMESPACE, + image=TEST_IMAGE, + name=TEST_NAME, + on_finish_action="keep_pod", + in_cluster=True, + deferrable=True, + execution_timeout=execution_timeout, + ) + # Skip the pod-creation path and pretend it's already running. + k.pod = MagicMock() + k.pod.metadata.name = TEST_NAME + k.pod.metadata.namespace = TEST_NAMESPACE + + ti_mock = MagicMock() + ti_start = datetime.datetime(2026, 1, 1, 12, 0, 0, tzinfo=datetime.timezone.utc) + ti_mock.start_date = ti_start + context = {"ti": ti_mock} + + # Freeze time at ``ti_start + 30s``. With ``execution_timeout=300s`` the + # remaining budget is exactly ``270s``, so ``defer.timeout`` must equal + # that — not the 60s minimum-buffer clamp. + elapsed = datetime.timedelta(seconds=30) + with time_machine.travel(ti_start + elapsed, tick=False): + with patch( + f"{TRIGGER_CLASS}.define_pod_container_state", + return_value=ContainerState.RUNNING, + ): + with pytest.raises(TaskDeferred) as exc: + k.invoke_defer_method(context=context) + + trigger = exc.value.trigger + assert isinstance(trigger, KubernetesPodTrigger) + # Deadline = ti_start + execution_timeout (integer Unix seconds). + expected_deadline = int((ti_start + execution_timeout).timestamp()) + assert trigger.execution_deadline == expected_deadline + # remaining = execution_timeout - elapsed = 270s. + assert exc.value.timeout == execution_timeout - elapsed + + @patch(KUB_OP_PATH.format("convert_config_file_to_dict")) + @patch("airflow.providers.cncf.kubernetes.operators.pod.BaseHook.get_connection") + def test_invoke_defer_method_clamps_defer_timeout_to_minimum_buffer_when_deadline_close( + self, mocked_get_connection, mocked_convert_config + ): + """ + When the remaining budget is less than 60 seconds (or already past), + ``defer.timeout`` is clamped to a 60-second minimum so the trigger's + first-iteration deadline check has runway to emit its own + ``status="timeout"`` event before the framework's ``trigger_timeout`` + cancels the trigger and bypasses the operator's cleanup path. + """ + mocked_get_connection.side_effect = AirflowNotFoundException("connection not found") + + execution_timeout = datetime.timedelta(seconds=300) + k = KubernetesPodOperator( + task_id=TEST_TASK_ID, + namespace=TEST_NAMESPACE, + image=TEST_IMAGE, + name=TEST_NAME, + on_finish_action="keep_pod", + in_cluster=True, + deferrable=True, + execution_timeout=execution_timeout, + ) + k.pod = MagicMock() + k.pod.metadata.name = TEST_NAME + k.pod.metadata.namespace = TEST_NAMESPACE + + ti_mock = MagicMock() + ti_start = datetime.datetime(2026, 1, 1, 12, 0, 0, tzinfo=datetime.timezone.utc) + ti_mock.start_date = ti_start + context = {"ti": ti_mock} + + # Freeze time well past the deadline (ti_start + 600s) — remaining is + # negative. Without the minimum-buffer clamp, ``defer.timeout`` would be + # ``timedelta(0)`` and the framework would cancel the trigger before it + # could emit its own timeout event. + with time_machine.travel(ti_start + datetime.timedelta(seconds=600), tick=False): + with patch( + f"{TRIGGER_CLASS}.define_pod_container_state", + return_value=ContainerState.RUNNING, + ): + with pytest.raises(TaskDeferred) as exc: + k.invoke_defer_method(context=context) + + assert exc.value.timeout == datetime.timedelta(seconds=60) + + @patch(KUB_OP_PATH.format("convert_config_file_to_dict")) + @patch("airflow.providers.cncf.kubernetes.operators.pod.BaseHook.get_connection") + def test_invoke_defer_method_passes_no_deadline_when_execution_timeout_not_set( + self, mocked_get_connection, mocked_convert_config + ): + """ + When the operator has no ``execution_timeout``, ``execution_deadline`` is + ``None`` (no enforcement) and ``defer.timeout`` is also ``None`` — + preserving the pre-fix behaviour for tasks that don't opt in. + """ + mocked_get_connection.side_effect = AirflowNotFoundException("connection not found") + + k = KubernetesPodOperator( + task_id=TEST_TASK_ID, + namespace=TEST_NAMESPACE, + image=TEST_IMAGE, + name=TEST_NAME, + on_finish_action="keep_pod", + in_cluster=True, + deferrable=True, + ) + k.pod = MagicMock() + k.pod.metadata.name = TEST_NAME + k.pod.metadata.namespace = TEST_NAMESPACE + + ti_mock = MagicMock() + ti_mock.start_date = datetime.datetime(2026, 1, 1, 12, 0, 0, tzinfo=datetime.timezone.utc) + context = {"ti": ti_mock} + + with patch( + f"{TRIGGER_CLASS}.define_pod_container_state", + return_value=ContainerState.RUNNING, + ): + with pytest.raises(TaskDeferred) as exc: + k.invoke_defer_method(context=context) + + assert exc.value.trigger.execution_deadline is None + assert exc.value.timeout is None + @pytest.mark.parametrize( ("kwargs", "actual_exit_code", "expected_exc", "pod_status", "event_status"), [ diff --git a/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/triggers/test_pod.py b/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/triggers/test_pod.py index 765a3f35e3d4d..89037e9b3757d 100644 --- a/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/triggers/test_pod.py +++ b/providers/cncf/kubernetes/tests/unit/cncf/kubernetes/triggers/test_pod.py @@ -21,6 +21,7 @@ import contextlib import datetime import logging +import time from asyncio import Future from unittest import mock from unittest.mock import MagicMock @@ -133,8 +134,34 @@ def test_serialize(self, trigger): "last_log_time": None, "logging_interval": None, "trigger_kwargs": {}, + "execution_deadline": None, } + def test_serialize_with_execution_deadline(self): + """``execution_deadline`` round-trips through serialization.""" + deadline = 1_700_000_000.0 + trigger = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_dict=CONFIG_DICT, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + startup_check_interval=STARTUP_CHECK_INTERVAL_SECS, + schedule_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action=ON_FINISH_ACTION, + execution_deadline=deadline, + ) + + _, kwargs_dict = trigger.serialize() + + assert kwargs_dict["execution_deadline"] == deadline + def test_serialize_with_connection_extras(self): extras = {"token": "abc"} trigger = KubernetesPodTrigger( @@ -263,6 +290,139 @@ async def test_run_loop_return_failed_event(self, mock_hook, mock_method, mock_w assert actual_event == expected_event + @pytest.mark.asyncio + @mock.patch(f"{TRIGGER_PATH}._wait_for_pod_start") + @mock.patch(f"{TRIGGER_PATH}.define_pod_container_state") + @mock.patch(f"{TRIGGER_PATH}.hook") + async def test_run_loop_emits_timeout_event_when_execution_deadline_reached( + self, mock_hook, mock_define_state, mock_wait_pod + ): + """ + When ``execution_deadline`` (the operator's translation of + ``execution_timeout``) has already passed before the trigger picks + the task up, the trigger short-circuits at the top of ``run()`` and + emits a ``timeout`` event immediately instead of starting the wait + chain. The operator's existing terminal-event path then fails the + task and runs ``on_finish_action`` (pod delete). + """ + # Already-past deadline → ``run()`` short-circuit trips it. + past_deadline = 1 + trigger_with_deadline = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_dict=CONFIG_DICT, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + startup_check_interval=STARTUP_CHECK_INTERVAL_SECS, + schedule_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action=ON_FINISH_ACTION, + execution_deadline=past_deadline, + ) + # If the short-circuit fails, ``_wait_for_pod_start`` would be called + # next; making it RUNNING ensures the test fails loudly rather than + # accidentally emitting some other terminal event. + mock_wait_pod.return_value = ContainerState.RUNNING + mock_define_state.return_value = ContainerState.RUNNING + mock_hook.get_pod.return_value = self._mock_pod_result(mock.AsyncMock()) + + actual_event = await trigger_with_deadline.run().asend(None) + + # ``last_log_time`` is intentionally absent from this short-circuit and + # from the ``PodLaunchTimeoutException`` handler — both fire before any + # log fetching has happened. The mid-poll deadline check (in + # ``_wait_for_container_completion``) is the only timeout site that + # carries ``last_log_time`` because that's where it can be meaningfully + # populated. + assert actual_event.payload["status"] == "timeout" + assert actual_event.payload["namespace"] == NAMESPACE + assert actual_event.payload["name"] == POD_NAME + assert "execution_timeout" in actual_event.payload["message"] + + @pytest.mark.asyncio + @mock.patch(f"{TRIGGER_PATH}._wait_for_pod_start") + @mock.patch(f"{TRIGGER_PATH}.define_pod_container_state") + @mock.patch(f"{TRIGGER_PATH}.hook") + async def test_run_loop_does_not_emit_timeout_when_execution_deadline_not_reached( + self, mock_hook, mock_define_state, mock_wait_pod + ): + """ + When ``execution_deadline`` is still in the future, the trigger keeps + polling normally — proves the deadline check doesn't fire preemptively + on every run. + """ + # Far-future deadline (~year 2286) — guaranteed not reached. + future_deadline = 9_999_999_999 + trigger_with_deadline = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_dict=CONFIG_DICT, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + startup_check_interval=STARTUP_CHECK_INTERVAL_SECS, + schedule_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action=ON_FINISH_ACTION, + execution_deadline=future_deadline, + ) + mock_wait_pod.return_value = ContainerState.RUNNING + mock_define_state.return_value = ContainerState.RUNNING + mock_hook.get_pod.return_value = self._mock_pod_result(mock.AsyncMock()) + + # Trigger must keep waiting (not emit any event yet). Use ``asyncio.wait_for`` + # rather than a real ``asyncio.sleep`` to avoid wall-clock cost / flake risk. + with pytest.raises(asyncio.TimeoutError): + await asyncio.wait_for(trigger_with_deadline.run().__anext__(), timeout=0.05) + + @pytest.mark.asyncio + async def test_run_emits_timeout_when_deadline_passed_during_pod_startup(self): + """ + ``execution_deadline`` enforcement must cover the pod startup phase too, + not just ``_wait_for_container_completion``. If the deadline elapses + while the pod is still Pending, the trigger must emit a ``timeout`` + event rather than waiting out the (potentially much longer) + ``startup_timeout``. + """ + trigger_with_deadline = KubernetesPodTrigger( + pod_name=POD_NAME, + pod_namespace=NAMESPACE, + base_container_name=BASE_CONTAINER_NAME, + kubernetes_conn_id=CONN_ID, + poll_interval=POLL_INTERVAL, + cluster_context=CLUSTER_CONTEXT, + config_dict=CONFIG_DICT, + in_cluster=IN_CLUSTER, + get_logs=GET_LOGS, + startup_timeout=STARTUP_TIMEOUT_SECS, + startup_check_interval=STARTUP_CHECK_INTERVAL_SECS, + schedule_timeout=STARTUP_TIMEOUT_SECS, + trigger_start_time=TRIGGER_START_TIME, + on_finish_action=ON_FINISH_ACTION, + # Deadline crosses 1 second from now; ``_wait_for_pod_start`` is mocked + # to never return so only the deadline can break the wait. + execution_deadline=int(time.time()) + 1, + ) + + async def _hang_forever(): + await asyncio.sleep(60) + return ContainerState.RUNNING + + with mock.patch.object(trigger_with_deadline, "_wait_for_pod_start", _hang_forever): + actual_event = await trigger_with_deadline.run().asend(None) + + assert actual_event.payload["status"] == "timeout" + assert "execution_timeout" in actual_event.payload["message"] + @pytest.mark.asyncio @mock.patch(f"{TRIGGER_PATH}._wait_for_pod_start") @mock.patch(f"{TRIGGER_PATH}.hook") From 6eec8cf87e13418055ec511c3d7c6b2d85a90d00 Mon Sep 17 00:00:00 2001 From: Paul Mathew Date: Thu, 21 May 2026 15:29:27 -0400 Subject: [PATCH 2/2] Re-trigger CI after upstream fix #67268 lands on main Co-authored-by: Cursor