airflow-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jeremiah Lowin <jlo...@apache.org>
Subject Re: ExternalTaskSensor offsets with monthly jobs
Date Thu, 30 Jun 2016 21:00:21 GMT
Thanks Laura -- it sounds like you're describing the "push" version of
Adrian's "pull" workflow. I hadn't considered that one.

Adrian, if that doesn't work for you please keep an eye on:
https://github.com/apache/incubator-airflow/pull/1641

On Thu, Jun 30, 2016 at 12:22 PM Laura Lorenz <llorenz@industrydive.com>
wrote:

> I think I mentioned this on another similar thread and I think our use case
> might be somewhat similar. We have a daily ETL that loads data in to a
> database in one DAG, and then need to do a weekly rollup report every
> Tuesday that is in another DAG. The first DAG has a final
> TriggerDagRunOperator that decides if today is Tuesday or not, and if yes,
> triggers the weekly rollup DAG that operates on the data in the database at
> that moment - which, since it's Tuesday, is all the data we want. If that
> sounds like what you're trying to do, your first DAG might have a
> TriggerDagRunOperator that decides if today is the first of the month, and
> then triggers some other DAG.
>
> Laura
>
> On Thu, Jun 30, 2016 at 12:09 PM, Jeremiah Lowin <jlowin@apache.org>
> wrote:
>
> > Interesting -- this could be an extension of open enhancement AIRFLOW-100
> > https://issues.apache.org/jira/browse/AIRFLOW-100. Let me see if I can
> > restate this correctly:
> >
> > - You have a daily ETL job
> > - You have a monthly reporting job, for arguments sake lets say it runs
> on
> > the last day of each month with an execution date equal to the last day
> of
> > the prior month (for example on 7/31/2016 the task with execution date
> > 6/30/2016 will run).
> > You want the monthly job with execution date 6/30/2016 to wait for (and
> > include) the daily ETLs through 7/31/2016. In some months, that requires
> a
> > 31 day delta, in others 30 (in others 28... and forget about leap years).
> >
> > It sounds like the simplest solution (and the one proposed in A-100) is
> to
> > allow ExternalTaskSensor to accept not just a static delta, but
> potentially
> > a callable that accepts the current execution date and returns the
> desired
> > execution date for the sensed task. In this case, it would take in
> > 6/30/2016 and return 7/31/2016 as the last day of the following month. I
> > don't think any headway has been made on actually implementing the
> solution
> > but it should be straightforward -- I will try to get to it if I have
> some
> > time in the next few days.
> >
> >
> > On Wed, Jun 29, 2016 at 11:25 AM Adrian Bridgett <adrian@opensignal.com>
> > wrote:
> >
> > > I'm hitting a bit of an annoying problem and wondering about the best
> > > course of action.
> > >
> > > We have several dags:
> > > - a daily ETL job
> > > - several reporting jobs (daily, weekly or monthly) which use the data
> > > from previous ETL jobs
> > >
> > > I wish to have a dependency such that the reporting jobs depend upon
> the
> > > last ETL job that the report uses.   We're happy to set depends_on_past
> > > in the ETL job.
> > >
> > > Daily jobs are easy - ExternalTaskSensor, job done.
> > > Weekly jobs are a little trickier - we need to work out the
> > > execution_delta - normally +6 for us (we deliberately run a day late to
> > > prioritise other jobs).
> > > Monthly jobs.... this is where I'm struggling - how to work out the
> > > execution_delta.   I guess the ideal would be an upgrade from timedelta
> > > to dateutil.relativedelta?   tomorrow_ds and ds_add don't help either.
> > >
> > > I must admit, ds being the time that's just gone has caused me no end
> of
> > > brain befudledness, especially when trying to get the initial job right
> > > (so much so that I wrote this up in our DAG README, posting here for
> > > others):
> > >
> > > When adding a new job, it's critical to ensure that you've set the
> > > schedule correctly:
> > > - frequency (monthly, weekly, daily)
> > > - schedule_interval ("0 0 2 * *", "0 0 * * 0", "0 0 * * *")
> > > - start_date (choose a day that matches schedule_interval at least one
> > > interval ago)
> > > -- e.g if today is Thursday 2016-06-09, go back in time to when the
> > > schedule will trigger,
> > >     then work out what "ds" (execution date) would be (remembering
> > > that's the lapsed date)
> > > --- for a monthly job, last trigger=2016-06-02, ds=2016-05-02
> > > --- for a weekly job, last trigger=2016-06-05, ds=2016-05-29
> > > --- for a daily job, last trigger=2016-06-09, ds=2016-06-08
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message