apache-airflow-providers-cncf-kubernetes 3.1.0__py3-none-any.whl → 10.10.0rc1__py3-none-any.whl
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- airflow/providers/cncf/kubernetes/__init__.py +18 -23
- airflow/providers/cncf/kubernetes/backcompat/__init__.py +17 -0
- airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py +31 -49
- airflow/providers/cncf/kubernetes/callbacks.py +200 -0
- airflow/providers/cncf/kubernetes/cli/__init__.py +16 -0
- airflow/providers/cncf/kubernetes/cli/kubernetes_command.py +195 -0
- airflow/providers/cncf/kubernetes/decorators/kubernetes.py +163 -0
- airflow/providers/cncf/kubernetes/decorators/kubernetes_cmd.py +118 -0
- airflow/providers/cncf/kubernetes/exceptions.py +37 -0
- airflow/providers/cncf/kubernetes/executors/__init__.py +17 -0
- airflow/providers/cncf/kubernetes/executors/kubernetes_executor.py +831 -0
- airflow/providers/cncf/kubernetes/executors/kubernetes_executor_types.py +91 -0
- airflow/providers/cncf/kubernetes/executors/kubernetes_executor_utils.py +736 -0
- airflow/providers/cncf/kubernetes/executors/local_kubernetes_executor.py +306 -0
- airflow/providers/cncf/kubernetes/get_provider_info.py +249 -50
- airflow/providers/cncf/kubernetes/hooks/kubernetes.py +846 -112
- airflow/providers/cncf/kubernetes/k8s_model.py +62 -0
- airflow/providers/cncf/kubernetes/kube_client.py +156 -0
- airflow/providers/cncf/kubernetes/kube_config.py +125 -0
- airflow/providers/cncf/kubernetes/kubernetes_executor_templates/__init__.py +16 -0
- airflow/providers/cncf/kubernetes/kubernetes_executor_templates/basic_template.yaml +79 -0
- airflow/providers/cncf/kubernetes/kubernetes_helper_functions.py +165 -0
- airflow/providers/cncf/kubernetes/operators/custom_object_launcher.py +368 -0
- airflow/providers/cncf/kubernetes/operators/job.py +646 -0
- airflow/providers/cncf/kubernetes/operators/kueue.py +132 -0
- airflow/providers/cncf/kubernetes/operators/pod.py +1417 -0
- airflow/providers/cncf/kubernetes/operators/resource.py +191 -0
- airflow/providers/cncf/kubernetes/operators/spark_kubernetes.py +336 -35
- airflow/providers/cncf/kubernetes/pod_generator.py +592 -0
- airflow/providers/cncf/kubernetes/pod_template_file_examples/__init__.py +16 -0
- airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_image_template.yaml +68 -0
- airflow/providers/cncf/kubernetes/pod_template_file_examples/dags_in_volume_template.yaml +74 -0
- airflow/providers/cncf/kubernetes/pod_template_file_examples/git_sync_template.yaml +95 -0
- airflow/providers/cncf/kubernetes/python_kubernetes_script.jinja2 +51 -0
- airflow/providers/cncf/kubernetes/python_kubernetes_script.py +92 -0
- airflow/providers/cncf/kubernetes/resource_convert/__init__.py +16 -0
- airflow/providers/cncf/kubernetes/resource_convert/configmap.py +52 -0
- airflow/providers/cncf/kubernetes/resource_convert/env_variable.py +39 -0
- airflow/providers/cncf/kubernetes/resource_convert/secret.py +40 -0
- airflow/providers/cncf/kubernetes/secret.py +128 -0
- airflow/providers/cncf/kubernetes/sensors/spark_kubernetes.py +30 -14
- airflow/providers/cncf/kubernetes/template_rendering.py +81 -0
- airflow/providers/cncf/kubernetes/triggers/__init__.py +16 -0
- airflow/providers/cncf/kubernetes/triggers/job.py +176 -0
- airflow/providers/cncf/kubernetes/triggers/pod.py +344 -0
- airflow/providers/cncf/kubernetes/utils/__init__.py +3 -0
- airflow/providers/cncf/kubernetes/utils/container.py +118 -0
- airflow/providers/cncf/kubernetes/utils/delete_from.py +154 -0
- airflow/providers/cncf/kubernetes/utils/k8s_resource_iterator.py +46 -0
- airflow/providers/cncf/kubernetes/utils/pod_manager.py +887 -152
- airflow/providers/cncf/kubernetes/utils/xcom_sidecar.py +25 -16
- airflow/providers/cncf/kubernetes/version_compat.py +38 -0
- apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info/METADATA +125 -0
- apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info/RECORD +62 -0
- {apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info → apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info}/WHEEL +1 -2
- apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info/entry_points.txt +3 -0
- apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info/licenses/NOTICE +5 -0
- airflow/providers/cncf/kubernetes/backcompat/pod.py +0 -119
- airflow/providers/cncf/kubernetes/backcompat/pod_runtime_info_env.py +0 -56
- airflow/providers/cncf/kubernetes/backcompat/volume.py +0 -62
- airflow/providers/cncf/kubernetes/backcompat/volume_mount.py +0 -58
- airflow/providers/cncf/kubernetes/example_dags/example_kubernetes.py +0 -163
- airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes.py +0 -66
- airflow/providers/cncf/kubernetes/example_dags/example_spark_kubernetes_spark_pi.yaml +0 -57
- airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py +0 -622
- apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info/METADATA +0 -452
- apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info/NOTICE +0 -6
- apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info/RECORD +0 -29
- apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info/entry_points.txt +0 -3
- apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info/top_level.txt +0 -1
- /airflow/providers/cncf/kubernetes/{example_dags → decorators}/__init__.py +0 -0
- {apache_airflow_providers_cncf_kubernetes-3.1.0.dist-info → apache_airflow_providers_cncf_kubernetes-10.10.0rc1.dist-info/licenses}/LICENSE +0 -0
|
@@ -0,0 +1,736 @@
|
|
|
1
|
+
# Licensed to the Apache Software Foundation (ASF) under one
|
|
2
|
+
# or more contributor license agreements. See the NOTICE file
|
|
3
|
+
# distributed with this work for additional information
|
|
4
|
+
# regarding copyright ownership. The ASF licenses this file
|
|
5
|
+
# to you under the Apache License, Version 2.0 (the
|
|
6
|
+
# "License"); you may not use this file except in compliance
|
|
7
|
+
# with the License. You may obtain a copy of the License at
|
|
8
|
+
#
|
|
9
|
+
# http://www.apache.org/licenses/LICENSE-2.0
|
|
10
|
+
#
|
|
11
|
+
# Unless required by applicable law or agreed to in writing,
|
|
12
|
+
# software distributed under the License is distributed on an
|
|
13
|
+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
|
14
|
+
# KIND, either express or implied. See the License for the
|
|
15
|
+
# specific language governing permissions and limitations
|
|
16
|
+
# under the License.
|
|
17
|
+
from __future__ import annotations
|
|
18
|
+
|
|
19
|
+
import contextlib
|
|
20
|
+
import json
|
|
21
|
+
import multiprocessing
|
|
22
|
+
import time
|
|
23
|
+
from queue import Empty, Queue
|
|
24
|
+
from typing import TYPE_CHECKING, Any, Literal, cast
|
|
25
|
+
|
|
26
|
+
from kubernetes import client, watch
|
|
27
|
+
from kubernetes.client.rest import ApiException
|
|
28
|
+
from urllib3.exceptions import ReadTimeoutError
|
|
29
|
+
|
|
30
|
+
from airflow.exceptions import AirflowException
|
|
31
|
+
from airflow.providers.cncf.kubernetes.backcompat import get_logical_date_key
|
|
32
|
+
from airflow.providers.cncf.kubernetes.executors.kubernetes_executor_types import (
|
|
33
|
+
ADOPTED,
|
|
34
|
+
ALL_NAMESPACES,
|
|
35
|
+
POD_EXECUTOR_DONE_KEY,
|
|
36
|
+
POD_REVOKED_KEY,
|
|
37
|
+
FailureDetails,
|
|
38
|
+
KubernetesJob,
|
|
39
|
+
KubernetesResults,
|
|
40
|
+
KubernetesWatch,
|
|
41
|
+
)
|
|
42
|
+
from airflow.providers.cncf.kubernetes.kube_client import get_kube_client
|
|
43
|
+
from airflow.providers.cncf.kubernetes.kubernetes_helper_functions import (
|
|
44
|
+
annotations_for_logging_task_metadata,
|
|
45
|
+
annotations_to_key,
|
|
46
|
+
create_unique_id,
|
|
47
|
+
)
|
|
48
|
+
from airflow.providers.cncf.kubernetes.pod_generator import PodGenerator, workload_to_command_args
|
|
49
|
+
from airflow.utils.log.logging_mixin import LoggingMixin
|
|
50
|
+
from airflow.utils.singleton import Singleton
|
|
51
|
+
from airflow.utils.state import TaskInstanceState
|
|
52
|
+
|
|
53
|
+
if TYPE_CHECKING:
|
|
54
|
+
from kubernetes.client import Configuration, models as k8s
|
|
55
|
+
|
|
56
|
+
|
|
57
|
+
class ResourceVersion(metaclass=Singleton):
|
|
58
|
+
"""Singleton for tracking resourceVersion from Kubernetes."""
|
|
59
|
+
|
|
60
|
+
resource_version: dict[str, str] = {}
|
|
61
|
+
|
|
62
|
+
|
|
63
|
+
class KubernetesJobWatcher(multiprocessing.Process, LoggingMixin):
|
|
64
|
+
"""Watches for Kubernetes jobs."""
|
|
65
|
+
|
|
66
|
+
def __init__(
|
|
67
|
+
self,
|
|
68
|
+
namespace: str,
|
|
69
|
+
watcher_queue: Queue[KubernetesWatch],
|
|
70
|
+
resource_version: str | None,
|
|
71
|
+
scheduler_job_id: str,
|
|
72
|
+
kube_config: Configuration,
|
|
73
|
+
):
|
|
74
|
+
super().__init__()
|
|
75
|
+
self.namespace = namespace
|
|
76
|
+
self.scheduler_job_id = scheduler_job_id
|
|
77
|
+
self.watcher_queue = watcher_queue
|
|
78
|
+
self.resource_version = resource_version
|
|
79
|
+
self.kube_config = kube_config
|
|
80
|
+
|
|
81
|
+
def run(self) -> None:
|
|
82
|
+
"""Perform watching."""
|
|
83
|
+
if TYPE_CHECKING:
|
|
84
|
+
assert self.scheduler_job_id
|
|
85
|
+
|
|
86
|
+
kube_client: client.CoreV1Api = get_kube_client()
|
|
87
|
+
while True:
|
|
88
|
+
try:
|
|
89
|
+
self.resource_version = self._run(
|
|
90
|
+
kube_client, self.resource_version, self.scheduler_job_id, self.kube_config
|
|
91
|
+
)
|
|
92
|
+
except ReadTimeoutError:
|
|
93
|
+
self.log.info("Kubernetes watch timed out waiting for events. Restarting watch.")
|
|
94
|
+
time.sleep(1)
|
|
95
|
+
except Exception:
|
|
96
|
+
self.log.exception("Unknown error in KubernetesJobWatcher. Failing")
|
|
97
|
+
self.resource_version = "0"
|
|
98
|
+
ResourceVersion().resource_version[self.namespace] = "0"
|
|
99
|
+
raise
|
|
100
|
+
else:
|
|
101
|
+
self.log.warning(
|
|
102
|
+
"Watch died gracefully, starting back up with: last resource_version: %s",
|
|
103
|
+
self.resource_version,
|
|
104
|
+
)
|
|
105
|
+
|
|
106
|
+
def _pod_events(self, kube_client: client.CoreV1Api, query_kwargs: dict):
|
|
107
|
+
watcher = watch.Watch()
|
|
108
|
+
try:
|
|
109
|
+
if self.namespace == ALL_NAMESPACES:
|
|
110
|
+
return watcher.stream(kube_client.list_pod_for_all_namespaces, **query_kwargs)
|
|
111
|
+
return watcher.stream(kube_client.list_namespaced_pod, self.namespace, **query_kwargs)
|
|
112
|
+
except ApiException as e:
|
|
113
|
+
if str(e.status) == "410": # Resource version is too old
|
|
114
|
+
if self.namespace == ALL_NAMESPACES:
|
|
115
|
+
pods = kube_client.list_pod_for_all_namespaces(watch=False)
|
|
116
|
+
else:
|
|
117
|
+
pods = kube_client.list_namespaced_pod(namespace=self.namespace, watch=False)
|
|
118
|
+
resource_version = pods.metadata.resource_version
|
|
119
|
+
query_kwargs["resource_version"] = resource_version
|
|
120
|
+
return self._pod_events(kube_client=kube_client, query_kwargs=query_kwargs)
|
|
121
|
+
raise
|
|
122
|
+
|
|
123
|
+
def _run(
|
|
124
|
+
self,
|
|
125
|
+
kube_client: client.CoreV1Api,
|
|
126
|
+
resource_version: str | None,
|
|
127
|
+
scheduler_job_id: str,
|
|
128
|
+
kube_config: Any,
|
|
129
|
+
) -> str | None:
|
|
130
|
+
self.log.info("Event: and now my watch begins starting at resource_version: %s", resource_version)
|
|
131
|
+
|
|
132
|
+
kwargs: dict[str, Any] = {
|
|
133
|
+
"label_selector": f"airflow-worker={scheduler_job_id},{POD_EXECUTOR_DONE_KEY}!=True",
|
|
134
|
+
}
|
|
135
|
+
if resource_version:
|
|
136
|
+
kwargs["resource_version"] = resource_version
|
|
137
|
+
if kube_config.kube_client_request_args:
|
|
138
|
+
for key, value in kube_config.kube_client_request_args.items():
|
|
139
|
+
kwargs[key] = value
|
|
140
|
+
|
|
141
|
+
last_resource_version: str | None = None
|
|
142
|
+
|
|
143
|
+
# For info about k8s timeout settings see
|
|
144
|
+
# https://github.com/kubernetes-client/python/blob/v29.0.0/examples/watch/timeout-settings.md
|
|
145
|
+
# and https://github.com/kubernetes-client/python/blob/v29.0.0/kubernetes/client/api_client.py#L336-L339
|
|
146
|
+
if "_request_timeout" not in kwargs:
|
|
147
|
+
kwargs["_request_timeout"] = 30
|
|
148
|
+
if "timeout_seconds" not in kwargs:
|
|
149
|
+
kwargs["timeout_seconds"] = 3600
|
|
150
|
+
|
|
151
|
+
logical_date_key = get_logical_date_key()
|
|
152
|
+
for event in self._pod_events(kube_client=kube_client, query_kwargs=kwargs):
|
|
153
|
+
task = event["object"]
|
|
154
|
+
self.log.debug("Event: %s had an event of type %s", task.metadata.name, event["type"])
|
|
155
|
+
if event["type"] == "ERROR":
|
|
156
|
+
return self.process_error(event)
|
|
157
|
+
annotations = task.metadata.annotations
|
|
158
|
+
task_instance_related_annotations = {
|
|
159
|
+
"dag_id": annotations["dag_id"],
|
|
160
|
+
"task_id": annotations["task_id"],
|
|
161
|
+
logical_date_key: annotations.get(logical_date_key),
|
|
162
|
+
"run_id": annotations.get("run_id"),
|
|
163
|
+
"try_number": annotations["try_number"],
|
|
164
|
+
}
|
|
165
|
+
map_index = annotations.get("map_index")
|
|
166
|
+
if map_index is not None:
|
|
167
|
+
task_instance_related_annotations["map_index"] = map_index
|
|
168
|
+
|
|
169
|
+
self.process_status(
|
|
170
|
+
pod_name=task.metadata.name,
|
|
171
|
+
namespace=task.metadata.namespace,
|
|
172
|
+
status=task.status.phase,
|
|
173
|
+
annotations=task_instance_related_annotations,
|
|
174
|
+
resource_version=task.metadata.resource_version,
|
|
175
|
+
event=event,
|
|
176
|
+
)
|
|
177
|
+
last_resource_version = task.metadata.resource_version
|
|
178
|
+
|
|
179
|
+
return last_resource_version
|
|
180
|
+
|
|
181
|
+
def process_error(self, event: Any) -> str:
|
|
182
|
+
"""Process error response."""
|
|
183
|
+
self.log.error("Encountered Error response from k8s list namespaced pod stream => %s", event)
|
|
184
|
+
raw_object = event["raw_object"]
|
|
185
|
+
if raw_object["code"] == 410:
|
|
186
|
+
self.log.info(
|
|
187
|
+
"Kubernetes resource version is too old, must reset to 0 => %s", (raw_object["message"],)
|
|
188
|
+
)
|
|
189
|
+
# Return resource version 0
|
|
190
|
+
return "0"
|
|
191
|
+
raise AirflowException(
|
|
192
|
+
f"Kubernetes failure for {raw_object['reason']} with code {raw_object['code']} and message: "
|
|
193
|
+
f"{raw_object['message']}"
|
|
194
|
+
)
|
|
195
|
+
|
|
196
|
+
def process_status(
|
|
197
|
+
self,
|
|
198
|
+
pod_name: str,
|
|
199
|
+
namespace: str,
|
|
200
|
+
status: str,
|
|
201
|
+
annotations: dict[str, str],
|
|
202
|
+
resource_version: str,
|
|
203
|
+
event: Any,
|
|
204
|
+
) -> None:
|
|
205
|
+
"""Process status response."""
|
|
206
|
+
pod = event["object"]
|
|
207
|
+
|
|
208
|
+
if POD_REVOKED_KEY in pod.metadata.labels.keys():
|
|
209
|
+
return
|
|
210
|
+
|
|
211
|
+
annotations_string = annotations_for_logging_task_metadata(annotations)
|
|
212
|
+
if event["type"] == "DELETED" and not pod.metadata.deletion_timestamp:
|
|
213
|
+
# This will happen only when the task pods are adopted by another executor.
|
|
214
|
+
# So, there is no change in the pod state.
|
|
215
|
+
# However, need to free the executor slot from the current executor.
|
|
216
|
+
self.log.info("Event: pod %s adopted, annotations: %s", pod_name, annotations_string)
|
|
217
|
+
self.watcher_queue.put(
|
|
218
|
+
KubernetesWatch(pod_name, namespace, ADOPTED, annotations, resource_version, None)
|
|
219
|
+
)
|
|
220
|
+
elif hasattr(pod.status, "reason") and pod.status.reason == "ProviderFailed":
|
|
221
|
+
# Most likely this happens due to Kubernetes setup (virtual kubelet, virtual nodes, etc.)
|
|
222
|
+
key = annotations_to_key(annotations=annotations)
|
|
223
|
+
task_key_str = f"{key.dag_id}.{key.task_id}.{key.try_number}" if key else "unknown"
|
|
224
|
+
self.log.warning(
|
|
225
|
+
"Event: %s failed to start with reason ProviderFailed, task: %s, annotations: %s",
|
|
226
|
+
pod_name,
|
|
227
|
+
task_key_str,
|
|
228
|
+
annotations_string,
|
|
229
|
+
)
|
|
230
|
+
self.watcher_queue.put(
|
|
231
|
+
KubernetesWatch(
|
|
232
|
+
pod_name,
|
|
233
|
+
namespace,
|
|
234
|
+
TaskInstanceState.FAILED,
|
|
235
|
+
annotations,
|
|
236
|
+
resource_version,
|
|
237
|
+
None,
|
|
238
|
+
)
|
|
239
|
+
)
|
|
240
|
+
elif status == "Pending":
|
|
241
|
+
# deletion_timestamp is set by kube server when a graceful deletion is requested.
|
|
242
|
+
if event["type"] == "DELETED" and pod.metadata.deletion_timestamp:
|
|
243
|
+
self.log.info("Event: Failed to start pod %s, annotations: %s", pod_name, annotations_string)
|
|
244
|
+
elif (
|
|
245
|
+
self.kube_config.worker_pod_pending_fatal_container_state_reasons
|
|
246
|
+
and "status" in event["raw_object"]
|
|
247
|
+
):
|
|
248
|
+
# Init containers and base container statuses to check.
|
|
249
|
+
# Skipping the other containers statuses check.
|
|
250
|
+
container_statuses_to_check = []
|
|
251
|
+
if "initContainerStatuses" in event["raw_object"]["status"]:
|
|
252
|
+
container_statuses_to_check.extend(event["raw_object"]["status"]["initContainerStatuses"])
|
|
253
|
+
if "containerStatuses" in event["raw_object"]["status"]:
|
|
254
|
+
container_statuses_to_check.append(event["raw_object"]["status"]["containerStatuses"][0])
|
|
255
|
+
for container_status in container_statuses_to_check:
|
|
256
|
+
container_status_state = container_status["state"]
|
|
257
|
+
if "waiting" in container_status_state:
|
|
258
|
+
if (
|
|
259
|
+
container_status_state["waiting"]["reason"]
|
|
260
|
+
in self.kube_config.worker_pod_pending_fatal_container_state_reasons
|
|
261
|
+
):
|
|
262
|
+
if (
|
|
263
|
+
container_status_state["waiting"]["reason"] == "ErrImagePull"
|
|
264
|
+
and container_status_state["waiting"]["message"] == "pull QPS exceeded"
|
|
265
|
+
):
|
|
266
|
+
continue
|
|
267
|
+
key = annotations_to_key(annotations=annotations)
|
|
268
|
+
task_key_str = (
|
|
269
|
+
f"{key.dag_id}.{key.task_id}.{key.try_number}" if key else "unknown"
|
|
270
|
+
)
|
|
271
|
+
self.log.warning(
|
|
272
|
+
"Event: %s has container %s with fatal reason %s, task: %s",
|
|
273
|
+
pod_name,
|
|
274
|
+
container_status["name"],
|
|
275
|
+
container_status_state["waiting"]["reason"],
|
|
276
|
+
task_key_str,
|
|
277
|
+
)
|
|
278
|
+
self.watcher_queue.put(
|
|
279
|
+
KubernetesWatch(
|
|
280
|
+
pod_name,
|
|
281
|
+
namespace,
|
|
282
|
+
TaskInstanceState.FAILED,
|
|
283
|
+
annotations,
|
|
284
|
+
resource_version,
|
|
285
|
+
None,
|
|
286
|
+
)
|
|
287
|
+
)
|
|
288
|
+
break
|
|
289
|
+
else:
|
|
290
|
+
self.log.info("Event: %s Pending, annotations: %s", pod_name, annotations_string)
|
|
291
|
+
else:
|
|
292
|
+
self.log.debug("Event: %s Pending, annotations: %s", pod_name, annotations_string)
|
|
293
|
+
elif status == "Failed":
|
|
294
|
+
# Collect failure details for failed pods
|
|
295
|
+
try:
|
|
296
|
+
failure_details = collect_pod_failure_details(pod, self.log)
|
|
297
|
+
except Exception as e:
|
|
298
|
+
self.log.warning(
|
|
299
|
+
"Failed to collect pod failure details for %s/%s: %s", namespace, pod_name, e
|
|
300
|
+
)
|
|
301
|
+
|
|
302
|
+
key = annotations_to_key(annotations=annotations)
|
|
303
|
+
task_key_str = f"{key.dag_id}.{key.task_id}.{key.try_number}" if key else "unknown"
|
|
304
|
+
self.log.warning(
|
|
305
|
+
"Event: %s Failed, task: %s, annotations: %s", pod_name, task_key_str, annotations_string
|
|
306
|
+
)
|
|
307
|
+
self.watcher_queue.put(
|
|
308
|
+
KubernetesWatch(
|
|
309
|
+
pod_name,
|
|
310
|
+
namespace,
|
|
311
|
+
TaskInstanceState.FAILED,
|
|
312
|
+
annotations,
|
|
313
|
+
resource_version,
|
|
314
|
+
failure_details,
|
|
315
|
+
)
|
|
316
|
+
)
|
|
317
|
+
elif status == "Succeeded":
|
|
318
|
+
self.log.info("Event: %s Succeeded, annotations: %s", pod_name, annotations_string)
|
|
319
|
+
self.watcher_queue.put(
|
|
320
|
+
KubernetesWatch(pod_name, namespace, None, annotations, resource_version, None)
|
|
321
|
+
)
|
|
322
|
+
elif status == "Running":
|
|
323
|
+
# deletion_timestamp is set by kube server when a graceful deletion is requested.
|
|
324
|
+
# since kube server have received request to delete pod set TI state failed
|
|
325
|
+
if event["type"] == "DELETED" and pod.metadata.deletion_timestamp:
|
|
326
|
+
self.log.info(
|
|
327
|
+
"Event: Pod %s deleted before it could complete, annotations: %s",
|
|
328
|
+
pod_name,
|
|
329
|
+
annotations_string,
|
|
330
|
+
)
|
|
331
|
+
self.watcher_queue.put(
|
|
332
|
+
KubernetesWatch(
|
|
333
|
+
pod_name,
|
|
334
|
+
namespace,
|
|
335
|
+
TaskInstanceState.FAILED,
|
|
336
|
+
annotations,
|
|
337
|
+
resource_version,
|
|
338
|
+
None,
|
|
339
|
+
)
|
|
340
|
+
)
|
|
341
|
+
else:
|
|
342
|
+
self.log.info("Event: %s is Running, annotations: %s", pod_name, annotations_string)
|
|
343
|
+
else:
|
|
344
|
+
self.log.warning(
|
|
345
|
+
"Event: Invalid state: %s on pod: %s in namespace %s with annotations: %s with "
|
|
346
|
+
"resource_version: %s",
|
|
347
|
+
status,
|
|
348
|
+
pod_name,
|
|
349
|
+
namespace,
|
|
350
|
+
annotations,
|
|
351
|
+
resource_version,
|
|
352
|
+
)
|
|
353
|
+
|
|
354
|
+
|
|
355
|
+
def collect_pod_failure_details(pod: k8s.V1Pod, logger) -> FailureDetails | None:
|
|
356
|
+
"""
|
|
357
|
+
Collect detailed failure information from a failed pod.
|
|
358
|
+
|
|
359
|
+
Analyzes both init containers and main containers to determine the root cause
|
|
360
|
+
of pod failure, prioritizing terminated containers with non-zero exit codes.
|
|
361
|
+
|
|
362
|
+
Args:
|
|
363
|
+
pod: The Kubernetes V1Pod object to analyze
|
|
364
|
+
logger: Logger instance to use for error logging
|
|
365
|
+
|
|
366
|
+
Returns:
|
|
367
|
+
FailureDetails dict with failure information, or None if no failure details found
|
|
368
|
+
"""
|
|
369
|
+
if not pod.status or pod.status.phase != "Failed":
|
|
370
|
+
return None
|
|
371
|
+
|
|
372
|
+
try:
|
|
373
|
+
# Basic pod-level information
|
|
374
|
+
failure_details: FailureDetails = {
|
|
375
|
+
"pod_status": getattr(pod.status, "phase", None),
|
|
376
|
+
"pod_reason": getattr(pod.status, "reason", None),
|
|
377
|
+
"pod_message": getattr(pod.status, "message", None),
|
|
378
|
+
}
|
|
379
|
+
|
|
380
|
+
# Check init containers first (they run before main containers)
|
|
381
|
+
container_failure = _analyze_init_containers(pod.status)
|
|
382
|
+
|
|
383
|
+
# If no init container failure found, check main containers
|
|
384
|
+
if not container_failure:
|
|
385
|
+
container_failure = _analyze_main_containers(pod.status)
|
|
386
|
+
|
|
387
|
+
# Merge container failure details
|
|
388
|
+
if container_failure:
|
|
389
|
+
failure_details.update(container_failure)
|
|
390
|
+
|
|
391
|
+
return failure_details
|
|
392
|
+
|
|
393
|
+
except Exception:
|
|
394
|
+
# Log unexpected exception for debugging
|
|
395
|
+
logger.exception(
|
|
396
|
+
"Unexpected error while collecting pod failure details for pod %s",
|
|
397
|
+
getattr(pod.metadata, "name", "unknown"),
|
|
398
|
+
)
|
|
399
|
+
# Return basic pod info if container analysis fails
|
|
400
|
+
return {
|
|
401
|
+
"pod_status": getattr(pod.status, "phase", None),
|
|
402
|
+
"pod_reason": getattr(pod.status, "reason", None),
|
|
403
|
+
"pod_message": getattr(pod.status, "message", None),
|
|
404
|
+
}
|
|
405
|
+
|
|
406
|
+
|
|
407
|
+
def _analyze_containers(
|
|
408
|
+
container_statuses: list[k8s.V1ContainerStatus] | None, container_type: Literal["init", "main"]
|
|
409
|
+
) -> FailureDetails | None:
|
|
410
|
+
"""Analyze container statuses for failure details."""
|
|
411
|
+
if not container_statuses:
|
|
412
|
+
return None
|
|
413
|
+
|
|
414
|
+
waiting_info: FailureDetails | None = None
|
|
415
|
+
|
|
416
|
+
for cs in container_statuses:
|
|
417
|
+
state_obj = cs.state
|
|
418
|
+
if state_obj.terminated:
|
|
419
|
+
terminated_reason = getattr(state_obj.terminated, "reason", None)
|
|
420
|
+
exit_code = getattr(state_obj.terminated, "exit_code", 0)
|
|
421
|
+
|
|
422
|
+
# Only treat as failure if exit code != 0 AND reason is not "Completed"
|
|
423
|
+
if exit_code != 0 and terminated_reason != "Completed":
|
|
424
|
+
return cast(
|
|
425
|
+
"FailureDetails",
|
|
426
|
+
{
|
|
427
|
+
"container_state": "terminated",
|
|
428
|
+
"container_reason": terminated_reason,
|
|
429
|
+
"container_message": getattr(state_obj.terminated, "message", None),
|
|
430
|
+
"exit_code": exit_code,
|
|
431
|
+
"container_type": container_type,
|
|
432
|
+
"container_name": getattr(cs, "name", "unknown"),
|
|
433
|
+
},
|
|
434
|
+
)
|
|
435
|
+
elif state_obj.waiting:
|
|
436
|
+
# Record waiting state but continue looking for terminated containers
|
|
437
|
+
waiting_info = cast(
|
|
438
|
+
"FailureDetails",
|
|
439
|
+
{
|
|
440
|
+
"container_state": "waiting",
|
|
441
|
+
"container_reason": getattr(state_obj.waiting, "reason", None),
|
|
442
|
+
"container_message": getattr(state_obj.waiting, "message", None),
|
|
443
|
+
"container_type": container_type,
|
|
444
|
+
"container_name": getattr(cs, "name", "unknown"),
|
|
445
|
+
},
|
|
446
|
+
)
|
|
447
|
+
|
|
448
|
+
# If we only found waiting containers, return the last one
|
|
449
|
+
return waiting_info
|
|
450
|
+
|
|
451
|
+
|
|
452
|
+
def _analyze_init_containers(pod_status: k8s.V1PodStatus) -> FailureDetails | None:
|
|
453
|
+
"""Analyze init container statuses for failure details."""
|
|
454
|
+
init_container_statuses = getattr(pod_status, "init_container_statuses", None)
|
|
455
|
+
return _analyze_containers(init_container_statuses, "init")
|
|
456
|
+
|
|
457
|
+
|
|
458
|
+
def _analyze_main_containers(pod_status: k8s.V1PodStatus) -> FailureDetails | None:
|
|
459
|
+
"""Analyze main container statuses for failure details."""
|
|
460
|
+
container_statuses = getattr(pod_status, "container_statuses", None)
|
|
461
|
+
return _analyze_containers(container_statuses, "main")
|
|
462
|
+
|
|
463
|
+
|
|
464
|
+
class AirflowKubernetesScheduler(LoggingMixin):
|
|
465
|
+
"""Airflow Scheduler for Kubernetes."""
|
|
466
|
+
|
|
467
|
+
def __init__(
|
|
468
|
+
self,
|
|
469
|
+
kube_config: Any,
|
|
470
|
+
result_queue: Queue[KubernetesResults],
|
|
471
|
+
kube_client: client.CoreV1Api,
|
|
472
|
+
scheduler_job_id: str,
|
|
473
|
+
):
|
|
474
|
+
super().__init__()
|
|
475
|
+
self.log.debug("Creating Kubernetes executor")
|
|
476
|
+
self.kube_config = kube_config
|
|
477
|
+
self.result_queue = result_queue
|
|
478
|
+
self.namespace = self.kube_config.kube_namespace
|
|
479
|
+
self.log.debug("Kubernetes using namespace %s", self.namespace)
|
|
480
|
+
self.kube_client = kube_client
|
|
481
|
+
self._manager = multiprocessing.Manager()
|
|
482
|
+
self.watcher_queue = self._manager.Queue()
|
|
483
|
+
self.scheduler_job_id = scheduler_job_id
|
|
484
|
+
self.kube_watchers = self._make_kube_watchers()
|
|
485
|
+
|
|
486
|
+
def run_pod_async(self, pod: k8s.V1Pod, **kwargs):
|
|
487
|
+
"""Run POD asynchronously."""
|
|
488
|
+
sanitized_pod = self.kube_client.api_client.sanitize_for_serialization(pod)
|
|
489
|
+
json_pod = json.dumps(sanitized_pod, indent=2)
|
|
490
|
+
|
|
491
|
+
self.log.debug("Pod Creation Request: \n%s", json_pod)
|
|
492
|
+
try:
|
|
493
|
+
resp = self.kube_client.create_namespaced_pod(
|
|
494
|
+
body=sanitized_pod, namespace=pod.metadata.namespace, **kwargs
|
|
495
|
+
)
|
|
496
|
+
self.log.debug("Pod Creation Response: %s", resp)
|
|
497
|
+
except Exception as e:
|
|
498
|
+
self.log.exception("Exception when attempting to create Namespaced Pod: %s", json_pod)
|
|
499
|
+
raise e
|
|
500
|
+
return resp
|
|
501
|
+
|
|
502
|
+
def _make_kube_watcher(self, namespace) -> KubernetesJobWatcher:
|
|
503
|
+
resource_version = ResourceVersion().resource_version.get(namespace, "0")
|
|
504
|
+
watcher = KubernetesJobWatcher(
|
|
505
|
+
watcher_queue=self.watcher_queue,
|
|
506
|
+
namespace=namespace,
|
|
507
|
+
resource_version=resource_version,
|
|
508
|
+
scheduler_job_id=self.scheduler_job_id,
|
|
509
|
+
kube_config=self.kube_config,
|
|
510
|
+
)
|
|
511
|
+
watcher.start()
|
|
512
|
+
return watcher
|
|
513
|
+
|
|
514
|
+
def _make_kube_watchers(self) -> dict[str, KubernetesJobWatcher]:
|
|
515
|
+
watchers = {}
|
|
516
|
+
if self.kube_config.multi_namespace_mode:
|
|
517
|
+
namespaces_to_watch = (
|
|
518
|
+
self.kube_config.multi_namespace_mode_namespace_list
|
|
519
|
+
if self.kube_config.multi_namespace_mode_namespace_list
|
|
520
|
+
else [ALL_NAMESPACES]
|
|
521
|
+
)
|
|
522
|
+
else:
|
|
523
|
+
namespaces_to_watch = [self.kube_config.kube_namespace]
|
|
524
|
+
|
|
525
|
+
for namespace in namespaces_to_watch:
|
|
526
|
+
watchers[namespace] = self._make_kube_watcher(namespace)
|
|
527
|
+
return watchers
|
|
528
|
+
|
|
529
|
+
def _health_check_kube_watchers(self):
|
|
530
|
+
for namespace, kube_watcher in self.kube_watchers.items():
|
|
531
|
+
if kube_watcher.is_alive():
|
|
532
|
+
self.log.debug("KubeJobWatcher for namespace %s alive, continuing", namespace)
|
|
533
|
+
else:
|
|
534
|
+
self.log.error(
|
|
535
|
+
(
|
|
536
|
+
"Error while health checking kube watcher process for namespace %s. "
|
|
537
|
+
"Process died for unknown reasons"
|
|
538
|
+
),
|
|
539
|
+
namespace,
|
|
540
|
+
)
|
|
541
|
+
ResourceVersion().resource_version[namespace] = "0"
|
|
542
|
+
self.kube_watchers[namespace] = self._make_kube_watcher(namespace)
|
|
543
|
+
|
|
544
|
+
def run_next(self, next_job: KubernetesJob) -> None:
|
|
545
|
+
"""Receives the next job to run, builds the pod, and creates it."""
|
|
546
|
+
key = next_job.key
|
|
547
|
+
command = next_job.command
|
|
548
|
+
kube_executor_config = next_job.kube_executor_config
|
|
549
|
+
pod_template_file = next_job.pod_template_file
|
|
550
|
+
|
|
551
|
+
dag_id, task_id, run_id, try_number, map_index = key
|
|
552
|
+
if len(command) == 1:
|
|
553
|
+
from airflow.executors.workloads import ExecuteTask
|
|
554
|
+
|
|
555
|
+
if isinstance(command[0], ExecuteTask):
|
|
556
|
+
workload = command[0]
|
|
557
|
+
command = workload_to_command_args(workload)
|
|
558
|
+
else:
|
|
559
|
+
raise ValueError(
|
|
560
|
+
f"KubernetesExecutor doesn't know how to handle workload of type: {type(command[0])}"
|
|
561
|
+
)
|
|
562
|
+
elif command[0:3] != ["airflow", "tasks", "run"]:
|
|
563
|
+
raise ValueError('The command must start with ["airflow", "tasks", "run"].')
|
|
564
|
+
|
|
565
|
+
base_worker_pod = get_base_pod_from_template(pod_template_file, self.kube_config)
|
|
566
|
+
|
|
567
|
+
if not base_worker_pod:
|
|
568
|
+
raise AirflowException(
|
|
569
|
+
f"could not find a valid worker template yaml at {self.kube_config.pod_template_file}"
|
|
570
|
+
)
|
|
571
|
+
|
|
572
|
+
pod = PodGenerator.construct_pod(
|
|
573
|
+
namespace=self.namespace,
|
|
574
|
+
scheduler_job_id=self.scheduler_job_id,
|
|
575
|
+
pod_id=create_unique_id(dag_id, task_id),
|
|
576
|
+
dag_id=dag_id,
|
|
577
|
+
task_id=task_id,
|
|
578
|
+
kube_image=self.kube_config.kube_image,
|
|
579
|
+
try_number=try_number,
|
|
580
|
+
map_index=map_index,
|
|
581
|
+
date=None,
|
|
582
|
+
run_id=run_id,
|
|
583
|
+
args=list(command),
|
|
584
|
+
pod_override_object=kube_executor_config,
|
|
585
|
+
base_worker_pod=base_worker_pod,
|
|
586
|
+
with_mutation_hook=True,
|
|
587
|
+
)
|
|
588
|
+
# Reconcile the pod generated by the Operator and the Pod
|
|
589
|
+
# generated by the .cfg file
|
|
590
|
+
self.log.info(
|
|
591
|
+
"Creating kubernetes pod for job is %s, with pod name %s, annotations: %s",
|
|
592
|
+
key,
|
|
593
|
+
pod.metadata.name,
|
|
594
|
+
annotations_for_logging_task_metadata(pod.metadata.annotations),
|
|
595
|
+
)
|
|
596
|
+
self.log.debug("Kubernetes running for command %s", command)
|
|
597
|
+
self.log.debug("Kubernetes launching image %s", pod.spec.containers[0].image)
|
|
598
|
+
|
|
599
|
+
# the watcher will monitor pods, so we do not block.
|
|
600
|
+
self.run_pod_async(pod, **self.kube_config.kube_client_request_args)
|
|
601
|
+
self.log.debug("Kubernetes Job created!")
|
|
602
|
+
|
|
603
|
+
def delete_pod(self, pod_name: str, namespace: str) -> None:
|
|
604
|
+
"""Delete Pod from a namespace; does not raise if it does not exist."""
|
|
605
|
+
try:
|
|
606
|
+
self.log.info("Deleting pod %s in namespace %s", pod_name, namespace)
|
|
607
|
+
self.kube_client.delete_namespaced_pod(
|
|
608
|
+
pod_name,
|
|
609
|
+
namespace,
|
|
610
|
+
body=client.V1DeleteOptions(**self.kube_config.delete_option_kwargs),
|
|
611
|
+
**self.kube_config.kube_client_request_args,
|
|
612
|
+
)
|
|
613
|
+
except ApiException as e:
|
|
614
|
+
# If the pod is already deleted
|
|
615
|
+
if str(e.status) != "404":
|
|
616
|
+
raise
|
|
617
|
+
|
|
618
|
+
def patch_pod_revoked(self, *, pod_name: str, namespace: str):
|
|
619
|
+
"""
|
|
620
|
+
Patch the pod with a label that ensures it's ignored by the kubernetes watcher.
|
|
621
|
+
|
|
622
|
+
:meta private:
|
|
623
|
+
"""
|
|
624
|
+
self.log.info(
|
|
625
|
+
"Patching pod %s in namespace %s to note that we are revoking the task.",
|
|
626
|
+
pod_name,
|
|
627
|
+
namespace,
|
|
628
|
+
)
|
|
629
|
+
try:
|
|
630
|
+
self.kube_client.patch_namespaced_pod(
|
|
631
|
+
name=pod_name,
|
|
632
|
+
namespace=namespace,
|
|
633
|
+
body={"metadata": {"labels": {POD_REVOKED_KEY: "True"}}},
|
|
634
|
+
)
|
|
635
|
+
except ApiException:
|
|
636
|
+
self.log.warning("Failed to patch pod %s with pod revoked key.", pod_name, exc_info=True)
|
|
637
|
+
|
|
638
|
+
def patch_pod_executor_done(self, *, pod_name: str, namespace: str):
|
|
639
|
+
"""Add a "done" annotation to ensure we don't continually adopt pods."""
|
|
640
|
+
self.log.debug("Patching pod %s in namespace %s to mark it as done", pod_name, namespace)
|
|
641
|
+
try:
|
|
642
|
+
self.kube_client.patch_namespaced_pod(
|
|
643
|
+
name=pod_name,
|
|
644
|
+
namespace=namespace,
|
|
645
|
+
body={"metadata": {"labels": {POD_EXECUTOR_DONE_KEY: "True"}}},
|
|
646
|
+
)
|
|
647
|
+
except ApiException as e:
|
|
648
|
+
self.log.info("Failed to patch pod %s with done annotation. Reason: %s", pod_name, e)
|
|
649
|
+
|
|
650
|
+
def sync(self) -> None:
|
|
651
|
+
"""
|
|
652
|
+
Check the status of all currently running kubernetes jobs.
|
|
653
|
+
|
|
654
|
+
If a job is completed, its status is placed in the result queue to be sent back to the scheduler.
|
|
655
|
+
"""
|
|
656
|
+
self.log.debug("Syncing KubernetesExecutor")
|
|
657
|
+
self._health_check_kube_watchers()
|
|
658
|
+
with contextlib.suppress(Empty):
|
|
659
|
+
while True:
|
|
660
|
+
task = self.watcher_queue.get_nowait()
|
|
661
|
+
try:
|
|
662
|
+
self.log.debug("Processing task %s", task)
|
|
663
|
+
self.process_watcher_task(task)
|
|
664
|
+
finally:
|
|
665
|
+
self.watcher_queue.task_done()
|
|
666
|
+
|
|
667
|
+
def process_watcher_task(self, task: KubernetesWatch) -> None:
|
|
668
|
+
"""Process the task by watcher."""
|
|
669
|
+
self.log.debug(
|
|
670
|
+
"Attempting to finish pod; pod_name: %s; state: %s; annotations: %s",
|
|
671
|
+
task.pod_name,
|
|
672
|
+
task.state,
|
|
673
|
+
annotations_for_logging_task_metadata(task.annotations),
|
|
674
|
+
)
|
|
675
|
+
key = annotations_to_key(annotations=task.annotations)
|
|
676
|
+
if key:
|
|
677
|
+
self.log.debug("finishing job %s - %s (%s)", key, task.state, task.pod_name)
|
|
678
|
+
self.result_queue.put(
|
|
679
|
+
KubernetesResults(
|
|
680
|
+
key,
|
|
681
|
+
task.state,
|
|
682
|
+
task.pod_name,
|
|
683
|
+
task.namespace,
|
|
684
|
+
task.resource_version,
|
|
685
|
+
task.failure_details,
|
|
686
|
+
)
|
|
687
|
+
)
|
|
688
|
+
|
|
689
|
+
def _flush_watcher_queue(self) -> None:
|
|
690
|
+
self.log.debug("Executor shutting down, watcher_queue approx. size=%d", self.watcher_queue.qsize())
|
|
691
|
+
with contextlib.suppress(Empty):
|
|
692
|
+
while True:
|
|
693
|
+
task = self.watcher_queue.get_nowait()
|
|
694
|
+
# Ignoring it since it can only have either FAILED or SUCCEEDED pods
|
|
695
|
+
self.log.warning("Executor shutting down, IGNORING watcher task=%s", task)
|
|
696
|
+
self.watcher_queue.task_done()
|
|
697
|
+
|
|
698
|
+
def terminate(self) -> None:
|
|
699
|
+
"""Terminates the watcher."""
|
|
700
|
+
self.log.debug("Terminating kube_watchers...")
|
|
701
|
+
for kube_watcher in self.kube_watchers.values():
|
|
702
|
+
kube_watcher.terminate()
|
|
703
|
+
self.log.debug("kube_watcher=%s", kube_watcher)
|
|
704
|
+
|
|
705
|
+
# for now 20 seconds is max wait time for kube watchers to terminate.
|
|
706
|
+
max_wait_time = 20
|
|
707
|
+
start_time = time.time()
|
|
708
|
+
for kube_watcher in self.kube_watchers.values():
|
|
709
|
+
kube_watcher.join(timeout=max(int(max_wait_time - (time.time() - start_time)), 0))
|
|
710
|
+
if kube_watcher.is_alive():
|
|
711
|
+
self.log.warning("kube_watcher didn't terminate in time=%s", kube_watcher)
|
|
712
|
+
kube_watcher.kill()
|
|
713
|
+
kube_watcher.join()
|
|
714
|
+
self.log.debug("kube_watcher=%s", kube_watcher)
|
|
715
|
+
self.log.debug("Flushing watcher_queue...")
|
|
716
|
+
self._flush_watcher_queue()
|
|
717
|
+
# Queue should be empty...
|
|
718
|
+
self.watcher_queue.join()
|
|
719
|
+
self.log.debug("Shutting down manager...")
|
|
720
|
+
self._manager.shutdown()
|
|
721
|
+
|
|
722
|
+
|
|
723
|
+
def get_base_pod_from_template(pod_template_file: str | None, kube_config: Any) -> k8s.V1Pod:
|
|
724
|
+
"""
|
|
725
|
+
Get base pod from template.
|
|
726
|
+
|
|
727
|
+
Reads either the pod_template_file set in the executor_config or the base pod_template_file
|
|
728
|
+
set in the airflow.cfg to craft a "base pod" that will be used by the KubernetesExecutor
|
|
729
|
+
|
|
730
|
+
:param pod_template_file: absolute path to a pod_template_file.yaml or None
|
|
731
|
+
:param kube_config: The KubeConfig class generated by airflow that contains all kube metadata
|
|
732
|
+
:return: a V1Pod that can be used as the base pod for k8s tasks
|
|
733
|
+
"""
|
|
734
|
+
if pod_template_file:
|
|
735
|
+
return PodGenerator.deserialize_model_file(pod_template_file)
|
|
736
|
+
return PodGenerator.deserialize_model_file(kube_config.pod_template_file)
|