external_task.py 24 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530
  1. #
  2. # Licensed to the Apache Software Foundation (ASF) under one
  3. # or more contributor license agreements. See the NOTICE file
  4. # distributed with this work for additional information
  5. # regarding copyright ownership. The ASF licenses this file
  6. # to you under the Apache License, Version 2.0 (the
  7. # "License"); you may not use this file except in compliance
  8. # with the License. You may obtain a copy of the License at
  9. #
  10. # http://www.apache.org/licenses/LICENSE-2.0
  11. #
  12. # Unless required by applicable law or agreed to in writing,
  13. # software distributed under the License is distributed on an
  14. # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
  15. # KIND, either express or implied. See the License for the
  16. # specific language governing permissions and limitations
  17. # under the License.
  18. from __future__ import annotations
  19. import datetime
  20. import os
  21. import warnings
  22. from typing import TYPE_CHECKING, Any, Callable, Collection, Iterable
  23. import attr
  24. from airflow.configuration import conf
  25. from airflow.exceptions import AirflowException, AirflowSkipException, RemovedInAirflow3Warning
  26. from airflow.models.baseoperatorlink import BaseOperatorLink
  27. from airflow.models.dag import DagModel
  28. from airflow.models.dagbag import DagBag
  29. from airflow.models.taskinstance import TaskInstance
  30. from airflow.operators.empty import EmptyOperator
  31. from airflow.sensors.base import BaseSensorOperator
  32. from airflow.triggers.external_task import WorkflowTrigger
  33. from airflow.utils.file import correct_maybe_zipped
  34. from airflow.utils.helpers import build_airflow_url_with_query
  35. from airflow.utils.sensor_helper import _get_count, _get_external_task_group_task_ids
  36. from airflow.utils.session import NEW_SESSION, provide_session
  37. from airflow.utils.state import State, TaskInstanceState
  38. if TYPE_CHECKING:
  39. from sqlalchemy.orm import Session
  40. from airflow.models.baseoperator import BaseOperator
  41. from airflow.models.taskinstancekey import TaskInstanceKey
  42. from airflow.utils.context import Context
  43. class ExternalDagLink(BaseOperatorLink):
  44. """
  45. Operator link for ExternalTaskSensor and ExternalTaskMarker.
  46. It allows users to access DAG waited with ExternalTaskSensor or cleared by ExternalTaskMarker.
  47. """
  48. name = "External DAG"
  49. def get_link(self, operator: BaseOperator, *, ti_key: TaskInstanceKey) -> str:
  50. from airflow.models.renderedtifields import RenderedTaskInstanceFields
  51. ti = TaskInstance.get_task_instance(
  52. dag_id=ti_key.dag_id, run_id=ti_key.run_id, task_id=ti_key.task_id, map_index=ti_key.map_index
  53. )
  54. if TYPE_CHECKING:
  55. assert ti is not None
  56. template_fields = RenderedTaskInstanceFields.get_templated_fields(ti)
  57. external_dag_id = (
  58. template_fields["external_dag_id"] if template_fields else operator.external_dag_id # type: ignore[attr-defined]
  59. )
  60. query = {
  61. "dag_id": external_dag_id,
  62. "execution_date": ti.execution_date.isoformat(), # type: ignore[union-attr]
  63. }
  64. return build_airflow_url_with_query(query)
  65. class ExternalTaskSensor(BaseSensorOperator):
  66. """
  67. Waits for a different DAG, task group, or task to complete for a specific logical date.
  68. If both `external_task_group_id` and `external_task_id` are ``None`` (default), the sensor
  69. waits for the DAG.
  70. Values for `external_task_group_id` and `external_task_id` can't be set at the same time.
  71. By default, the ExternalTaskSensor will wait for the external task to
  72. succeed, at which point it will also succeed. However, by default it will
  73. *not* fail if the external task fails, but will continue to check the status
  74. until the sensor times out (thus giving you time to retry the external task
  75. without also having to clear the sensor).
  76. By default, the ExternalTaskSensor will not skip if the external task skips.
  77. To change this, simply set ``skipped_states=[TaskInstanceState.SKIPPED]``.
  78. Note that if you are monitoring multiple tasks, and one enters error state
  79. and the other enters a skipped state, then the external task will react to
  80. whichever one it sees first. If both happen together, then the failed state
  81. takes priority.
  82. It is possible to alter the default behavior by setting states which
  83. cause the sensor to fail, e.g. by setting ``allowed_states=[DagRunState.FAILED]``
  84. and ``failed_states=[DagRunState.SUCCESS]`` you will flip the behaviour to
  85. get a sensor which goes green when the external task *fails* and immediately
  86. goes red if the external task *succeeds*!
  87. Note that ``soft_fail`` is respected when examining the failed_states. Thus
  88. if the external task enters a failed state and ``soft_fail == True`` the
  89. sensor will _skip_ rather than fail. As a result, setting ``soft_fail=True``
  90. and ``failed_states=[DagRunState.SKIPPED]`` will result in the sensor
  91. skipping if the external task skips. However, this is a contrived
  92. example---consider using ``skipped_states`` if you would like this
  93. behaviour. Using ``skipped_states`` allows the sensor to skip if the target
  94. fails, but still enter failed state on timeout. Using ``soft_fail == True``
  95. as above will cause the sensor to skip if the target fails, but also if it
  96. times out.
  97. :param external_dag_id: The dag_id that contains the task you want to
  98. wait for. (templated)
  99. :param external_task_id: The task_id that contains the task you want to
  100. wait for. (templated)
  101. :param external_task_ids: The list of task_ids that you want to wait for. (templated)
  102. If ``None`` (default value) the sensor waits for the DAG. Either
  103. external_task_id or external_task_ids can be passed to
  104. ExternalTaskSensor, but not both.
  105. :param external_task_group_id: The task_group_id that contains the task you want to
  106. wait for. (templated)
  107. :param allowed_states: Iterable of allowed states, default is ``['success']``
  108. :param skipped_states: Iterable of states to make this task mark as skipped, default is ``None``
  109. :param failed_states: Iterable of failed or dis-allowed states, default is ``None``
  110. :param execution_delta: time difference with the previous execution to
  111. look at, the default is the same logical date as the current task or DAG.
  112. For yesterday, use [positive!] datetime.timedelta(days=1). Either
  113. execution_delta or execution_date_fn can be passed to
  114. ExternalTaskSensor, but not both.
  115. :param execution_date_fn: function that receives the current execution's logical date as the first
  116. positional argument and optionally any number of keyword arguments available in the
  117. context dictionary, and returns the desired logical dates to query.
  118. Either execution_delta or execution_date_fn can be passed to ExternalTaskSensor,
  119. but not both.
  120. :param check_existence: Set to `True` to check if the external task exists (when
  121. external_task_id is not None) or check if the DAG to wait for exists (when
  122. external_task_id is None), and immediately cease waiting if the external task
  123. or DAG does not exist (default value: False).
  124. :param poll_interval: polling period in seconds to check for the status
  125. :param deferrable: Run sensor in deferrable mode
  126. """
  127. template_fields = ["external_dag_id", "external_task_id", "external_task_ids", "external_task_group_id"]
  128. ui_color = "#4db7db"
  129. operator_extra_links = [ExternalDagLink()]
  130. def __init__(
  131. self,
  132. *,
  133. external_dag_id: str,
  134. external_task_id: str | None = None,
  135. external_task_ids: Collection[str] | None = None,
  136. external_task_group_id: str | None = None,
  137. allowed_states: Iterable[str] | None = None,
  138. skipped_states: Iterable[str] | None = None,
  139. failed_states: Iterable[str] | None = None,
  140. execution_delta: datetime.timedelta | None = None,
  141. execution_date_fn: Callable | None = None,
  142. check_existence: bool = False,
  143. poll_interval: float = 2.0,
  144. deferrable: bool = conf.getboolean("operators", "default_deferrable", fallback=False),
  145. **kwargs,
  146. ):
  147. super().__init__(**kwargs)
  148. self.allowed_states = list(allowed_states) if allowed_states else [TaskInstanceState.SUCCESS.value]
  149. self.skipped_states = list(skipped_states) if skipped_states else []
  150. self.failed_states = list(failed_states) if failed_states else []
  151. total_states = set(self.allowed_states + self.skipped_states + self.failed_states)
  152. if len(total_states) != len(self.allowed_states) + len(self.skipped_states) + len(self.failed_states):
  153. raise AirflowException(
  154. "Duplicate values provided across allowed_states, skipped_states and failed_states."
  155. )
  156. # convert [] to None
  157. if not external_task_ids:
  158. external_task_ids = None
  159. # can't set both single task id and a list of task ids
  160. if external_task_id is not None and external_task_ids is not None:
  161. raise ValueError(
  162. "Only one of `external_task_id` or `external_task_ids` may "
  163. "be provided to ExternalTaskSensor; "
  164. "use external_task_id or external_task_ids or external_task_group_id."
  165. )
  166. # since both not set, convert the single id to a 1-elt list - from here on, we only consider the list
  167. if external_task_id is not None:
  168. external_task_ids = [external_task_id]
  169. if external_task_group_id is not None and external_task_ids is not None:
  170. raise ValueError(
  171. "Only one of `external_task_group_id` or `external_task_ids` may "
  172. "be provided to ExternalTaskSensor; "
  173. "use external_task_id or external_task_ids or external_task_group_id."
  174. )
  175. # check the requested states are all valid states for the target type, be it dag or task
  176. if external_task_ids or external_task_group_id:
  177. if not total_states <= set(State.task_states):
  178. raise ValueError(
  179. "Valid values for `allowed_states`, `skipped_states` and `failed_states` "
  180. "when `external_task_id` or `external_task_ids` or `external_task_group_id` "
  181. f"is not `None`: {State.task_states}"
  182. )
  183. elif not total_states <= set(State.dag_states):
  184. raise ValueError(
  185. "Valid values for `allowed_states`, `skipped_states` and `failed_states` "
  186. f"when `external_task_id` and `external_task_group_id` is `None`: {State.dag_states}"
  187. )
  188. if execution_delta is not None and execution_date_fn is not None:
  189. raise ValueError(
  190. "Only one of `execution_delta` or `execution_date_fn` may "
  191. "be provided to ExternalTaskSensor; not both."
  192. )
  193. self.execution_delta = execution_delta
  194. self.execution_date_fn = execution_date_fn
  195. self.external_dag_id = external_dag_id
  196. self.external_task_id = external_task_id
  197. self.external_task_ids = external_task_ids
  198. self.external_task_group_id = external_task_group_id
  199. self.check_existence = check_existence
  200. self._has_checked_existence = False
  201. self.deferrable = deferrable
  202. self.poll_interval = poll_interval
  203. def _get_dttm_filter(self, context):
  204. if self.execution_delta:
  205. dttm = context["logical_date"] - self.execution_delta
  206. elif self.execution_date_fn:
  207. dttm = self._handle_execution_date_fn(context=context)
  208. else:
  209. dttm = context["logical_date"]
  210. return dttm if isinstance(dttm, list) else [dttm]
  211. @provide_session
  212. def poke(self, context: Context, session: Session = NEW_SESSION) -> bool:
  213. # delay check to poke rather than __init__ in case it was supplied as XComArgs
  214. if self.external_task_ids and len(self.external_task_ids) > len(set(self.external_task_ids)):
  215. raise ValueError("Duplicate task_ids passed in external_task_ids parameter")
  216. dttm_filter = self._get_dttm_filter(context)
  217. serialized_dttm_filter = ",".join(dt.isoformat() for dt in dttm_filter)
  218. if self.external_task_ids:
  219. self.log.info(
  220. "Poking for tasks %s in dag %s on %s ... ",
  221. self.external_task_ids,
  222. self.external_dag_id,
  223. serialized_dttm_filter,
  224. )
  225. if self.external_task_group_id:
  226. self.log.info(
  227. "Poking for task_group '%s' in dag '%s' on %s ... ",
  228. self.external_task_group_id,
  229. self.external_dag_id,
  230. serialized_dttm_filter,
  231. )
  232. if self.external_dag_id and not self.external_task_group_id and not self.external_task_ids:
  233. self.log.info(
  234. "Poking for DAG '%s' on %s ... ",
  235. self.external_dag_id,
  236. serialized_dttm_filter,
  237. )
  238. # In poke mode this will check dag existence only once
  239. if self.check_existence and not self._has_checked_existence:
  240. self._check_for_existence(session=session)
  241. count_failed = -1
  242. if self.failed_states:
  243. count_failed = self.get_count(dttm_filter, session, self.failed_states)
  244. # Fail if anything in the list has failed.
  245. if count_failed > 0:
  246. if self.external_task_ids:
  247. if self.soft_fail:
  248. raise AirflowSkipException(
  249. f"Some of the external tasks {self.external_task_ids} "
  250. f"in DAG {self.external_dag_id} failed. Skipping due to soft_fail."
  251. )
  252. raise AirflowException(
  253. f"Some of the external tasks {self.external_task_ids} "
  254. f"in DAG {self.external_dag_id} failed."
  255. )
  256. elif self.external_task_group_id:
  257. if self.soft_fail:
  258. raise AirflowSkipException(
  259. f"The external task_group '{self.external_task_group_id}' "
  260. f"in DAG '{self.external_dag_id}' failed. Skipping due to soft_fail."
  261. )
  262. raise AirflowException(
  263. f"The external task_group '{self.external_task_group_id}' "
  264. f"in DAG '{self.external_dag_id}' failed."
  265. )
  266. else:
  267. if self.soft_fail:
  268. raise AirflowSkipException(
  269. f"The external DAG {self.external_dag_id} failed. Skipping due to soft_fail."
  270. )
  271. raise AirflowException(f"The external DAG {self.external_dag_id} failed.")
  272. count_skipped = -1
  273. if self.skipped_states:
  274. count_skipped = self.get_count(dttm_filter, session, self.skipped_states)
  275. # Skip if anything in the list has skipped. Note if we are checking multiple tasks and one skips
  276. # before another errors, we'll skip first.
  277. if count_skipped > 0:
  278. if self.external_task_ids:
  279. raise AirflowSkipException(
  280. f"Some of the external tasks {self.external_task_ids} "
  281. f"in DAG {self.external_dag_id} reached a state in our states-to-skip-on list. Skipping."
  282. )
  283. elif self.external_task_group_id:
  284. raise AirflowSkipException(
  285. f"The external task_group '{self.external_task_group_id}' "
  286. f"in DAG {self.external_dag_id} reached a state in our states-to-skip-on list. Skipping."
  287. )
  288. else:
  289. raise AirflowSkipException(
  290. f"The external DAG {self.external_dag_id} reached a state in our states-to-skip-on list. "
  291. "Skipping."
  292. )
  293. # only go green if every single task has reached an allowed state
  294. count_allowed = self.get_count(dttm_filter, session, self.allowed_states)
  295. return count_allowed == len(dttm_filter)
  296. def execute(self, context: Context) -> None:
  297. """Run on the worker and defer using the triggers if deferrable is set to True."""
  298. if not self.deferrable:
  299. super().execute(context)
  300. else:
  301. self.defer(
  302. timeout=self.execution_timeout,
  303. trigger=WorkflowTrigger(
  304. external_dag_id=self.external_dag_id,
  305. external_task_group_id=self.external_task_group_id,
  306. external_task_ids=self.external_task_ids,
  307. execution_dates=self._get_dttm_filter(context),
  308. allowed_states=self.allowed_states,
  309. poke_interval=self.poll_interval,
  310. soft_fail=self.soft_fail,
  311. ),
  312. method_name="execute_complete",
  313. )
  314. def execute_complete(self, context, event=None):
  315. """Execute when the trigger fires - return immediately."""
  316. if event["status"] == "success":
  317. self.log.info("External tasks %s has executed successfully.", self.external_task_ids)
  318. elif event["status"] == "skipped":
  319. raise AirflowSkipException("External job has skipped skipping.")
  320. else:
  321. if self.soft_fail:
  322. raise AirflowSkipException("External job has failed skipping.")
  323. else:
  324. raise AirflowException(
  325. "Error occurred while trying to retrieve task status. Please, check the "
  326. "name of executed task and Dag."
  327. )
  328. def _check_for_existence(self, session) -> None:
  329. dag_to_wait = DagModel.get_current(self.external_dag_id, session)
  330. if not dag_to_wait:
  331. raise AirflowException(f"The external DAG {self.external_dag_id} does not exist.")
  332. if not os.path.exists(correct_maybe_zipped(dag_to_wait.fileloc)):
  333. raise AirflowException(f"The external DAG {self.external_dag_id} was deleted.")
  334. if self.external_task_ids:
  335. refreshed_dag_info = DagBag(dag_to_wait.fileloc).get_dag(self.external_dag_id)
  336. for external_task_id in self.external_task_ids:
  337. if not refreshed_dag_info.has_task(external_task_id):
  338. raise AirflowException(
  339. f"The external task {external_task_id} in "
  340. f"DAG {self.external_dag_id} does not exist."
  341. )
  342. if self.external_task_group_id:
  343. refreshed_dag_info = DagBag(dag_to_wait.fileloc).get_dag(self.external_dag_id)
  344. if not refreshed_dag_info.has_task_group(self.external_task_group_id):
  345. raise AirflowException(
  346. f"The external task group '{self.external_task_group_id}' in "
  347. f"DAG '{self.external_dag_id}' does not exist."
  348. )
  349. self._has_checked_existence = True
  350. def get_count(self, dttm_filter, session, states) -> int:
  351. """
  352. Get the count of records against dttm filter and states.
  353. :param dttm_filter: date time filter for execution date
  354. :param session: airflow session object
  355. :param states: task or dag states
  356. :return: count of record against the filters
  357. """
  358. warnings.warn(
  359. "This method is deprecated and will be removed in future.", DeprecationWarning, stacklevel=2
  360. )
  361. return _get_count(
  362. dttm_filter,
  363. self.external_task_ids,
  364. self.external_task_group_id,
  365. self.external_dag_id,
  366. states,
  367. session,
  368. )
  369. def get_external_task_group_task_ids(self, session, dttm_filter):
  370. warnings.warn(
  371. "This method is deprecated and will be removed in future.", DeprecationWarning, stacklevel=2
  372. )
  373. return _get_external_task_group_task_ids(
  374. dttm_filter, self.external_task_group_id, self.external_dag_id, session
  375. )
  376. def _handle_execution_date_fn(self, context) -> Any:
  377. """
  378. Handle backward compatibility.
  379. This function is to handle backwards compatibility with how this operator was
  380. previously where it only passes the execution date, but also allow for the newer
  381. implementation to pass all context variables as keyword arguments, to allow
  382. for more sophisticated returns of dates to return.
  383. """
  384. from airflow.utils.operator_helpers import make_kwargs_callable
  385. # Remove "logical_date" because it is already a mandatory positional argument
  386. logical_date = context["logical_date"]
  387. kwargs = {k: v for k, v in context.items() if k not in {"execution_date", "logical_date"}}
  388. # Add "context" in the kwargs for backward compatibility (because context used to be
  389. # an acceptable argument of execution_date_fn)
  390. kwargs["context"] = context
  391. if TYPE_CHECKING:
  392. assert self.execution_date_fn is not None
  393. kwargs_callable = make_kwargs_callable(self.execution_date_fn)
  394. return kwargs_callable(logical_date, **kwargs)
  395. class ExternalTaskMarker(EmptyOperator):
  396. """
  397. Use this operator to indicate that a task on a different DAG depends on this task.
  398. When this task is cleared with "Recursive" selected, Airflow will clear the task on
  399. the other DAG and its downstream tasks recursively. Transitive dependencies are followed
  400. until the recursion_depth is reached.
  401. :param external_dag_id: The dag_id that contains the dependent task that needs to be cleared.
  402. :param external_task_id: The task_id of the dependent task that needs to be cleared.
  403. :param execution_date: The logical date of the dependent task execution that needs to be cleared.
  404. :param recursion_depth: The maximum level of transitive dependencies allowed. Default is 10.
  405. This is mostly used for preventing cyclic dependencies. It is fine to increase
  406. this number if necessary. However, too many levels of transitive dependencies will make
  407. it slower to clear tasks in the web UI.
  408. """
  409. template_fields = ["external_dag_id", "external_task_id", "execution_date"]
  410. ui_color = "#4db7db"
  411. operator_extra_links = [ExternalDagLink()]
  412. # The _serialized_fields are lazily loaded when get_serialized_fields() method is called
  413. __serialized_fields: frozenset[str] | None = None
  414. def __init__(
  415. self,
  416. *,
  417. external_dag_id: str,
  418. external_task_id: str,
  419. execution_date: str | datetime.datetime | None = "{{ logical_date.isoformat() }}",
  420. recursion_depth: int = 10,
  421. **kwargs,
  422. ):
  423. super().__init__(**kwargs)
  424. self.external_dag_id = external_dag_id
  425. self.external_task_id = external_task_id
  426. if isinstance(execution_date, datetime.datetime):
  427. self.execution_date = execution_date.isoformat()
  428. elif isinstance(execution_date, str):
  429. self.execution_date = execution_date
  430. else:
  431. raise TypeError(
  432. f"Expected str or datetime.datetime type for execution_date. Got {type(execution_date)}"
  433. )
  434. if recursion_depth <= 0:
  435. raise ValueError("recursion_depth should be a positive integer")
  436. self.recursion_depth = recursion_depth
  437. @classmethod
  438. def get_serialized_fields(cls):
  439. """Serialize ExternalTaskMarker to contain exactly these fields + templated_fields ."""
  440. if not cls.__serialized_fields:
  441. cls.__serialized_fields = frozenset(super().get_serialized_fields() | {"recursion_depth"})
  442. return cls.__serialized_fields
  443. @attr.s(auto_attribs=True)
  444. class ExternalTaskSensorLink(ExternalDagLink):
  445. """
  446. This external link is deprecated.
  447. Please use :class:`airflow.sensors.external_task.ExternalDagLink`.
  448. """
  449. def __attrs_post_init__(self):
  450. warnings.warn(
  451. "This external link is deprecated. "
  452. "Please use :class:`airflow.sensors.external_task.ExternalDagLink`.",
  453. RemovedInAirflow3Warning,
  454. stacklevel=2,
  455. )