airflow-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jeremiah Lowin <jlo...@apache.org>
Subject Re: 1.7.1 release status
Date Fri, 06 May 2016 05:22:57 GMT
Tonight I was working with Dan on fixing the speed regression with large
DAGs: https://github.com/apache/incubator-airflow/pull/1470. That clears
the first blocker for 1.7.1 as described in AIRFLOW-52.

We wanted to ask for the group's thoughts on the second blocker. Basically,
the issue centers on this pattern:

```python
# email is an operator WITHOUT a DAG.
email = Operator(...)

# create dependencies for email
email.set_upstream(dag.roots)

# add email to the DAG
dag.add_task(email)
```

Why is this a problem? Under Airflow 1.7.0, this DAG is actually completely
broken after the set_upstream command, because it has a dependency to a
task that's not in the DAG. It can't be run and will even raise an
exception if you do something simple like access dag.roots. HOWEVER, this
building this broken DAG is allowed in 1.7.0 and the user cures it in the
last line by explicitly adding the task.

In https://github.com/apache/incubator-airflow/pull/1318, which will be
part of 1.7.1, I took steps that prevent users from creating broken DAGs at
all. The relevant fix in this case is that the email task would be
automatically added to the DAG (it would infer its membership from the
tasks in dag.roots). However, once that inference is made, the last line
becomes illegal, since you can't add a task to a DAG it's already in.

So here's the thing: because the last line becomes illegal, this code
snippet will no longer run under 1.7.1. My understanding is that it is
being used in production at Airbnb, so I wanted to raise the issue to see
if we can get comfortable with the change.

My opinion is that being able to build a broken DAG is *always* a bug, and
so the issue should be fixed even if that creates some incompatibilities
for anyone exploiting it. Particularly in this case, where the remedy is
simply to delete the last line.

We thought about deprecating the behavior, but I don't see how we can
because 1) we don't know for sure that the user is trying to do something
illegal at the time of the set_upstream call, and 2) the guard against
double-adding a task to a DAG has been in Airflow for a very long time, so
reverting it would constitute a really massive behavior change.

So my vote is to proceed with the fix, but as it could potentially
inconvenience the hand that feeds (and by "feeds" I mean "gave us Airflow")
I'd like to be sensitive to their needs.

J

On Thu, May 5, 2016 at 2:09 PM Dan Davydov <dan.davydov@airbnb.com.invalid>
wrote:

