Dan

Can you elaborate on 2, cause I thought I specifically took care of that. 

Cheers
Bolke

Sent from my iPhone

> On 27 Feb 2017, at 20:27, Dan Davydov <dan.davy...@airbnb.com.INVALID> wrote:
> 
> I created https://issues.apache.org/jira/browse/AIRFLOW-921 to track the
> pending issues.
> 
> There are two more issues we found which I included there:
> 1. Task instances that have their state manually set to running make the UI
> for their DAG unable to parse
> 2. Mark success doesn't work for non existent task instances/dagruns which
> breaks the subdag use case (setting tasks as successful via the graph view)
> 
>> On Mon, Feb 27, 2017 at 11:06 AM, Bolke de Bruin <bdbr...@gmail.com> wrote:
>> 
>> Hey Max
>> 
>> It is massive for sure. Sorry about that ;-). However it is not as massive
>> as you might deduct from a first view. 0) run tasks concurrently across dag
>> runs 1) ordering of the tasks was added to the loop. 2) calculating of
>> deadlocks, running tasks, tasks to run was corrected, 3) relying on the
>> executor for status updates was replaced, 4) (tbd) executor failure check
>> to protect against endless Ioops.
>> 
>> 0+1 seem bigger than they are due to the amount of lines changed. 2 is a
>> subtle change, that touches a couple of lines to pop/push properly. 3) is
>> bigger, as I didn't like the reliance on the executor. 4) is old code that
>> needs to be added again.
>> 
>> I probably can leave out 3 which makes 4 mood. The change would be
>> smaller. Maybe I could even completely remove 3 and just add 4. What are
>> your thoughts?
>> 
>> The random failures we were seeing were the "implicit" test of not a
>> executing in the right order and then deadlocking. But no explicit tests
>> exist. Help would definitely be appreciated.
>> 
>> Yes I thought about using the scheduler and/or reusing logic from the
>> scheduler. I even experimented a little with it but it didn't allow me to
>> pass the tests effectively.
>> 
>> What I am planning to do is split the function and make it unit testable
>> if you agree with the current approach.
>> 
>> Bolke
>> 
>> Sent from my iPhone
>> 
>>> On 27 Feb 2017, at 18:35, Maxime Beauchemin <maximebeauche...@gmail.com>
>> wrote:
>>> 
>>> This PR is pretty massive and complex! It looks like solid work but let's
>>> be really careful around testing and rolling this out.
>>> 
>>> This may be out of scope for this PR, but wanted to discuss the idea of
>>> using the scheduler's logic to perform backfills. It'd be nice to have
>> that
>>> logic in one place, though I lost grasp on the details around feasibility
>>> around this approach. I'm sure you looked into this option before issuing
>>> this PR and I'm curious to hear your thoughts on blockers/challenges
>> around
>>> this alternate approach.
>>> 
>>> Also I'm wondering whether we have any sort of mechanisms in our
>>> integration test to validate that task dependencies are respected and run
>>> in the right order. If not I was thinking we could build some abstraction
>>> to make it easy to write this type of tests in an expressive way.
>>> 
>>> ```
>>> #[some code to run a backfill, or a scheduler session]
>>> it = IntegrationTestResults(dag_id='exmaple1')
>>> assert it.ran_before('task1', 'task_2')
>>> assert ti.overlapped('task1', 'task_3') # confirms 2 tasks ran in
>> parallel
>>> assert ti.none_failed()
>>> assert ti.ran_last('root')
>>> assert ti.max_concurrency_reached() == POOL_LIMIT
>>> ```
>>> 
>>> Max
>>> 
>>>> On Mon, Feb 27, 2017 at 5:41 AM, Bolke de Bruin <bdbr...@gmail.com>
>> wrote:
>>>> 
>>>> I have worked in the Backfill issue also in collaboration with Jeremiah.
>>>> 
>>>> The refactor to use dag runs in backfills caused a regression
>>>> in task execution performance as dag runs were executed
>>>> sequentially. Next to that, the backfills were non deterministic
>>>> due to the random execution of tasks, causing root tasks
>>>> being added to the non ready list too soon.
>>>> 
>>>> This updates the backfill logic as follows:
>>>> 
>>>>       • Parallelize execution of tasks
>>>>       • Use a leave first execution model; Breadth-first algorithm by
>>>> Jerermiah
>>>>       • Replace state updates from the executor by task based only
>>>> updates
>>>> 
>>>> https://github.com/apache/incubator-airflow/pull/2107
>>>> 
>>>> Please review and test properly.
>>>> 
>>>> What has been left out at the moment is the checking the executor itself
>>>> for multiple failures of a task run, where the task itself was never
>> able
>>>> to execute. Let me know if this is a real world scenario (maybe when
>> disk
>>>> space issue?). I will add it back in.
>>>> 
>>>> - Bolke
>>>> 
>>>> 
>>>>> On 25 Feb 2017, at 09:07, Bolke de Bruin <bdbr...@gmail.com> wrote:
>>>>> 
>>>>> Hi Dan,
>>>>> 
>>>>> - Backfill indeed runs only one dagrun at the time, see line 1755 of
>>>> jobs.py. I’ll think about how to fix this over the weekend (I think it
>> was
>>>> my change that introduced this). Suggestions always welcome. Depending
>> the
>>>> impact it is a blocker or not. We don’t often use backfills and
>> definitely
>>>> not at your size, so that is why it didn’t pop up with us. I’m assuming
>>>> blocker for now, btw.
>>>>> - Speculation on the High DB Load. I’m not sure what your benchmark is
>>>> here (1.7.1 + multi processor dags?), but as you mentioned in the code
>>>> dependencies are checked a couple of times for one run and even task
>>>> instance. Dependency checking requires aggregation on the DB, which is a
>>>> performance killer. Annoying but not a blocker.
>>>>> - Skipped tasks potentially cause a dagrun to be marked failure/success
>>>> prematurely. BranchOperators are widely used if it affects these
>> operators,
>>>> then it is a blocker.
>>>>> 
>>>>> - Bolke
>>>>> 
>>>>>> On 25 Feb 2017, at 02:04, Dan Davydov <dan.davy...@airbnb.com.
>> INVALID>
>>>> wrote:
>>>>>> 
>>>>>> Update on old pending issues:
>>>>>> - Black Squares in UI: Fix merged
>>>>>> - Double Trigger Issue That Alex G Mentioned: Alex has a PR in flight
>>>>>> 
>>>>>> New Issues:
>>>>>> - Backfill seems to be having issues (only running one dagrun at a
>>>> time),
>>>>>> we are still investigating - might be a blocker
>>>>>> - High DB Load (~8x more than 1.7) - We are still investigating but
>> it's
>>>>>> probably not a blocker for the release
>>>>>> - Skipped tasks potentially cause a dagrun to be marked as
>>>> failure/success
>>>>>> prematurely - not sure whether or not to classify this as a blocker
>>>> (only
>>>>>> really an issue for users who use the BranchingPythonOperator, which
>>>> AirBnB
>>>>>> does)
>>>>>> 
>>>>>> On Thu, Feb 23, 2017 at 5:59 PM, siddharth anand <san...@apache.org>
>>>> wrote:
>>>>>> 
>>>>>>> IMHO, a DAG run without a start date is non-sensical but is not
>>>> enforced
>>>>>>> That said, our UI allows for the manual creation of DAG Runs without
>> a
>>>>>>> start date as shown in the images below:
>>>>>>> 
>>>>>>> 
>>>>>>> - https://www.dropbox.com/s/3sxcqh04eztpl7p/Screenshot%
>>>>>>> 202017-02-22%2016.00.40.png?dl=0
>>>>>>> <https://www.dropbox.com/s/3sxcqh04eztpl7p/Screenshot%
>>>>>>> 202017-02-22%2016.00.40.png?dl=0>
>>>>>>> - https://www.dropbox.com/s/4q6rr9dwghag1yy/Screenshot%
>>>>>>> 202017-02-22%2016.02.22.png?dl=0
>>>>>>> <https://www.dropbox.com/s/4q6rr9dwghag1yy/Screenshot%
>>>>>>> 202017-02-22%2016.02.22.png?dl=0>
>>>>>>> 
>>>>>>> 
>>>>>>> On Wed, Feb 22, 2017 at 2:26 PM, Maxime Beauchemin <
>>>>>>> maximebeauche...@gmail.com> wrote:
>>>>>>> 
>>>>>>>> Our database may have edge cases that could be associated with
>> running
>>>>>>> any
>>>>>>>> previous version that may or may not have been part of an official
>>>>>>> release.
>>>>>>>> 
>>>>>>>> Let's see if anyone else reports the issue. If no one does, one
>>>> option is
>>>>>>>> to release 1.8.0 as is with a comment in the release notes, and
>> have a
>>>>>>>> future official minor apache release 1.8.1 that would fix these
>> minor
>>>>>>>> issues that are not deal breaker.
>>>>>>>> 
>>>>>>>> @bolke, I'm curious, how long does it take you to go through one
>>>> release
>>>>>>>> cycle? Oh, and do you have a documented step by step process for
>>>>>>> releasing?
>>>>>>>> I'd like to add the Pypi part to this doc and add committers that
>> are
>>>>>>>> interested to have rights on the project on Pypi.
>>>>>>>> 
>>>>>>>> Max
>>>>>>>> 
>>>>>>>> On Wed, Feb 22, 2017 at 2:00 PM, Bolke de Bruin <bdbr...@gmail.com>
>>>>>>> wrote:
>>>>>>>> 
>>>>>>>>> So it is a database integrity issue? Afaik a start_date should
>> always
>>>>>>> be
>>>>>>>>> set for a DagRun (create_dagrun) does so  I didn't check the code
>>>>>>> though.
>>>>>>>>> 
>>>>>>>>> Sent from my iPhone
>>>>>>>>> 
>>>>>>>>>> On 22 Feb 2017, at 22:19, Dan Davydov <dan.davy...@airbnb.com.
>>>>>>> INVALID>
>>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>> Should clarify this occurs when a dagrun does not have a start
>> date,
>>>>>>>> not
>>>>>>>>> a
>>>>>>>>>> dag (which makes it even less likely to happen). I don't think
>> this
>>>>>>> is
>>>>>>>> a
>>>>>>>>>> blocker for releasing.
>>>>>>>>>> 
>>>>>>>>>>> On Wed, Feb 22, 2017 at 1:15 PM, Dan Davydov <
>>>>>>> dan.davy...@airbnb.com>
>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> I rolled this out in our prod and the webservers failed to load
>> due
>>>>>>> to
>>>>>>>>>>> this commit:
>>>>>>>>>>> 
>>>>>>>>>>> [AIRFLOW-510] Filter Paused Dags, show Last Run & Trigger Dag
>>>>>>>>>>> 7c94d81c390881643f94d5e3d7d6fb351a445b72
>>>>>>>>>>> 
>>>>>>>>>>> This fixed it:
>>>>>>>>>>> -                            </a> <span id="statuses_info"
>>>>>>>>>>> class="glyphicon glyphicon-info-sign" aria-hidden="true"
>>>>>>> title="Start
>>>>>>>>> Date:
>>>>>>>>>>> {{last_run.start_date.strftime('%Y-%m-%d %H:%M')}}"></span>
>>>>>>>>>>> +                            </a> <span id="statuses_info"
>>>>>>>>>>> class="glyphicon glyphicon-info-sign" aria-hidden="true"></span>
>>>>>>>>>>> 
>>>>>>>>>>> This is caused by assuming that all DAGs have start dates set,
>> so a
>>>>>>>>> broken
>>>>>>>>>>> DAG will take down the whole UI. Not sure if we want to make
>> this a
>>>>>>>>> blocker
>>>>>>>>>>> for the release or not, I'm guessing for most deployments this
>>>> would
>>>>>>>>> occur
>>>>>>>>>>> pretty rarely. I'll submit a PR to fix it soon.
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> On Tue, Feb 21, 2017 at 9:49 AM, Chris Riccomini <
>>>>>>>> criccom...@apache.org
>>>>>>>>>> 
>>>>>>>>>>> wrote:
>>>>>>>>>>> 
>>>>>>>>>>>> Ack that the vote has already passed, but belated +1 (binding)
>>>>>>>>>>>> 
>>>>>>>>>>>> On Tue, Feb 21, 2017 at 7:42 AM, Bolke de Bruin <
>>>> bdbr...@gmail.com
>>>>>>>> 
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> IPMC Voting can be found here:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> http://mail-archives.apache.org/mod_mbox/incubator-general/
>>>>>>>>>>>> 201702.mbox/%
>>>>>>>>>>>>> 3c676bdc9f-1b55-4469-92a7-9ff309ad0...@gmail.com%3e <
>>>>>>>>>>>>> http://mail-archives.apache.org/mod_mbox/incubator-general/
>>>>>>>>>>>> 201702.mbox/%
>>>>>>>>>>>>> 3c676bdc9f-1b55-4469-92a7-9ff309ad0...@gmail.com%3E>
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Kind regards,
>>>>>>>>>>>>> Bolke
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On 21 Feb 2017, at 08:20, Bolke de Bruin <bdbr...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Hello,
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Apache Airflow (incubating) 1.8.0 (based on RC4) has been
>>>>>>> accepted.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 9 “+1” votes received:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> - Maxime Beauchemin (binding)
>>>>>>>>>>>>>> - Arthur Wiedmer (binding)
>>>>>>>>>>>>>> - Dan Davydov (binding)
>>>>>>>>>>>>>> - Jeremiah Lowin (binding)
>>>>>>>>>>>>>> - Siddharth Anand (binding)
>>>>>>>>>>>>>> - Alex van Boxel (binding)
>>>>>>>>>>>>>> - Bolke de Bruin (binding)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> - Jayesh Senjaliya (non-binding)
>>>>>>>>>>>>>> - Yi (non-binding)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Vote thread (start):
>>>>>>>>>>>>>> http://mail-archives.apache.org/mod_mbox/incubator-
>>>>>>>>>>>>> airflow-dev/201702.mbox/%3cD360D9BE-C358-42A1-9188-
>>>>>>>>>>>>> 6c92c31a2...@gmail.com%3e <http://mail-archives.apache.
>>>>>>>>>>>>> org/mod_mbox/incubator-airflow-dev/201702.mbox/%3C7EB7B6D6-
>>>>>>>>>>>> 092E-48D2-AA0F-
>>>>>>>>>>>>> 15f44376a...@gmail.com%3E>
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Next steps:
>>>>>>>>>>>>>> 1) will start the voting process at the IPMC mailinglist. I do
>>>>>>>> expect
>>>>>>>>>>>>> some changes to be required mostly in documentation maybe a
>>>>>>> license
>>>>>>>>> here
>>>>>>>>>>>>> and there. So, we might end up with changes to stable. As long
>> as
>>>>>>>>> these
>>>>>>>>>>>> are
>>>>>>>>>>>>> not (significant) code changes I will not re-raise the vote.
>>>>>>>>>>>>>> 2) Only after the positive voting on the IPMC and
>> finalisation I
>>>>>>>> will
>>>>>>>>>>>>> rebrand the RC to Release.
>>>>>>>>>>>>>> 3) I will upload it to the incubator release page, then the
>> tar
>>>>>>>> ball
>>>>>>>>>>>>> needs to propagate to the mirrors.
>>>>>>>>>>>>>> 4) Update the website (can someone volunteer please?)
>>>>>>>>>>>>>> 5) Finally, I will ask Maxime to upload it to pypi. It seems
>> we
>>>>>>> can
>>>>>>>>>>>> keep
>>>>>>>>>>>>> the apache branding as lib cloud is doing this as well (
>>>>>>>>>>>>> https://libcloud.apache.org/downloads.html#pypi-package <
>>>>>>>>>>>>> https://libcloud.apache.org/downloads.html#pypi-package>).
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Jippie!
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Bolke
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>> 
>>>> 
>>>> 
>> 

Reply via email to