All very reasonable to me, one reason we may not have hit the bugs in our production is because we are running off a different merge base and our cherries aren't 1-1 with what we are running in production (we still test them but we can't run them in production), that being said I don't think I authored the commits you are referring to so I don't have full context.
On Tue, Jan 3, 2017 at 1:27 PM, Bolke de Bruin <[email protected]> wrote: > Hi Dan et al, > > That sounds good to me, however I will be pretty critical of the changes > in the scheduler and the cleanliness of the patches. This is due to the > fact I have been chasing quite some bugs in master that were pretty hard to > track down even with a debugger at hand. I’m surprised that those didn’t > pop up in your production or maybe I am concerned ;-). Anyways, I hope you > understand I might be a bit picky in understanding and needing (design) > documentation for some of the changes. > > What I would like to suggest is that for the Alpha versions we still > accept “new” features so these PRs can get in, but from Beta we will not > accept new features anymore. For new features in the area of the scheduler > an integration DummyDag should be supplied, so others can test the > behaviour. Does this sound ok? > > My list of open code items for a release looks now like this: > > Blockers > * one_failed not honoured > * Alex’s sensor issue > > New features: > * Schedule all pending DAGs in a single loop > * Add support for backfill true/false > * Impersonation > * CGroups > * Add Cloud Storage updated sensor > > Alpha2 I will package tomorrow. Packages are signed now by my apache.org < > http://apache.org/> key. Please verify and let me know if something is > off. I’m still waiting for access to the incubating dist repository. > > Bolke > > > > On 3 Jan 2017, at 14:38, Dan Davydov <[email protected]> > wrote: > > > > I have also started on this effort, recently Alex Guziel and I have been > > pushing Airbnb's custom cherries onto master to get Airbnb back onto > master > > in order for us to do a release. > > > > I think it might make sense to wait for these two commits to get merged > in > > since they would be quite nice to have for all Airflow users and seem > like > > they will be merged soon: > > Schedule all pending DAG runs in a single scheduler loop - > > https://github.com/apache/incubator-airflow/pull/1906 < > https://github.com/apache/incubator-airflow/pull/1906> > > Add Support for dag.backfill=(True|False) Option - > > https://github.com/apache/incubator-airflow/pull/1830 < > https://github.com/apache/incubator-airflow/pull/1830> > > Impersonation Support + Cgroups - https://github.com/apache/ < > https://github.com/apache/> > > incubator-airflow/pull/1934 (this is kind of important from the Airbnb > side > > so that we can help test the new master without having to cherrypick this > > PR on top of it which would make the testing unreliable for others). > > > > If there are PRs that affect the core of Airflow that other committers > > think are important to merge we could include these too. I can commit to > > pushing out the Impersonation/Cgroups PR this week pending PR comments. > > What do you think Bolke? > > > > On Tue, Jan 3, 2017 at 4:26 AM, Bolke de Bruin <[email protected] > <mailto:[email protected]>> wrote: > > > >> Hey Alex, > >> > >> I have noticed the same, and it is also the reason why we have Alpha > >> versions. For now I have noticed the following: > >> > >> * Tasks can get in limbo between scheduler and executor: > >> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948> < > >> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948>> > >> * Try_number not increased due to reset in LocalTaskJob: > >> https://github.com/apache/incubator-airflow/pull/1969 < > https://github.com/apache/incubator-airflow/pull/1969> < > >> https://github.com/apache/incubator-airflow/pull/1969 < > https://github.com/apache/incubator-airflow/pull/1969>> > >> * one_failed trigger not executed > >> > >> My idea is to move to a Samba style of releases eventually, but for now > I > >> would like to get master into a state that we understand and therefore > not > >> accept any patches that do not address any bugs. > >> > >> If you (or anyone else) can review the above PRs and add your own as > well > >> then I can create another Alpha version. I’ll be on gitter as much as I > can > >> so we can speed up if needed. > >> > >> - Bolke > >> > >>> On 3 Jan 2017, at 08:51, Alex Van Boxel <[email protected]> wrote: > >>> > >>> Hey Bolke, > >>> > >>> thanks for getting this moving. But I already have some blockers, > since I > >>> moved up master to this release (moved from end November to now) > >> stability > >>> has gone down (certainly on Celary). I'm trying to identify the core > >>> problems and see if I can fix them. > >>> > >>> On Sat, Dec 31, 2016 at 9:52 PM Bolke de Bruin <[email protected] > >> <mailto:[email protected] <mailto:[email protected]>>> wrote: > >>> > >>> Dear All, > >>> > >>> On the verge of the New Year, I decided to be a little bit cheeky and > to > >>> make available an Airflow 1.8.0 Alpha 1. We have been talking about it > >> for > >>> a long time now and by doing this I wanted bootstrap the process. It > >> should > >>> by no means be considered an Apache release yet. This is for testing > >>> purposes in the dev community around Airflow, nothing else. > >>> > >>> The build is exactly the same as the state of master (git 410736d) plus > >> the > >>> change to version “1.8.0.alpha1” in version.py. > >>> > >>> I am dedicating quite some time next week and beyond to get a release > >> out. > >>> Hopefully we can get some help with testing, changelog etc. To make > this > >>> possible I would like to propose a freeze to adding new features for at > >>> least two weeks - say until Jan 15. > >>> > >>> You can find the tar here: http://people.apache.org/~bolke/ < > http://people.apache.org/~bolke/> < > >>> http://people.apache.org/~bolke/ <http://people.apache.org/~bolke/> < > http://people.apache.org/~bolke/ <http://people.apache.org/~bolke/>>> . > >> It isn’t signed. Following versions > >>> will be. SHA is available. > >>> > >>> Lastly, Alpha 1 does not have the fix for retries yet. So we will get > an > >>> Alpha 2 :-). @Max / @Dan / @Paul: a potential fix is in > >>> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948> < > >> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948>> < > >>> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948> < > >> https://github.com/apache/incubator-airflow/pull/1948 < > https://github.com/apache/incubator-airflow/pull/1948>>> , but your > >> feedback > >>> is required as it is entrenched in new processing code that you are > >> running > >>> in production afaik - so I wonder what happens in your fork. > >>> > >>> Happy New Year! > >>> > >>> Bolke > >>> > >>> > >>> > >>> -- > >>> _/ > >>> _/ Alex Van Boxel > >
