This is an automated email from the ASF dual-hosted git repository. potiuk pushed a commit to branch v2-1-test in repository https://gitbox.apache.org/repos/asf/airflow.git
commit 366aaf675632d9b3d2b98d4d620dbe2f02f771d3 Author: Alex Kennedy <[email protected]> AuthorDate: Sat Sep 18 13:18:32 2021 -0600 add sla_miss_callback section to the documentation (#18305) (cherry picked from commit dcfa14d60dade3fdefa001d10013466fe4d77f0d) --- airflow/models/dag.py | 8 +++--- docs/apache-airflow/concepts/dags.rst | 2 ++ docs/apache-airflow/concepts/tasks.rst | 48 +++++++++++++++++++++++++++++++++- 3 files changed, 54 insertions(+), 4 deletions(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index a1419fe..b483ecc 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -177,8 +177,10 @@ class DAG(LoggingMixin): is only enforced for scheduled DagRuns. :type dagrun_timeout: datetime.timedelta :param sla_miss_callback: specify a function to call when reporting SLA - timeouts. - :type sla_miss_callback: types.FunctionType + timeouts. See :ref:`sla_miss_callback<concepts:sla_miss_callback>` for + more information about the function signature and parameters that are + passed to the callback. + :type sla_miss_callback: callable :param default_view: Specify DAG default view (tree, graph, duration, gantt, landing_times), default tree :type default_view: str @@ -248,7 +250,7 @@ class DAG(LoggingMixin): concurrency: int = conf.getint('core', 'dag_concurrency'), max_active_runs: int = conf.getint('core', 'max_active_runs_per_dag'), dagrun_timeout: Optional[timedelta] = None, - sla_miss_callback: Optional[Callable] = None, + sla_miss_callback: Optional[Callable[["DAG", str, str, List[str], List[TaskInstance]], None]] = None, default_view: str = conf.get('webserver', 'dag_default_view').lower(), orientation: str = conf.get('webserver', 'dag_orientation'), catchup: bool = conf.getboolean('scheduler', 'catchup_by_default'), diff --git a/docs/apache-airflow/concepts/dags.rst b/docs/apache-airflow/concepts/dags.rst index 593eba6..2c8252f 100644 --- a/docs/apache-airflow/concepts/dags.rst +++ b/docs/apache-airflow/concepts/dags.rst @@ -15,6 +15,8 @@ specific language governing permissions and limitations under the License. +.. _concepts:dags: + DAGs ==== diff --git a/docs/apache-airflow/concepts/tasks.rst b/docs/apache-airflow/concepts/tasks.rst index 3988368..918443c 100644 --- a/docs/apache-airflow/concepts/tasks.rst +++ b/docs/apache-airflow/concepts/tasks.rst @@ -122,12 +122,58 @@ An SLA, or a Service Level Agreement, is an expectation for the maximum time a T Tasks over their SLA are not cancelled, though - they are allowed to run to completion. If you want to cancel a task after a certain runtime is reached, you want :ref:`concepts:timeouts` instead. -To set an SLA for a task, pass a ``datetime.timedelta`` object to the Task/Operator's ``sla`` parameter. You can also supply an ``sla_miss_callback`` that will be called when the SLA is missed if you want to run your own logic. +To set an SLA for a task, pass a ``datetime.timedelta`` object to the Task/Operator's ``sla`` parameter. You can also supply an ``sla_miss_callback`` that will be called when the SLA is missed if you want to run your own logic. If you want to disable SLA checking entirely, you can set ``check_slas = False`` in Airflow's ``[core]`` configuration. To read more about configuring the emails, see :doc:`/howto/email-config`. +.. _concepts:sla_miss_callback: + +sla_miss_callback +~~~~~~~~~~~~~~~~~ + +You can also supply an ``sla_miss_callback`` that will be called when the SLA is missed if you want to run your own logic. +The function signature of an ``sla_miss_callback`` requires 5 parameters. + +#. ``dag`` + + * Parent :ref:`DAG <concepts:dags>` Object for the :doc:`DAGRun </dag-run>` in which tasks missed their + :ref:`SLA <concepts:slas>`. + +#. ``task_list`` + + * String list (new-line separated, \\n) of all tasks that missed their :ref:`SLA <concepts:slas>` + since the last time that the ``sla_miss_callback`` ran. + +#. ``blocking_task_list`` + + * Any task in the :doc:`DAGRun(s)</dag-run>` (with the same ``execution_date`` as a task that missed + :ref:`SLA <concepts:slas>`) that is not in a **SUCCESS** state at the time that the ``sla_miss_callback`` + runs. i.e. 'running', 'failed'. These tasks are described as tasks that are blocking itself or another + task from completing before its SLA window is complete. + +#. ``slas`` + + * List of :py:mod:`SlaMiss<airflow.models.slamiss>` objects associated with the tasks in the + ``task_list`` parameter. + +#. ``blocking_tis`` + + * List of the :ref:`TaskInstance <concepts:task-instances>` objects that are associated with the tasks + in the ``blocking_task_list`` parameter. + +Examples of ``sla_miss_callback`` function signature: + +.. code-block:: python + + def my_sla_miss_callback(dag, task_list, blocking_task_list, slas, blocking_tis): + ... + +.. code-block:: python + + def my_sla_miss_callback(*args): + ... Special Exceptions ------------------
