[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]

Re: Is `airflow backfill` disfunctional?


Thanks everyone for the feedback especially on the background for backfill.
After reading the discussion, I think it would be safest to add a flag for
auto rerun failed tasks for backfill with default to be false. I have
updated the pr accordingly.

Thanks a lot,
-Tao

On Wed, Jun 6, 2018 at 1:47 PM, Mark Whitfield <mark.whitfield@xxxxxxxxxxx>
wrote:

> I've been doing some work setting up a large, collaborative Airflow
> pipeline with a group that makes heavy use of backfills, and have been
> encountering a lot of these issues myself.
>
> Other gripes:
>
> Backfills do not obey concurrency pool restrictions. We had been making
> heavy use of SubDAGs and using concurrency pools to prevent deadlocks (why
> does the SubDAG itself even need to occupy a concurrency slot if none of
> its constituent tasks are running?), but this quickly became untenable when
> using backfills and we were forced to mostly abandon SubDAGs.
>
> Backfills do use DagRuns now, which is a big improvement. However, it's a
> common use case for us to add new tasks to a DAG and backfill to a date
> specific to that task. When we do this, the BackfillJob will pick up
> previous backfill DagRuns and re-use them, which is mostly nice because it
> keeps the Tree view neatly organized in the UI. However, it does not reset
> the start time of the DagRun when it does this. Combined with a DAG-level
> timeout, this means that the backfill job will activate a DagRun, but then
> the run will immediately time out (since it still thinks it's been running
> since the previous backfill). This will cause tasks to deadlock spuriously,
> making backfills extremely cumbersome to carry out.
>
> *Mark Whitfield*
> Data Scientist
> New York Times
>
>
> On Wed, Jun 6, 2018 at 3:33 PM Maxime Beauchemin <
> maximebeauchemin@xxxxxxxxx>
> wrote:
>
> > Thanks for the input, this is helpful.
> >
> > To add to the list, there's some complexity around concurrency management
> > and multiple executors:
> > I just hit this thing where backfill doesn't check DAG-level concurrency,
> > fires up 32 tasks, and `airlfow run` double-checks DAG-level concurrency
> > limit and exits. Right after backfill reschedules right away and so on,
> > burning a bunch of CPU doing nothing. In this specific case it seems like
> > `airflow run` should skip that specific check when in the context of a
> > backfill.
> >
> > Max
> >
> > On Tue, Jun 5, 2018 at 9:23 PM Bolke de Bruin <bdbruin@xxxxxxxxx> wrote:
> >
> > > Thinking out loud here, because it is a while back that I did work on
> > > backfills. There were some real issues with backfills:
> > >
> > > 1. Tasks were running in non deterministic order ending up in regular
> > > deadlocks
> > > 2. Didn’t create dag runs, making behavior inconsistent. Max dag runs
> > > could not be enforced. Ui could really display it, lots of minor other
> > > issues because of it.
> > > 3. Behavior was different from the scheduler, while subdagoperators
> > > particularly make use of backfills at the moment.
> > >
> > > I think with 3 the behavior you are observing crept in. And given 3 I
> > > would argue a consistent behavior between the scheduler and the
> backfill
> > > mechanism is still paramount. Thus we should explicitly clear tasks
> from
> > > failed if we want to rerun them. This at least until we move the
> > > subdagoperator out of backfill and into the scheduler (which is
> actually
> > > not too hard). Also we need those command line options anyway.
> > >
> > > Bolke
> > >
> > > Verstuurd vanaf mijn iPad
> > >
> > > > Op 6 jun. 2018 om 01:27 heeft Scott Halgrim <
> scott.halgrim@xxxxxxxxxx
> > .INVALID>
> > > het volgende geschreven:
> > > >
> > > > The request was for opposition, but I’d like to weigh in on the side
> of
> > > “it’s a better behavior [to have failed tasks re-run when cleared in a
> > > backfill"
> > > >> On Jun 5, 2018, 4:16 PM -0700, Maxime Beauchemin <
> > > maximebeauchemin@xxxxxxxxx>, wrote:
> > > >> @Jeremiah Lowin <jlowin@xxxxxxxxx> & @Bolke de Bruin <
> > bdbruin@xxxxxxxxx>
> > > I
> > > >> think you may have some context on why this may have changed at some
> > > point.
> > > >> I'm assuming that when DagRun handling was added to the backfill
> > logic,
> > > the
> > > >> behavior just happened to change to what it is now.
> > > >>
> > > >> Any opposition in moving back towards re-running failed tasks when
> > > starting
> > > >> a backfill? I think it's a better behavior, though it's a change in
> > > >> behavior that we should mention in UPDATE.md.
> > > >>
> > > >> One of our goals is to make sure that a failed or killed backfill
> can
> > be
> > > >> restarted and just seamlessly pick up where it left off.
> > > >>
> > > >> Max
> > > >>
> > > >>> On Tue, Jun 5, 2018 at 3:25 PM Tao Feng <fengtao04@xxxxxxxxx>
> wrote:
> > > >>>
> > > >>> After discussing with Max, we think it would be great if `airflow
> > > backfill`
> > > >>> could be able to auto pick up and rerun those failed tasks.
> > Currently,
> > > it
> > > >>> will throw exceptions(
> > > >>>
> > > >>>
> > >
> > https://github.com/apache/incubator-airflow/blob/master/airf
> low/jobs.py#L2489
> > > >>> )
> > > >>> without rerunning the failed tasks.
> > > >>>
> > > >>> But since it broke some of the previous assumptions for backfill,
> we
> > > would
> > > >>> like to get some feedback and see if anyone has any concerns(pr
> could
> > > be
> > > >>> found at https://github.com/apache/incu
> bator-airflow/pull/3464/files
> > ).
> > > >>>
> > > >>> Thanks,
> > > >>> -Tao
> > > >>>
> > > >>> On Thu, May 24, 2018 at 10:26 AM, Maxime Beauchemin <
> > > >>> maximebeauchemin@xxxxxxxxx> wrote:
> > > >>>
> > > >>>> So I'm running a backfill for what feels like the first time in
> > years
> > > >>> using
> > > >>>> a simple `airflow backfill --local` commands.
> > > >>>>
> > > >>>> First I start getting a ton of `logging.info` of each tasks that
> > > cannot
> > > >>> be
> > > >>>> started just yet at every tick flooding my terminal with the
> keyword
> > > >>>> `FAILED` in it, looking like a million of lines like this one:
> > > >>>>
> > > >>>> [2018-05-24 14:33:07,852] {models.py:1123} INFO - Dependencies not
> > met
> > > >>> for
> > > >>>> <TaskInstance: some_dag.some_task_id 2018-01-28 00:00:00
> > [scheduled]>,
> > > >>>> dependency 'Trigger Rule' FAILED: Task's trigger rule
> 'all_success'
> > re
> > > >>>> quires all upstream tasks to have succeeded, but found 1
> > > non-success(es).
> > > >>>> upstream_tasks_state={'successes': 0L, 'failed': 0L,
> > > 'upstream_failed':
> > > >>>> 0L,
> > > >>>> 'skipped': 0L, 'done': 0L}, upstream_task_ids=['some_other
> _task_id']
> > > >>>>
> > > >>>> Good thing I triggered 1 month and not 2 years like I actually
> need,
> > > just
> > > >>>> the logs here would be "big data". Now I'm unclear whether there's
> > > >>> anything
> > > >>>> actually running or if I did something wrong, so I decide to kill
> > the
> > > >>>> process so I can set a smaller date range and get a better picture
> > of
> > > >>>> what's up.
> > > >>>>
> > > >>>> I check my logging level, am I in DEBUG? Nope. Just INFO. So I
> take
> > a
> > > >>> note
> > > >>>> that I'll need to find that log-flooding line and demote it to
> DEBUG
> > > in a
> > > >>>> quick PR, no biggy.
> > > >>>>
> > > >>>> Now I restart with just a single schedule, and get an error `Dag
> > > >>> {some_dag}
> > > >>>> has reached maximum amount of 3 dag runs`. Hmmm, I wish backfill
> > could
> > > >>> just
> > > >>>> pickup where it left off. Maybe I need to run an `airflow clear`
> > > command
> > > >>>> and restart? Ok, ran my clear command, same error is showing up.
> > Dead
> > > >>> end.
> > > >>>>
> > > >>>> Maybe there is some new `airflow clear --reset-dagruns` option?
> > > Doesn't
> > > >>>> look like it... Maybe `airflow backfill` has some new switches to
> > > pick up
> > > >>>> where it left off? Can't find it. Am I supposed to clear the DAG
> > Runs
> > > >>>> manually in the UI? This is a pre-production, in-development DAG,
> so
> > > >>> it's
> > > >>>> not on the production web server. Am I supposed to fire up my own
> > web
> > > >>>> server to go and manually handle the backfill-related DAG Runs?
> > > Cannot to
> > > >>>> my staging MySQL and do manually clear some DAG runs?
> > > >>>>
> > > >>>> So. Fire up a web server, navigate to my dag_id, delete the DAG
> > runs,
> > > it
> > > >>>> appears I can finally start over.
> > > >>>>
> > > >>>> Next thought was: "Alright looks like I need to go Linus on the
> > > mailing
> > > >>>> list".
> > > >>>>
> > > >>>> What am I missing? I'm really hoping these issues specific to
> 1.8.2!
> > > >>>>
> > > >>>> Backfilling is core to Airflow and should work very well. I want
> to
> > > >>> restate
> > > >>>> some reqs for Airflow backfill:
> > > >>>> * when failing / interrupted, it should seamlessly be able to
> pickup
> > > >>> where
> > > >>>> it left off
> > > >>>> * terminal logging at the INFO level should be a clear, human
> > > consumable,
> > > >>>> indicator of progress
> > > >>>> * backfill-related operations (including restarts) should be
> doable
> > > >>> through
> > > >>>> CLI interactions, and not require web server interactions as the
> > > typical
> > > >>>> sandbox (dev environment) shouldn't assume the existence of a web
> > > server
> > > >>>>
> > > >>>> Let's fix this.
> > > >>>>
> > > >>>> Max
> > > >>>>
> > > >>>
> > >
> >
>