Since we are talking about the SLA implementation, The current SLA miss
implementation is part of the scheduler code. So in the cases like
scheduler max out the process / not running for some reason, we will miss
all the SLA alert. It is worth to decouple SLA alert from the scheduler
path and run as a separate process.


Regards,
Ananth.P,






On 2 May 2018 at 20:31, David Capwell <[email protected]> wrote:

> We use SLA as well and works great for some DAGs and painful for others
>
> We rely on sensors to validate the data is ready before we run and each dag
> waits on sensors for different times (one dag waits for 8 hours since it
> expects date at the start of day but tends to get it 8 hours later).  We
> also have some nested dags that have about 10 tasks deep.
>
> In these two cases SLA warnings come very late since the semantics we see
> is DAG completion time; what we really want is what you were talking about,
> expected execution times
>
> Also SLA trigger on backfills and manual reruns of tasks
>
> I see this as a critical feature for production monitoring so would love to
> see this get improved
>
> On Wed, May 2, 2018, 12:00 PM James Meickle <[email protected]>
> wrote:
>
> > At Quantopian we use Airflow to produce artifacts based on the previous
> > day's stock market data. These artifacts are required for us to trade on
> > today's stock market. Therefore, I've been investing time in improving
> > Airflow notifications (such as writing PagerDuty and Slack integrations).
> > My attention has turned to Airflow's SLA system, which has some drawbacks
> > for our use case:
> >
> > 1) Airflow SLAs are not skip-aware, so a task that has an SLA but is
> > skipped for this execution date will still trigger emails/callbacks. This
> > is a huge problem for us because we run almost no tasks on weekends
> (since
> > the stock market isn't open).
> >
> > 2) Defining SLAs can be awkward because they are relative to the
> execution
> > date instead of the task start time. There's no way to alert if a task
> runs
> > for "more than an hour", for any non-trivial DAG. Instead you can only
> > express "more than an hour from execution date".  The financial data we
> use
> > varies in when it arrives, and how long it takes to process (data volume
> > changes frequently); we also have tight timelines that make retries
> > difficult, so we want to alert an operator while leaving the task
> running,
> > rather than failing and then alerting.
> >
> > 3) SLA miss emails don't have a subject line containing the instance URL
> > (important for us because we run the same DAGs in both
> staging/production)
> > or the execution date they apply to. When opened, they can get hard to
> read
> > for even a moderately sized DAG because they include a flat list of task
> > instances that are unsorted (neither alpha nor topo). They are also
> lacking
> > any links back to the Airflow instance.
> >
> > 4) SLA emails are not callbacks, and can't be turned off (other than
> either
> > removing the SLA or removing the email attribute on the task instance).
> The
> > way that SLA miss callbacks are defined is not intuitive, as in contrast
> to
> > all other callbacks, they are DAG-level rather than task-level. Also, the
> > call signature is poorly defined: for instance, two of the arguments are
> > just strings produced from the other two arguments.
> >
> > I have some thoughts about ways to fix these issues:
> >
> > 1) I just consider this one a bug. If a task instance is skipped, that
> was
> > intentional, and it should not trigger any alerts.
> >
> > 2) I think that the `sla=` parameter should be split into something like
> > this:
> >
> > `expected_start`: Timedelta after execution date, representing when this
> > task must have started by.
> > `expected_finish`: Timedelta after execution date, representing when this
> > task must have finished by.
> > `expected_duration`: Timedelta after task start, representing how long it
> > is expected to run including all retries.
> >
> > This would give better operator control over SLAs, particularly for tasks
> > deeper in larger DAGs where exact ordering may be hard to predict.
> >
> > 3) The emails should be improved to be more operator-friendly, and take
> > into account that someone may get a callback for a DAG they don't know
> very
> > well, or be paged by this notification.
> >
> > 4.1) All Airflow callbacks should support a list, rather than requiring a
> > single function. (I've written a wrapper that does this, but it would be
> > better for Airflow to just handle this itself.)
> >
> > 4.2) SLA miss callbacks should be task callbacks that receive context,
> like
> > all the other callbacks. Having a DAG figure out which tasks have missed
> > SLAs collectively is fine, but getting SLA failures in a batched callback
> > doesn't really make much sense. Per-task callbacks can be fired
> > individually within a batch of failures detected at the same time.
> >
> > 4.3) SLA emails should be the default SLA miss callback function, rather
> > than being hardcoded.
> >
> > Also, overall, the SLA miss logic is very complicated. It's stuffed into
> > one overloaded function that is responsible for checking for SLA misses,
> > creating database objects for them, filtering tasks, selecting emails,
> > rendering, and sending. Refactoring it would be a good maintainability
> win.
> >
> > I am already implementing some of the above in a private branch, but I'd
> be
> > curious to hear community feedback as to which of these suggestions might
> > be desirable upstream. I could have this ready for Airflow 2.0 if there
> is
> > interest beyond my own use case.
> >
>

Reply via email to