This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new b18b2e5  Add Note to SLA regarding schedule_interval (#19173)
b18b2e5 is described below

commit b18b2e502509fd492f65e18871b4db20f9e6cc4d
Author: john-jac <[email protected]>
AuthorDate: Tue Nov 2 15:50:46 2021 -0700

    Add Note to SLA regarding schedule_interval (#19173)
    
    This document entry is intended to make explicit the requirement that SLAs 
will only be triggered for tasks in that are part of a scheduled DAG run.
    
    Manually triggering DAGs with schedule_interval of None causes the error
    ```
      File 
"/home/airflow/.local/lib/python3.6/site-packages/airflow/dag_processing/processor.py",
 line 411, in manage_slas
        while dttm < timezone.utcnow():
    TypeError: '<' not supported between instances of 'NoneType' and 
'datetime.datetime'
    ```
    
    And manually triggering DAGs with a valid schedule_interval do not produce 
tasks that can invoke an SLA Miss.  Only scheduled DAGs will check tasks for 
SLA misses.
---
 airflow/dag_processing/processor.py    | 25 +++++++++++++++----------
 docs/apache-airflow/concepts/tasks.rst |  4 ++++
 2 files changed, 19 insertions(+), 10 deletions(-)

diff --git a/airflow/dag_processing/processor.py 
b/airflow/dag_processing/processor.py
index dd7937f..c9a9d08 100644
--- a/airflow/dag_processing/processor.py
+++ b/airflow/dag_processing/processor.py
@@ -414,16 +414,21 @@ class DagFileProcessor(LoggingMixin):
 
             sla_misses = []
             next_info = 
dag.next_dagrun_info(dag.get_run_data_interval(ti.dag_run), restricted=False)
-            while next_info.logical_date < ts:
-                next_info = dag.next_dagrun_info(next_info.data_interval, 
restricted=False)
-                if next_info.logical_date + task.sla < ts:
-                    sla_miss = SlaMiss(
-                        task_id=ti.task_id,
-                        dag_id=ti.dag_id,
-                        execution_date=next_info.logical_date,
-                        timestamp=ts,
-                    )
-                    sla_misses.append(sla_miss)
+            if next_info is None:
+                self.log.info("Skipping SLA check for %s because task does not 
have scheduled date", ti)
+            else:
+                while next_info.logical_date < ts:
+                    next_info = dag.next_dagrun_info(next_info.data_interval, 
restricted=False)
+                    if next_info is None:
+                        break
+                    if next_info.logical_date + task.sla < ts:
+                        sla_miss = SlaMiss(
+                            task_id=ti.task_id,
+                            dag_id=ti.dag_id,
+                            execution_date=next_info.logical_date,
+                            timestamp=ts,
+                        )
+                        sla_misses.append(sla_miss)
             if sla_misses:
                 session.add_all(sla_misses)
         session.commit()
diff --git a/docs/apache-airflow/concepts/tasks.rst 
b/docs/apache-airflow/concepts/tasks.rst
index efc7090..0f30dd1 100644
--- a/docs/apache-airflow/concepts/tasks.rst
+++ b/docs/apache-airflow/concepts/tasks.rst
@@ -163,6 +163,10 @@ If you want to disable SLA checking entirely, you can set 
``check_slas = False``
 
 To read more about configuring the emails, see :doc:`/howto/email-config`.
 
+.. note::
+
+    Only scheduled tasks will be checked against SLA. For example, manually 
triggered tasks will not invoke an SLA miss. For more information on 
``schedule_interval`` values see :doc:`DAG Run </dag-run>`.
+
 .. _concepts:sla_miss_callback:
 
 sla_miss_callback

Reply via email to