> Moved discussion to https://issues.apache.org/jira/browse/AIRFLOW-52 and
> updated the status of the task there.
>
> On Tue, May 3, 2016 at 2:32 AM, Dan Davydov <dan.davydov@airbnb.com>
> wrote:
>
> > It's per DAG unfortunately (we have some pretty funky DAGs here).
> > On May 2, 2016 10:26 PM, "Bolke de Bruin" <bdbruin@gmail.com> wrote:
> >
> >> Hi dan
> >>
> >> Is that per dag or per dag bag? Multiprocessing should parallelize dag
> >> parsing so I am very curious. Let me know if I can help out.
> >> Bolke
> >>
> >> Sent from my iPhone
> >>
> >> > On 3 mei 2016, at 01:47, Dan Davydov <dan.davydov@airbnb.com.INVALID>
> >> wrote:
> >> >
> >> > So a quick update, unfortunately we saw some DAGBag parsing time
> >> increases
> >> > (~10x for some DAGs) on the webservers with the 1.7.1rc3. Because of
> >> this I
> >> > will be working on a staging cluster that has a copy of our production
> >> > production DAGBag, and is a copy of our production airflow
> >> infrastructure,
> >> > just without the workers. This will let us debug the release outside
> of
> >> > production.
> >> >
> >> > On Thu, Apr 28, 2016 at 10:20 AM, Dan Davydov <dan.davydov@airbnb.com
> >
> >> > wrote:
> >> >
> >> >> Definitely, here were the issues we hit:
> >> >> - airbnb/airflow#1365 occured
> >> >> - Webservers/scheduler were timing out and stuck in restart cycles
> due
> >> to
> >> >> increased time spent on parsing DAGs due to airbnb/airflow#1213/files
> >> >> - Failed tasks that ran after the upgrade and the revert (after we
> >> >> reverted the upgrade) were unable to be cleared (but running the
> tasks
> >> >> through the UI worked without clearing them)
> >> >> - The way log files were stored on S3 was changed (airflow now
> >> requires a
> >> >> connection to be setup) which broke log storage
> >> >> - Some DAGs were broken (unable to be parsed) due to package
> >> >> reorganization in open-source (the import paths were changed) (the
> >> utils
> >> >> refactor commit)
> >> >>
> >> >> On Thu, Apr 28, 2016 at 12:17 AM, Bolke de Bruin <bdbruin@gmail.com>
> >> >> wrote:
> >> >>
> >> >>> Dan,
> >> >>>
> >> >>> Are you able to share some of the bugs you have been hitting and
> >> >>> connected commits?
> >> >>>
> >> >>> We could at the very least learn from them and maybe even improve
> >> testing.
> >> >>>
> >> >>> Bolke
> >> >>>
> >> >>>
> >> >>>>> Op 28 apr. 2016, om 06:51 heeft Dan Davydov
> >> >>>> <dan.davydov@airbnb.com.INVALID> het volgende geschreven:
> >> >>>>
> >> >>>> All of the blockers were fixed as of yesterday (there was some
> issue
> >> >>> that
> >> >>>> Jeremiah was looking at with the last release candidate which
I
> >> think is
> >> >>>> fixed but I'm not sure). I started staging the airbnb_1.7.1rc3
tag
> >> >>> earlier
> >> >>>> today, so as long as metrics look OK and the 1.7.1rc2 issues
seem
> >> >>> resolved
> >> >>>> tomorrow I will release internally either tomorrow or Monday
(we
> try
> >> to
> >> >>>> avoid releases on Friday). If there aren't any issues we can
push
> the
> >> >>> 1.7.1
> >> >>>> tag on Monday/Tuesday.
> >> >>>>
> >> >>>> @Sid
> >> >>>> I think we were originally aiming to deploy internally once
every
> two
> >> >>> weeks
> >> >>>> but we decided to do it once a month in the end. I'm not too
sure
> >> about
> >> >>>> that so Max can comment there.
> >> >>>>
> >> >>>> We have been running 1.7.0 in production for about a month
now and
> it
> >> >>>> stable.
> >> >>>>
> >> >>>> I think what really slowed down this release cycle is some
commits
> >> that
> >> >>>> caused severe bugs that we decided to roll-forward with instead
of
> >> >>> rolling
> >> >>>> back. We can potentially try reverting these commits next time
> while
> >> the
> >> >>>> fixes are applied for the next version, although this is not
always
> >> >>> trivial
> >> >>>> to do.
> >> >>>>
> >> >>>> On Wed, Apr 27, 2016 at 9:31 PM, Siddharth Anand <
> >> >>>> siddharthanand@yahoo.com.invalid> wrote:
> >> >>>>
> >> >>>>> Btw, is anyone of the committers running 1.7.0 or later
in any
> >> staging
> >> >>> or
> >> >>>>> production env? I have to say that given that 1.6.2 was
the most
> >> stable
> >> >>>>> release and is 4 or more months old does not say much for
our
> >> release
> >> >>>>> cadence or process. What's our plan for 1.7.1?
> >> >>>>>
> >> >>>>> Sent from Sid's iPhone
> >> >>>>>
> >> >>>>>>> On Apr 27, 2016, at 9:05 PM, Chris Riccomini <
> >> criccomini@apache.org>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>> Hey all,
> >> >>>>>>
> >> >>>>>> I just wanted to check in on the 1.7.1 release status.
I know
> there
> >> >>> have
> >> >>>>>> been some major-ish bugs, as well as several people
doing tests.
> >> >>> Should
> >> >>>>> we
> >> >>>>>> create a 1.7.1 release JIRA, and track outstanding
issues there?
> >> >>>>>>
> >> >>>>>> Cheers,
> >> >>>>>> Chris
> >> >>
> >>
> >
>

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