Agreed, dependencies between tasks that don't have DAGs make no sense, this should absolutely raise.
Thanks for following through on this! Max On Mon, May 9, 2016 at 9:02 PM, Jeremiah Lowin <[email protected]> wrote: > Max, appreciate the clarity on this. Dan and I added a PR today that warns > when users try to double-add tasks to a DAG, and that moves us in the right > direction while not breaking anything, so I think everybody's happy! > > There is one last issue that has come up -- it appears that at least one > Airbnb dagfile sets a relationship between two tasks when neither task has > a DAG. That creates an illegal situation and raises an error starting in > 1.7.1. If you think about it, it really makes no sense to have a dependency > without a DAG... DAGs are literally expressions of workflows, and tasks are > just the units of work! > > Dan and I looked to see if it could easily be deprecated and it turns out > it can't be. The issue is that when we set relationships between tasks, we > check to make sure we didn't introduce downstream cycles into the DAG. That > check uses the DAG to look up tasks (it actually turns out that you > authored the change here: > > https://github.com/apache/incubator-airflow/commit/6c1207b636de4d74c0faf804dc003918ae4a8b16#diff-a32a363fa616685db3bfefba947535b2R1772 > < > https://github.com/apache/incubator-airflow/commit/6c1207b636de4d74c0faf804dc003918ae4a8b16 > >). > In this case, there is no DAG at all -- therefore: error. > > So we are debating how to handle this. On the last issue, we were able to > deprecate it in a fairly unobtrusive way. But I really think this one > should stay an error -- it's an illegal situation at a very deep level. If > we introduce a special case, we're compromising the integrity of a core > check (detect_downstream_cycles) and possibly others. So, I submit that on > balance it's better in this case for Airbnb to change the offending DAG > than to build a backdoor into the cycles check. But again we're in that > grey area between "bug-fix" and "api-change" and I want to be sure you are > ok with it. Fortunately I think it's only the one DAG that would have to > change. > > Thanks, > Jeremiah > > On Fri, May 6, 2016 at 3:23 PM Maxime Beauchemin < > [email protected]> > wrote: > > > My thought was 2.0 should require tasks to be associated with a DAG on > > creation (maybe needs to be debated), 1.7.1 would **not** require this > but > > start warning in prevision of 2.0 > > > > Same with dag inference, we do start doing it in 1.7.1, but warn when we > > do, though it doesn't matter because of the previous statement, there > won't > > be inference if all tasks are associated. > > > > I have to share this analogy about inference I used yesterday. If you get > > in line, in a line of people that are in line to go see star wars, we can > > assume you're want to go see star wars. If you tell the next person in > line > > "I'm here because I want to see star wars", the person should just answer > > "well yeah, that's pretty obvious", not "of course you dummy now get off > > this line" (which would be the equivalent of raising) > > > > Max > > > > On Fri, May 6, 2016 at 9:54 AM, Jeremiah Lowin <[email protected]> > wrote: > > > > > The second point about moving add_task to a private method makes sense > to > > > me, and I love `set_as_last` and `set_as_first` for convenience. Here > are > > > my thoughts on the other two points: > > > > > > > > > * The absence of dag=dag (or equivalent context manager) WARNS about > > > deprecation in 2.0 > > > > > > - dag has never been a required argument for tasks (pre 1.7.1 or post) > so > > > I'm not sure what behavior we would be deprecating. The only difference > > is > > > that 1.7.1 takes additional care when working with dag-less tasks to > make > > > sure they are automatically added to any DAGs they interact with (via > > > set_upstream, for example). Otherwise you get in situations like the > > above > > > where a task in a DAG is allowed to depend on a task outside the DAG, > > > resulting in a broken DAG. > > > > > > > > > * set_upstream/downstream infers where needed, but WARNS about > > deprecation > > > of inference in 2.0 > > > > > > - inference is new in 1.7.1; previously, set_upstream could be called > on > > > any two tasks without validating that they were in compatible DAGs (or > > any > > > DAG for that matter). I don't think we should put off doing that > > > inference/validation until 2.0, since the current behavior clearly > allows > > > broken DAGs and should be fixed. To me, this doesn't represent an API > > > change that needs deprecation (especially since the API is identical), > > it's > > > a bug fix that prevents broken DAGs from being created. Maybe we have a > > > warning message that tells you if validation was done since that piece > > is a > > > new feature, but I don't see a reason to wait for 2.0 to do this > > > validation. The DAG is broken either way; in 1.7.0 you have to run it > to > > > find that out and in 1.7.1 we tell you about it right away. > > > > > > > > > On Fri, May 6, 2016 at 10:53 AM Maxime Beauchemin < > > > [email protected]> wrote: > > > > > > > I think that there are currently 4 ways to attach a task to a DAG: > > > > * Operator(task_id='foo', dag=dag) > > > > * dag.add_task(t) > > > > * Context manager: `with DAG('foo') as dag:` > > > > * inferred from set_upstream() and set_downstream in 1.7.1 (from my > > > > understanding) > > > > > > > > The pattern of passing the DAG object to BaseOperator constructor > > should > > > > probably be mandatory as this is how the `default_args` and `params` > > > magic > > > > happens and it may be unclear to users that it is the case. I'm > > assuming > > > > that the behavior of the context manager is equivalent to the > explictit > > > > dag=dag > > > > > > > > Knowing this, set_upstream shouldn't have to infer dag attribution, > but > > > > simply check that the dags on both sides is the same object. > > > > > > > > So the changes I'd suggest would be: > > > > * The absence of dag=dag (or equivalent context manager) WARNS about > > > > deprecation in 2.0 > > > > * DAG.add_task becomes private DAG._add_task and now WARNS about > > > > deprecation in 2.0 on usage, and also warns when reattributing a DAG > > > twice > > > > * set_upstream/downstream infers where needed, but WARNS about > > > deprecation > > > > of inference in 2.0 > > > > * Add new methods to replace the upstream(dag.root) pattern, the > terms > > > root > > > > is unclear (does it mean firsts or lasts?). SE I'd suggest > convenience > > > > methods `set_as_last` and `set_as_first` for clarity > > > > > > > > Max > > > > > > > > On Fri, May 6, 2016 at 6:32 AM, Bolke de Bruin <[email protected]> > > > wrote: > > > > > > > > > I agree with disabling building broken DAGs and also disabling / > not > > > > > supporting orphaned Operators/TaskInstances. It creates so many > > issues > > > > down > > > > > the line and the fixes are relatively easy. It will make airflow > and > > > the > > > > > used DAGs more maintainable. > > > > > > > > > > So +1, just make sure it is well documented in UPDATING.md > > > > > > > > > > my 2 cents. > > > > > > > > > > Bolke > > > > > > > > > > > Op 6 mei 2016, om 07:22 heeft Jeremiah Lowin <[email protected]> > > het > > > > > volgende geschreven: > > > > > > > > > > > > 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 > > > > > <[email protected]> > > > > > > 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 < > > [email protected] > > > > > > > > > >> wrote: > > > > > >> > > > > > >>> It's per DAG unfortunately (we have some pretty funky DAGs > here). > > > > > >>> On May 2, 2016 10:26 PM, "Bolke de Bruin" <[email protected]> > > > 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 <[email protected] > > > > .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 < > > > > > [email protected] > > > > > >>> > > > > > >>>>> 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 < > > > > [email protected] > > > > > > > > > > > >>>>>> 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 > > > > > >>>>>>>> <[email protected]> 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 < > > > > > >>>>>>>> [email protected]> 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 < > > > > > >>>> [email protected]> > > > > > >>>>>>>>>> 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 > > > > > >>>>>> > > > > > >>>> > > > > > >>> > > > > > >> > > > > > > > > > > > > > > > > > > > >
