Hi all, I'm looking to get some clarity on the intended behavior for SLAs. This has come up several times in the past, but as far as I can tell there hasn't been a definitive answer. As pointed out in https://issues.apache.org/jira/browse/AIRFLOW-249 (open for several years now):
the SLA logic is only being fired after following_schedule + sla has elapsed, in other words one has to wait for the next TI before having a chance of getting any email. Also the email reports dag.following_schedule time (I guess because it is close of TI.start_date), but unfortunately that doesn't match what the task instances shows nor the log filename Example: Consider a TI from a @daily DAG with execution date of Monday at 00:00. It will start executing soon after Tuesday 00:00. If I set the SLA to 5 minutes, I would expect an SlaMiss to be created at Tuesday 00:05, but it's actually not created until *Wednesday* 00:05. I find this behavior very surprising, and it seems I'm not the only one (see [1], [2]). Can someone confirm whether this is really the desired behavior? I think removing a single line [3] from the manage_slas implementation would bring the behavior in line with what I expected - namely, that an SlaMiss will be created based on: execution_date + schedule_interval + sla ...as opposed to the current behavior of: execution_date + (2 * schedule_interval) + sla I'd be happy to open a PR for that if we reach consensus on the desired behavior. Thanks, Andrew [1] https://stackoverflow.com/questions/44071519/how-to-set-a-sla-in-airflow?rq=1 , [2] https://issues.apache.org/jira/browse/AIRFLOW-2781 [3] https://github.com/apache/incubator-airflow/blob/6afb12f0e5c18e8634daa0119d6e5797aa770b80/airflow/jobs/scheduler_job.py#L425