Re: 1.10.1 Release?

2018-11-05 Thread Bolke de Bruin
The fix is in master and should work across all DST changes. It will be 
included in 1.10.1. 

B. 

Sent from my iPhone

> On 5 Nov 2018, at 19:54, Dave Fisher  wrote:
> 
> 
> 
>> On 2018/10/28 00:09:05, Bolke de Bruin  wrote: 
>> I wonder how to treat this:
>> 
>> This is what I think happens (need to verify more, but I am pretty sure) the 
>> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM)
> 
> FYI - In the US the DST change is 2AM -> 1AM. Yes, TZ is hard stuff.
> 
> we basically hit a schedule that we have already seen. 2AM -> 3AM has already 
> happened. Obviously the intention is to run every 5 minutes. But what do we 
> do with the execution_date? Is this still idempotent? Should we indeed 
> reschedule? 
>> 
>> B.
>> 
>>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
>>> 
>>> I've done a bit more digging - the issue is of our tz-aware handling inside 
>>> following_schedule (and previous schedule) - causing it to loop.
>>> 
>>> This section of the croniter docs seems relevant 
>>> https://github.com/kiorky/croniter#about-dst
>>> 
>>>   Be sure to init your croniter instance with a TZ aware datetime for this 
>>> to work !:
>> local_date = tz.localize(datetime(2017, 3, 26))
>> val = croniter('0 0 * * *', local_date).get_next(datetime)
>>> 
>>> I think the problem is that we are _not_ passing a TZ aware dag in and we 
>>> should be.
>>> 
 On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
 
 Oh that’s a great environment to start digging. Thanks. I’ll have a look.
 
 B.
 
 Verstuurd vanaf mijn iPad
 
> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
> 
> last_run = dag.get_last_dagrun(session=session)
> if last_run and next_run_date:
> while next_run_date <= last_run.execution_date:
> next_run_date = dag.following_schedule(next_run_date)
> 
> 
> 
>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
>> 
>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>> https://github.com/kaczors/airflow_1_10_tz_bug
>> 
>> Rough repro steps: In a VM, with time syncing disabled, and configured 
>> with system timezone of Europe/Zurich (or any other CEST one) run 
>> 
>> - `date 10280250.00`
>> - initdb, start scheduler, webserver, enable dag etc.
>> - `date 10280259.00`
>> - wait 5-10 mins for scheduler to catch up
>> - After the on-the-hour task run the scheduler will spin up another 
>> process to parse the dag... and it never returns.
>> 
>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>> quick hacky debug print shows something is stuck in an infinite loop.
>> 
>> -ash
>> 
>>> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
>>> 
>>> Can this be confirmed? Then I can have a look at it. Preferably with 
>>> dag definition code.
>>> 
>>> On the licensing requirements:
>>> 
>>> 1. Indeed licensing header for markdown documents. It was suggested to 
>>> use html comments. I’m not sure how that renders with others like PDF 
>>> though.
>>> 2. The licensing notifications need to be tied to a specific version as 
>>> licenses might change with versions.
>>> 
>>> Cheers
>>> Bolke
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
 Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
 volgende geschreven:
 
 I was going to make a start on the release, but two people have 
 reported that there might be an issue around non-UTC dags and the 
 scheduler changing over from Summer time.
 
> 08:45 Emmanuel> Hi there, we are currently experiencing a very 
> strange issue : we have hourly DAGs with a start_date in a local 
> timezone (not UTC) and since (Sunday) the last winter time change 
> they don’t run anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am 
> (Europe/Paris), the exact time of winter time change :(
 
 I am going to take a look at this today and see if I can get to the 
 bottom of it.
 
 Bolke: are there any outstanding tasks/issues that you know of that 
 might slow down the vote for a 1.10.1? (i.e. did we sort of out all 
 the licensing issues that were asked of us? I thought I read something 
 about license declarations in markdown files?)
 
 -ash
 
> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> 
> I agree with that, but I would favor time based releases instead. We 
> are again at the point that a release takes so much time that the gap 
> is getting really big again. @ash why not start releasing now and 

Re: 1.10.1 Release?

2018-11-05 Thread Dave Fisher



On 2018/10/28 00:09:05, Bolke de Bruin  wrote: 
> I wonder how to treat this:
> 
> This is what I think happens (need to verify more, but I am pretty sure) the 
> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM)

FYI - In the US the DST change is 2AM -> 1AM. Yes, TZ is hard stuff.

 we basically hit a schedule that we have already seen. 2AM -> 3AM has already 
happened. Obviously the intention is to run every 5 minutes. But what do we do 
with the execution_date? Is this still idempotent? Should we indeed reschedule? 
> 
> B.
> 
> > On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> > 
> > I've done a bit more digging - the issue is of our tz-aware handling inside 
> > following_schedule (and previous schedule) - causing it to loop.
> > 
> > This section of the croniter docs seems relevant 
> > https://github.com/kiorky/croniter#about-dst
> > 
> >Be sure to init your croniter instance with a TZ aware datetime for this 
> > to work !:
>  local_date = tz.localize(datetime(2017, 3, 26))
>  val = croniter('0 0 * * *', local_date).get_next(datetime)
> > 
> > I think the problem is that we are _not_ passing a TZ aware dag in and we 
> > should be.
> > 
> >> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
> >> 
> >> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
> >> 
> >> B.
> >> 
> >> Verstuurd vanaf mijn iPad
> >> 
> >>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
> >>> volgende geschreven:
> >>> 
> >>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
> >>> 
> >>>  last_run = dag.get_last_dagrun(session=session)
> >>>  if last_run and next_run_date:
> >>>  while next_run_date <= last_run.execution_date:
> >>>  next_run_date = dag.following_schedule(next_run_date)
> >>> 
> >>> 
> >>> 
>  On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
>  
>  Hi, kaczors on gitter has produced a minmal reproduction case: 
>  https://github.com/kaczors/airflow_1_10_tz_bug
>  
>  Rough repro steps: In a VM, with time syncing disabled, and configured 
>  with system timezone of Europe/Zurich (or any other CEST one) run 
>  
>  - `date 10280250.00`
>  - initdb, start scheduler, webserver, enable dag etc.
>  - `date 10280259.00`
>  - wait 5-10 mins for scheduler to catch up
>  - After the on-the-hour task run the scheduler will spin up another 
>  process to parse the dag... and it never returns.
>  
>  I've only just managed to reproduce it, so haven't dug in to why yet. A 
>  quick hacky debug print shows something is stuck in an infinite loop.
>  
>  -ash
>  
> > On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> > 
> > Can this be confirmed? Then I can have a look at it. Preferably with 
> > dag definition code.
> > 
> > On the licensing requirements:
> > 
> > 1. Indeed licensing header for markdown documents. It was suggested to 
> > use html comments. I’m not sure how that renders with others like PDF 
> > though.
> > 2. The licensing notifications need to be tied to a specific version as 
> > licenses might change with versions.
> > 
> > Cheers
> > Bolke
> > 
> > Verstuurd vanaf mijn iPad
> > 
> >> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
> >> volgende geschreven:
> >> 
> >> I was going to make a start on the release, but two people have 
> >> reported that there might be an issue around non-UTC dags and the 
> >> scheduler changing over from Summer time.
> >> 
> >>> 08:45 Emmanuel> Hi there, we are currently experiencing a very 
> >>> strange issue : we have hourly DAGs with a start_date in a local 
> >>> timezone (not UTC) and since (Sunday) the last winter time change 
> >>> they don’t run anymore. Any idea ?
> >>> 09:41  it impacted all our DAG that had a run at 3am 
> >>> (Europe/Paris), the exact time of winter time change :(
> >> 
> >> I am going to take a look at this today and see if I can get to the 
> >> bottom of it.
> >> 
> >> Bolke: are there any outstanding tasks/issues that you know of that 
> >> might slow down the vote for a 1.10.1? (i.e. did we sort of out all 
> >> the licensing issues that were asked of us? I thought I read something 
> >> about license declarations in markdown files?)
> >> 
> >> -ash
> >> 
> >>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> >>> 
> >>> I agree with that, but I would favor time based releases instead. We 
> >>> are again at the point that a release takes so much time that the gap 
> >>> is getting really big again. @ash why not start releasing now and 
> >>> move the remainder to 1.10.2? I dont think there are real blockers 
> >>> (although we might find them).
> >>> 
> >>> 
>  On 28 Oct 2018, at 15:35, airflowuser 
> 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
They are :-)

I might have to adjust some tests, let's see. Will do that tomorrow if required.

B.

> On 30 Oct 2018, at 21:53, Ash Berlin-Taylor  wrote:
> 
> Fair :) Timezones are _hard_
> 
> Giving it a look now.
> 
> -ash
> 
>> On 30 Oct 2018, at 20:50, Bolke de Bruin  wrote:
>> 
>> The reason for not passing a TZ aware object is, is that many libraries make 
>> mistakes (pytz, arrow etc) when doing transitions hence to use of pendulum 
>> which seem most complete. I don’t know what croniter is relying on and I 
>> don’t want to find out ;-).
>> 
>> B.
>> 
>>> On 30 Oct 2018, at 21:13, Ash Berlin-Taylor >> > wrote:
>>> 
>>> I think if we give croniter a tz-aware DT in the local tz it will deal with 
>>> DST (i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it 
>>> to UTC for return - but right now we are giving it a TZ-unaware local time.
>>> 
>>> I think.
>>> 
>>> Ash
>>> 
>>> On 30 October 2018 19:40:27 GMT, Bolke de Bruin  wrote:
>>> I think we should use the UTC date for cron instead of the naive local date 
>>> time. I will check of croniter implements this so we can rely on that.
>>> 
>>> B.
>>> 
>>> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
>>> 
>>> I wonder how to treat this:
>>> 
>>> This is what I think happens (need to verify more, but I am pretty sure) 
>>> the specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
>>> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
>>> happened. Obviously the intention is to run every 5 minutes. But what do we 
>>> do with the execution_date? Is this still idempotent? Should we indeed 
>>> reschedule? 
>>> 
>>> B.
>>> 
>>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
>>> 
>>> I've done a bit more digging - the issue is of our tz-aware handling inside 
>>> following_schedule (and previous schedule) - causing it to loop.
>>> 
>>> This section of the croniter docs seems relevant 
>>> https://github.com/kiorky/croniter#about-dst 
>>> >>  >
>>> 
>>> Be sure to init your croniter instance with a TZ aware datetime for this to 
>>> work !:
>>> local_date = tz.localize(datetime(2017, 3, 26))
>>> val = croniter('0 0 * * *', local_date).get_next(datetime)
>>> 
>>> I think the problem is that we are _not_ passing a TZ aware dag in and we 
>>> should be.
>>> 
>>> On 30 Oct 2018, at 17:35, Bolke de Bruin >> > wrote:
>>> 
>>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>>> 
>>> B.
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
>>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor >> > het volgende geschreven:
>>> 
>>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>>> 
>>>   last_run = dag.get_last_dagrun(session=session)
>>>   if last_run and next_run_date:
>>>   while next_run_date <= last_run.execution_date:
>>>   next_run_date = dag.following_schedule(next_run_date)
>>> 
>>> 
>>> 
>>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor >> > wrote:
>>> 
>>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>>> https://github.com/kaczors/airflow_1_10_tz_bug 
>>>  
>>> >> >
>>> 
>>> Rough repro steps: In a VM, with time syncing disabled, and configured with 
>>> system timezone of Europe/Zurich (or any other CEST one) run 
>>> 
>>> - `date 10280250.00`
>>> - initdb, start scheduler, webserver, enable dag etc.
>>> - `date 10280259.00`
>>> - wait 5-10 mins for scheduler to catch up
>>> - After the on-the-hour task run the scheduler will spin up another process 
>>> to parse the dag... and it never returns.
>>> 
>>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>>> quick hacky debug print shows something is stuck in an infinite loop.
>>> 
>>> -ash
>>> 
>>> On 29 Oct 2018, at 17:59, Bolke de Bruin >> > wrote:
>>> 
>>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>>> definition code.
>>> 
>>> On the licensing requirements:
>>> 
>>> 1. Indeed licensing header for markdown documents. It was suggested to use 
>>> html comments. I’m not sure how that renders with others like PDF though.
>>> 2. The licensing notifications need to be tied to a specific version as 
>>> licenses might change with versions.
>>> 
>>> Cheers
>>> Bolke
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
>>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor >> > het volgende geschreven:
>>> 
>>> I was going to make a start on the release, but two people have reported 
>>> that there might be an issue around non-UTC dags and the scheduler changing 
>>> over from Summer time.

Re: 1.10.1 Release?

2018-10-30 Thread Ash Berlin-Taylor
Fair :) Timezones are _hard_

Giving it a look now.

-ash

> On 30 Oct 2018, at 20:50, Bolke de Bruin  wrote:
> 
> The reason for not passing a TZ aware object is, is that many libraries make 
> mistakes (pytz, arrow etc) when doing transitions hence to use of pendulum 
> which seem most complete. I don’t know what croniter is relying on and I 
> don’t want to find out ;-).
> 
> B.
> 
>> On 30 Oct 2018, at 21:13, Ash Berlin-Taylor > > wrote:
>> 
>> I think if we give croniter a tz-aware DT in the local tz it will deal with 
>> DST (i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it 
>> to UTC for return - but right now we are giving it a TZ-unaware local time.
>> 
>> I think.
>> 
>> Ash
>> 
>> On 30 October 2018 19:40:27 GMT, Bolke de Bruin  wrote:
>> I think we should use the UTC date for cron instead of the naive local date 
>> time. I will check of croniter implements this so we can rely on that.
>> 
>> B.
>> 
>> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
>> 
>> I wonder how to treat this:
>> 
>> This is what I think happens (need to verify more, but I am pretty sure) the 
>> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
>> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
>> happened. Obviously the intention is to run every 5 minutes. But what do we 
>> do with the execution_date? Is this still idempotent? Should we indeed 
>> reschedule? 
>> 
>> B.
>> 
>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
>> 
>> I've done a bit more digging - the issue is of our tz-aware handling inside 
>> following_schedule (and previous schedule) - causing it to loop.
>> 
>> This section of the croniter docs seems relevant 
>> https://github.com/kiorky/croniter#about-dst 
>> >  >
>> 
>>  Be sure to init your croniter instance with a TZ aware datetime for this to 
>> work !:
>> local_date = tz.localize(datetime(2017, 3, 26))
>> val = croniter('0 0 * * *', local_date).get_next(datetime)
>> 
>> I think the problem is that we are _not_ passing a TZ aware dag in and we 
>> should be.
>> 
>> On 30 Oct 2018, at 17:35, Bolke de Bruin > > wrote:
>> 
>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>> 
>> B.
>> 
>> Verstuurd vanaf mijn iPad
>> 
>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor > > het volgende geschreven:
>> 
>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>> 
>>last_run = dag.get_last_dagrun(session=session)
>>if last_run and next_run_date:
>>while next_run_date <= last_run.execution_date:
>>next_run_date = dag.following_schedule(next_run_date)
>> 
>> 
>> 
>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor > > wrote:
>> 
>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>> https://github.com/kaczors/airflow_1_10_tz_bug 
>>  
>> > >
>> 
>> Rough repro steps: In a VM, with time syncing disabled, and configured with 
>> system timezone of Europe/Zurich (or any other CEST one) run 
>> 
>> - `date 10280250.00`
>> - initdb, start scheduler, webserver, enable dag etc.
>> - `date 10280259.00`
>> - wait 5-10 mins for scheduler to catch up
>> - After the on-the-hour task run the scheduler will spin up another process 
>> to parse the dag... and it never returns.
>> 
>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>> quick hacky debug print shows something is stuck in an infinite loop.
>> 
>> -ash
>> 
>> On 29 Oct 2018, at 17:59, Bolke de Bruin > > wrote:
>> 
>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>> definition code.
>> 
>> On the licensing requirements:
>> 
>> 1. Indeed licensing header for markdown documents. It was suggested to use 
>> html comments. I’m not sure how that renders with others like PDF though.
>> 2. The licensing notifications need to be tied to a specific version as 
>> licenses might change with versions.
>> 
>> Cheers
>> Bolke
>> 
>> Verstuurd vanaf mijn iPad
>> 
>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor > > het volgende geschreven:
>> 
>> I was going to make a start on the release, but two people have reported 
>> that there might be an issue around non-UTC dags and the scheduler changing 
>> over from Summer time.
>> 
>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
>> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
>> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
>> 09:41  it impacted all our DAG that had a run 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
The reason for not passing a TZ aware object is, is that many libraries make 
mistakes (pytz, arrow etc) when doing transitions hence to use of pendulum 
which seem most complete. I don’t know what croniter is relying on and I don’t 
want to find out ;-).

B.

> On 30 Oct 2018, at 21:13, Ash Berlin-Taylor  wrote:
> 
> I think if we give croniter a tz-aware DT in the local tz it will deal with 
> DST (i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it to 
> UTC for return - but right now we are giving it a TZ-unaware local time.
> 
> I think.
> 
> Ash
> 
> On 30 October 2018 19:40:27 GMT, Bolke de Bruin  wrote:
> I think we should use the UTC date for cron instead of the naive local date 
> time. I will check of croniter implements this so we can rely on that.
> 
> B.
> 
> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
> 
> I wonder how to treat this:
> 
> This is what I think happens (need to verify more, but I am pretty sure) the 
> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
> happened. Obviously the intention is to run every 5 minutes. But what do we 
> do with the execution_date? Is this still idempotent? Should we indeed 
> reschedule? 
> 
> B.
> 
> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> 
> I've done a bit more digging - the issue is of our tz-aware handling inside 
> following_schedule (and previous schedule) - causing it to loop.
> 
> This section of the croniter docs seems relevant 
> https://github.com/kiorky/croniter#about-dst 
> 
> 
>   Be sure to init your croniter instance with a TZ aware datetime for this to 
> work !:
> local_date = tz.localize(datetime(2017, 3, 26))
> val = croniter('0 0 * * *', local_date).get_next(datetime)
> 
> I think the problem is that we are _not_ passing a TZ aware dag in and we 
> should be.
> 
> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
> 
> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
> 
> B.
> 
> Verstuurd vanaf mijn iPad
> 
> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
> 
> last_run = dag.get_last_dagrun(session=session)
> if last_run and next_run_date:
> while next_run_date <= last_run.execution_date:
> next_run_date = dag.following_schedule(next_run_date)
> 
> 
> 
> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
> 
> Hi, kaczors on gitter has produced a minmal reproduction case: 
> https://github.com/kaczors/airflow_1_10_tz_bug 
> 
> 
> Rough repro steps: In a VM, with time syncing disabled, and configured with 
> system timezone of Europe/Zurich (or any other CEST one) run 
> 
> - `date 10280250.00`
> - initdb, start scheduler, webserver, enable dag etc.
> - `date 10280259.00`
> - wait 5-10 mins for scheduler to catch up
> - After the on-the-hour task run the scheduler will spin up another process 
> to parse the dag... and it never returns.
> 
> I've only just managed to reproduce it, so haven't dug in to why yet. A quick 
> hacky debug print shows something is stuck in an infinite loop.
> 
> -ash
> 
> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> 
> Can this be confirmed? Then I can have a look at it. Preferably with dag 
> definition code.
> 
> On the licensing requirements:
> 
> 1. Indeed licensing header for markdown documents. It was suggested to use 
> html comments. I’m not sure how that renders with others like PDF though.
> 2. The licensing notifications need to be tied to a specific version as 
> licenses might change with versions.
> 
> Cheers
> Bolke
> 
> Verstuurd vanaf mijn iPad
> 
> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> I was going to make a start on the release, but two people have reported that 
> there might be an issue around non-UTC dags and the scheduler changing over 
> from Summer time.
> 
> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am 
> (Europe/Paris), the exact time of winter time change :(
> 
> I am going to take a look at this today and see if I can get to the bottom of 
> it.
> 
> Bolke: are there any outstanding tasks/issues that you know of that might 
> slow down the vote for a 1.10.1? (i.e. did we sort of out all the licensing 
> issues that were asked of us? I thought I read something about license 
> declarations in markdown files?)
> 
> -ash
> 
> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> 
> I agree with that, but I would favor time based releases instead. We are 
> again at the point that a 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
Patch available at:

https://github.com/apache/incubator-airflow/pull/4117

please test.

B.

> On 30 Oct 2018, at 21:14, Bolke de Bruin  wrote:
> 
> We should just pass it the UTC date (we should never use local time except at 
> the user interface). I’m testing a patch right now.
> 
> B.
> 
>> On 30 Oct 2018, at 21:13, Ash Berlin-Taylor > > wrote:
>> 
>> I think if we give croniter a tz-aware DT in the local tz it will deal with 
>> DST (i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it 
>> to UTC for return - but right now we are giving it a TZ-unaware local time.
>> 
>> I think.
>> 
>> Ash
>> 
>> On 30 October 2018 19:40:27 GMT, Bolke de Bruin > > wrote:
>> I think we should use the UTC date for cron instead of the naive local date 
>> time. I will check of croniter implements this so we can rely on that.
>> 
>> B.
>> 
>> On 28 Oct 2018, at 02:09, Bolke de Bruin > > wrote:
>> 
>> I wonder how to treat this:
>> 
>> This is what I think happens (need to verify more, but I am pretty sure) the 
>> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
>> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
>> happened. Obviously the intention is to run every 5 minutes. But what do we 
>> do with the execution_date? Is this still idempotent? Should we indeed 
>> reschedule? 
>> 
>> B.
>> 
>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor > > wrote:
>> 
>> I've done a bit more digging - the issue is of our tz-aware handling inside 
>> following_schedule (and previous schedule) - causing it to loop.
>> 
>> This section of the croniter docs seems relevant 
>> https://github.com/kiorky/croniter#about-dst 
>> 
>> 
>>   Be sure to init your croniter instance with a TZ aware datetime for this 
>> to work !:
>> local_date = tz.localize(datetime(2017, 3, 26))
>> val = croniter('0 0 * * *', local_date).get_next(datetime)
>> 
>> I think the problem is that we are _not_ passing a TZ aware dag in and we 
>> should be.
>> 
>> On 30 Oct 2018, at 17:35, Bolke de Bruin > > wrote:
>> 
>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>> 
>> B.
>> 
>> Verstuurd vanaf mijn iPad
>> 
>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor > > het volgende geschreven:
>> 
>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>> 
>> last_run = dag.get_last_dagrun(session=session)
>> if last_run and next_run_date:
>> while next_run_date <= last_run.execution_date:
>> next_run_date = dag.following_schedule(next_run_date)
>> 
>> 
>> 
>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor > > wrote:
>> 
>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>> https://github.com/kaczors/airflow_1_10_tz_bug 
>> 
>> 
>> Rough repro steps: In a VM, with time syncing disabled, and configured with 
>> system timezone of Europe/Zurich (or any other CEST one) run 
>> 
>> - `date 10280250.00`
>> - initdb, start scheduler, webserver, enable dag etc.
>> - `date 10280259.00`
>> - wait 5-10 mins for scheduler to catch up
>> - After the on-the-hour task run the scheduler will spin up another process 
>> to parse the dag... and it never returns.
>> 
>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>> quick hacky debug print shows something is stuck in an infinite loop.
>> 
>> -ash
>> 
>> On 29 Oct 2018, at 17:59, Bolke de Bruin > > wrote:
>> 
>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>> definition code.
>> 
>> On the licensing requirements:
>> 
>> 1. Indeed licensing header for markdown documents. It was suggested to use 
>> html comments. I’m not sure how that renders with others like PDF though.
>> 2. The licensing notifications need to be tied to a specific version as 
>> licenses might change with versions.
>> 
>> Cheers
>> Bolke
>> 
>> Verstuurd vanaf mijn iPad
>> 
>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor > > het volgende geschreven:
>> 
>> I was going to make a start on the release, but two people have reported 
>> that there might be an issue around non-UTC dags and the scheduler changing 
>> over from Summer time.
>> 
>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
>> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
>> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
>> 09:41  it impacted all our DAG that had a run at 3am 
>> (Europe/Paris), the exact time of winter time change :(
>> 
>> I am going to take a look at this today and see if I can get to the bottom 
>> of it.
>> 
>> Bolke: are there any outstanding 

Re: 1.10.1 Release?

2018-10-30 Thread Ash Berlin-Taylor
I think if we give croniter a tz-aware DT in the local tz it will deal with DST 
(i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it to UTC 
for return - but right now we are giving it a TZ-unaware local time.

I think.

Ash

On 30 October 2018 19:40:27 GMT, Bolke de Bruin  wrote:
>I think we should use the UTC date for cron instead of the naive local
>date time. I will check of croniter implements this so we can rely on
>that.
>
>B.
>
>> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
>> 
>> I wonder how to treat this:
>> 
>> This is what I think happens (need to verify more, but I am pretty
>sure) the specified DAG should run every 5 minutes. At DST change (3AM
>-> 2AM) we basically hit a schedule that we have already seen. 2AM ->
>3AM has already happened. Obviously the intention is to run every 5
>minutes. But what do we do with the execution_date? Is this still
>idempotent? Should we indeed reschedule? 
>> 
>> B.
>> 
>>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
>>> 
>>> I've done a bit more digging - the issue is of our tz-aware handling
>inside following_schedule (and previous schedule) - causing it to loop.
>>> 
>>> This section of the croniter docs seems relevant
>https://github.com/kiorky/croniter#about-dst
>>> 
>>>   Be sure to init your croniter instance with a TZ aware datetime
>for this to work !:
>> local_date = tz.localize(datetime(2017, 3, 26))
>> val = croniter('0 0 * * *', local_date).get_next(datetime)
>>> 
>>> I think the problem is that we are _not_ passing a TZ aware dag in
>and we should be.
>>> 
 On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
 
 Oh that’s a great environment to start digging. Thanks. I’ll have a
>look.
 
 B.
 
 Verstuurd vanaf mijn iPad
 
> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor 
>het volgende geschreven:
> 
> This line in airflow.jobs (line 874 in my checkout) is causing the
>loop:
> 
> last_run = dag.get_last_dagrun(session=session)
> if last_run and next_run_date:
> while next_run_date <= last_run.execution_date:
> next_run_date =
>dag.following_schedule(next_run_date)
> 
> 
> 
>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor 
>wrote:
>> 
>> Hi, kaczors on gitter has produced a minmal reproduction case:
>https://github.com/kaczors/airflow_1_10_tz_bug
>> 
>> Rough repro steps: In a VM, with time syncing disabled, and
>configured with system timezone of Europe/Zurich (or any other CEST
>one) run 
>> 
>> - `date 10280250.00`
>> - initdb, start scheduler, webserver, enable dag etc.
>> - `date 10280259.00`
>> - wait 5-10 mins for scheduler to catch up
>> - After the on-the-hour task run the scheduler will spin up
>another process to parse the dag... and it never returns.
>> 
>> I've only just managed to reproduce it, so haven't dug in to why
>yet. A quick hacky debug print shows something is stuck in an infinite
>loop.
>> 
>> -ash
>> 
>>> On 29 Oct 2018, at 17:59, Bolke de Bruin 
>wrote:
>>> 
>>> Can this be confirmed? Then I can have a look at it. Preferably
>with dag definition code.
>>> 
>>> On the licensing requirements:
>>> 
>>> 1. Indeed licensing header for markdown documents. It was
>suggested to use html comments. I’m not sure how that renders with
>others like PDF though.
>>> 2. The licensing notifications need to be tied to a specific
>version as licenses might change with versions.
>>> 
>>> Cheers
>>> Bolke
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
 Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor
> het volgende geschreven:
 
 I was going to make a start on the release, but two people have
>reported that there might be an issue around non-UTC dags and the
>scheduler changing over from Summer time.
 
> 08:45 Emmanuel> Hi there, we are currently experiencing a very
>strange issue : we have hourly DAGs with a start_date in a local
>timezone (not UTC) and since (Sunday) the last winter time change they
>don’t run anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am
>(Europe/Paris), the exact time of winter time change :(
 
 I am going to take a look at this today and see if I can get to
>the bottom of it.
 
 Bolke: are there any outstanding tasks/issues that you know of
>that might slow down the vote for a 1.10.1? (i.e. did we sort of out
>all the licensing issues that were asked of us? I thought I read
>something about license declarations in markdown files?)
 
 -ash
 
> On 28 Oct 2018, at 14:46, Bolke de Bruin 
>wrote:
> 
> I agree with that, but I would favor time based releases
>instead. We are again at the point that a release takes so much time
>that the gap is getting really big again. @ash why not start releasing
>now and 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
We should just pass it the UTC date (we should never use local time except at 
the user interface). I’m testing a patch right now.

B.

> On 30 Oct 2018, at 21:13, Ash Berlin-Taylor  wrote:
> 
> I think if we give croniter a tz-aware DT in the local tz it will deal with 
> DST (i.e. will give 2:55 CEST followed by 2:00 CET) and then we convert it to 
> UTC for return - but right now we are giving it a TZ-unaware local time.
> 
> I think.
> 
> Ash
> 
> On 30 October 2018 19:40:27 GMT, Bolke de Bruin  wrote:
> I think we should use the UTC date for cron instead of the naive local date 
> time. I will check of croniter implements this so we can rely on that.
> 
> B.
> 
> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
> 
> I wonder how to treat this:
> 
> This is what I think happens (need to verify more, but I am pretty sure) the 
> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
> happened. Obviously the intention is to run every 5 minutes. But what do we 
> do with the execution_date? Is this still idempotent? Should we indeed 
> reschedule? 
> 
> B.
> 
> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> 
> I've done a bit more digging - the issue is of our tz-aware handling inside 
> following_schedule (and previous schedule) - causing it to loop.
> 
> This section of the croniter docs seems relevant 
> https://github.com/kiorky/croniter#about-dst 
> 
> 
>   Be sure to init your croniter instance with a TZ aware datetime for this to 
> work !:
> local_date = tz.localize(datetime(2017, 3, 26))
> val = croniter('0 0 * * *', local_date).get_next(datetime)
> 
> I think the problem is that we are _not_ passing a TZ aware dag in and we 
> should be.
> 
> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
> 
> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
> 
> B.
> 
> Verstuurd vanaf mijn iPad
> 
> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
> 
> last_run = dag.get_last_dagrun(session=session)
> if last_run and next_run_date:
> while next_run_date <= last_run.execution_date:
> next_run_date = dag.following_schedule(next_run_date)
> 
> 
> 
> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
> 
> Hi, kaczors on gitter has produced a minmal reproduction case: 
> https://github.com/kaczors/airflow_1_10_tz_bug 
> 
> 
> Rough repro steps: In a VM, with time syncing disabled, and configured with 
> system timezone of Europe/Zurich (or any other CEST one) run 
> 
> - `date 10280250.00`
> - initdb, start scheduler, webserver, enable dag etc.
> - `date 10280259.00`
> - wait 5-10 mins for scheduler to catch up
> - After the on-the-hour task run the scheduler will spin up another process 
> to parse the dag... and it never returns.
> 
> I've only just managed to reproduce it, so haven't dug in to why yet. A quick 
> hacky debug print shows something is stuck in an infinite loop.
> 
> -ash
> 
> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> 
> Can this be confirmed? Then I can have a look at it. Preferably with dag 
> definition code.
> 
> On the licensing requirements:
> 
> 1. Indeed licensing header for markdown documents. It was suggested to use 
> html comments. I’m not sure how that renders with others like PDF though.
> 2. The licensing notifications need to be tied to a specific version as 
> licenses might change with versions.
> 
> Cheers
> Bolke
> 
> Verstuurd vanaf mijn iPad
> 
> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> I was going to make a start on the release, but two people have reported that 
> there might be an issue around non-UTC dags and the scheduler changing over 
> from Summer time.
> 
> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am 
> (Europe/Paris), the exact time of winter time change :(
> 
> I am going to take a look at this today and see if I can get to the bottom of 
> it.
> 
> Bolke: are there any outstanding tasks/issues that you know of that might 
> slow down the vote for a 1.10.1? (i.e. did we sort of out all the licensing 
> issues that were asked of us? I thought I read something about license 
> declarations in markdown files?)
> 
> -ash
> 
> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> 
> I agree with that, but I would favor time based releases instead. We are 
> again at the point that a release takes so much time that the gap is getting 
> really big again. @ash why not start releasing now and move the remainder to 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
I think we should use the UTC date for cron instead of the naive local date 
time. I will check of croniter implements this so we can rely on that.

B.

> On 28 Oct 2018, at 02:09, Bolke de Bruin  wrote:
> 
> I wonder how to treat this:
> 
> This is what I think happens (need to verify more, but I am pretty sure) the 
> specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
> basically hit a schedule that we have already seen. 2AM -> 3AM has already 
> happened. Obviously the intention is to run every 5 minutes. But what do we 
> do with the execution_date? Is this still idempotent? Should we indeed 
> reschedule? 
> 
> B.
> 
>> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
>> 
>> I've done a bit more digging - the issue is of our tz-aware handling inside 
>> following_schedule (and previous schedule) - causing it to loop.
>> 
>> This section of the croniter docs seems relevant 
>> https://github.com/kiorky/croniter#about-dst
>> 
>>   Be sure to init your croniter instance with a TZ aware datetime for this 
>> to work !:
> local_date = tz.localize(datetime(2017, 3, 26))
> val = croniter('0 0 * * *', local_date).get_next(datetime)
>> 
>> I think the problem is that we are _not_ passing a TZ aware dag in and we 
>> should be.
>> 
>>> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
>>> 
>>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>>> 
>>> B.
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
 Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
 volgende geschreven:
 
 This line in airflow.jobs (line 874 in my checkout) is causing the loop:
 
 last_run = dag.get_last_dagrun(session=session)
 if last_run and next_run_date:
 while next_run_date <= last_run.execution_date:
 next_run_date = dag.following_schedule(next_run_date)
 
 
 
> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
> 
> Hi, kaczors on gitter has produced a minmal reproduction case: 
> https://github.com/kaczors/airflow_1_10_tz_bug
> 
> Rough repro steps: In a VM, with time syncing disabled, and configured 
> with system timezone of Europe/Zurich (or any other CEST one) run 
> 
> - `date 10280250.00`
> - initdb, start scheduler, webserver, enable dag etc.
> - `date 10280259.00`
> - wait 5-10 mins for scheduler to catch up
> - After the on-the-hour task run the scheduler will spin up another 
> process to parse the dag... and it never returns.
> 
> I've only just managed to reproduce it, so haven't dug in to why yet. A 
> quick hacky debug print shows something is stuck in an infinite loop.
> 
> -ash
> 
>> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
>> 
>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>> definition code.
>> 
>> On the licensing requirements:
>> 
>> 1. Indeed licensing header for markdown documents. It was suggested to 
>> use html comments. I’m not sure how that renders with others like PDF 
>> though.
>> 2. The licensing notifications need to be tied to a specific version as 
>> licenses might change with versions.
>> 
>> Cheers
>> Bolke
>> 
>> Verstuurd vanaf mijn iPad
>> 
>>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
>>> volgende geschreven:
>>> 
>>> I was going to make a start on the release, but two people have 
>>> reported that there might be an issue around non-UTC dags and the 
>>> scheduler changing over from Summer time.
>>> 
 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
 issue : we have hourly DAGs with a start_date in a local timezone (not 
 UTC) and since (Sunday) the last winter time change they don’t run 
 anymore. Any idea ?
 09:41  it impacted all our DAG that had a run at 3am 
 (Europe/Paris), the exact time of winter time change :(
>>> 
>>> I am going to take a look at this today and see if I can get to the 
>>> bottom of it.
>>> 
>>> Bolke: are there any outstanding tasks/issues that you know of that 
>>> might slow down the vote for a 1.10.1? (i.e. did we sort of out all the 
>>> licensing issues that were asked of us? I thought I read something 
>>> about license declarations in markdown files?)
>>> 
>>> -ash
>>> 
 On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
 
 I agree with that, but I would favor time based releases instead. We 
 are again at the point that a release takes so much time that the gap 
 is getting really big again. @ash why not start releasing now and move 
 the remainder to 1.10.2? I dont think there are real blockers 
 (although we might find them).
 
 
> On 28 Oct 2018, at 15:35, airflowuser 
>  wrote:

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
I think DAGs will start running by themselves again as soon as the interval has 
passed that it has already seen. So depending your schedule this can take a 
while.

B.

> On 30 Oct 2018, at 19:53, a...@apache.org wrote:
> 
> 1.10.1 isn't out yet, so 1.10.0 :)
> 
> I think this only affects days with a schedule interval that is one hour or 
> more frequently, and where the dag timezone is not UTC.
> 
> For anyone stuck on this in prod I think the fix is to manually create (via 
> backfill or trigger) a dag run after the tz change over time. This should 
> unblock the scheduler.
> 
> Ash
> 
> On 30 October 2018 18:43:30 GMT, David Klosowski  wrote:
>> Hi Airflow Devs:
>> 
>> Is this timezone issue in Airflow version 1.10.0 or only in 1.10.1?
>> 
>> Thanks.
>> 
>> Regards,
>> David
>> 
>> On Tue, Oct 30, 2018 at 11:11 AM Bolke de Bruin 
>> wrote:
>> 
>>> we specifically remove timezone info to determine the next schedule.
>> Ie.
>>> cron sets exact date times so tz info should not make sense. I’m
>> going to
>>> have a look now.
>>> 
>>> 
 On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
 
 I've done a bit more digging - the issue is of our tz-aware
>> handling
>>> inside following_schedule (and previous schedule) - causing it to
>> loop.
 
 This section of the croniter docs seems relevant
>>> https://github.com/kiorky/croniter#about-dst
 
   Be sure to init your croniter instance with a TZ aware datetime
>> for
>>> this to work !:
>>> local_date = tz.localize(datetime(2017, 3, 26))
>>> val = croniter('0 0 * * *', local_date).get_next(datetime)
 
 I think the problem is that we are _not_ passing a TZ aware dag in
>> and
>>> we should be.
 
> On 30 Oct 2018, at 17:35, Bolke de Bruin 
>> wrote:
> 
> Oh that’s a great environment to start digging. Thanks. I’ll have
>> a
>>> look.
> 
> B.
> 
> Verstuurd vanaf mijn iPad
> 
>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor 
>> het
>>> volgende geschreven:
>> 
>> This line in airflow.jobs (line 874 in my checkout) is causing
>> the
>>> loop:
>> 
>> last_run = dag.get_last_dagrun(session=session)
>> if last_run and next_run_date:
>> while next_run_date <= last_run.execution_date:
>> next_run_date =
>> dag.following_schedule(next_run_date)
>> 
>> 
>> 
>>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor 
>> wrote:
>>> 
>>> Hi, kaczors on gitter has produced a minmal reproduction case:
>>> https://github.com/kaczors/airflow_1_10_tz_bug
>>> 
>>> Rough repro steps: In a VM, with time syncing disabled, and
>>> configured with system timezone of Europe/Zurich (or any other CEST
>> one)
>>> run
>>> 
>>> - `date 10280250.00`
>>> - initdb, start scheduler, webserver, enable dag etc.
>>> - `date 10280259.00`
>>> - wait 5-10 mins for scheduler to catch up
>>> - After the on-the-hour task run the scheduler will spin up
>> another
>>> process to parse the dag... and it never returns.
>>> 
>>> I've only just managed to reproduce it, so haven't dug in to why
>> yet.
>>> A quick hacky debug print shows something is stuck in an infinite
>> loop.
>>> 
>>> -ash
>>> 
 On 29 Oct 2018, at 17:59, Bolke de Bruin 
>> wrote:
 
 Can this be confirmed? Then I can have a look at it. Preferably
>> with
>>> dag definition code.
 
 On the licensing requirements:
 
 1. Indeed licensing header for markdown documents. It was
>> suggested
>>> to use html comments. I’m not sure how that renders with others like
>> PDF
>>> though.
 2. The licensing notifications need to be tied to a specific
>> version
>>> as licenses might change with versions.
 
 Cheers
 Bolke
 
 Verstuurd vanaf mijn iPad
 
> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor
>> 
>>> het volgende geschreven:
> 
> I was going to make a start on the release, but two people
>> have
>>> reported that there might be an issue around non-UTC dags and the
>> scheduler
>>> changing over from Summer time.
> 
>> 08:45 Emmanuel> Hi there, we are currently experiencing a
>> very
>>> strange issue : we have hourly DAGs with a start_date in a local
>> timezone
>>> (not UTC) and since (Sunday) the last winter time change they don’t
>> run
>>> anymore. Any idea ?
>> 09:41  it impacted all our DAG that had a run at
>> 3am
>>> (Europe/Paris), the exact time of winter time change :(
> 
> I am going to take a look at this today and see if I can get
>> to the
>>> bottom of it.
> 
> Bolke: are there any outstanding tasks/issues that you know of
>> that
>>> might slow down the vote for a 1.10.1? (i.e. did we sort of out all
>> the
>>> licensing issues that were asked of us? I thought I read something
>> about
>>> license declarations in 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
I wonder how to treat this:

This is what I think happens (need to verify more, but I am pretty sure) the 
specified DAG should run every 5 minutes. At DST change (3AM -> 2AM) we 
basically hit a schedule that we have already seen. 2AM -> 3AM has already 
happened. Obviously the intention is to run every 5 minutes. But what do we do 
with the execution_date? Is this still idempotent? Should we indeed reschedule? 

B.

> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> 
> I've done a bit more digging - the issue is of our tz-aware handling inside 
> following_schedule (and previous schedule) - causing it to loop.
> 
> This section of the croniter docs seems relevant 
> https://github.com/kiorky/croniter#about-dst
> 
>Be sure to init your croniter instance with a TZ aware datetime for this 
> to work !:
 local_date = tz.localize(datetime(2017, 3, 26))
 val = croniter('0 0 * * *', local_date).get_next(datetime)
> 
> I think the problem is that we are _not_ passing a TZ aware dag in and we 
> should be.
> 
>> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
>> 
>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>> 
>> B.
>> 
>> Verstuurd vanaf mijn iPad
>> 
>>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
>>> volgende geschreven:
>>> 
>>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>>> 
>>>  last_run = dag.get_last_dagrun(session=session)
>>>  if last_run and next_run_date:
>>>  while next_run_date <= last_run.execution_date:
>>>  next_run_date = dag.following_schedule(next_run_date)
>>> 
>>> 
>>> 
 On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
 
 Hi, kaczors on gitter has produced a minmal reproduction case: 
 https://github.com/kaczors/airflow_1_10_tz_bug
 
 Rough repro steps: In a VM, with time syncing disabled, and configured 
 with system timezone of Europe/Zurich (or any other CEST one) run 
 
 - `date 10280250.00`
 - initdb, start scheduler, webserver, enable dag etc.
 - `date 10280259.00`
 - wait 5-10 mins for scheduler to catch up
 - After the on-the-hour task run the scheduler will spin up another 
 process to parse the dag... and it never returns.
 
 I've only just managed to reproduce it, so haven't dug in to why yet. A 
 quick hacky debug print shows something is stuck in an infinite loop.
 
 -ash
 
> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> 
> Can this be confirmed? Then I can have a look at it. Preferably with dag 
> definition code.
> 
> On the licensing requirements:
> 
> 1. Indeed licensing header for markdown documents. It was suggested to 
> use html comments. I’m not sure how that renders with others like PDF 
> though.
> 2. The licensing notifications need to be tied to a specific version as 
> licenses might change with versions.
> 
> Cheers
> Bolke
> 
> Verstuurd vanaf mijn iPad
> 
>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
>> volgende geschreven:
>> 
>> I was going to make a start on the release, but two people have reported 
>> that there might be an issue around non-UTC dags and the scheduler 
>> changing over from Summer time.
>> 
>>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
>>> issue : we have hourly DAGs with a start_date in a local timezone (not 
>>> UTC) and since (Sunday) the last winter time change they don’t run 
>>> anymore. Any idea ?
>>> 09:41  it impacted all our DAG that had a run at 3am 
>>> (Europe/Paris), the exact time of winter time change :(
>> 
>> I am going to take a look at this today and see if I can get to the 
>> bottom of it.
>> 
>> Bolke: are there any outstanding tasks/issues that you know of that 
>> might slow down the vote for a 1.10.1? (i.e. did we sort of out all the 
>> licensing issues that were asked of us? I thought I read something about 
>> license declarations in markdown files?)
>> 
>> -ash
>> 
>>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
>>> 
>>> I agree with that, but I would favor time based releases instead. We 
>>> are again at the point that a release takes so much time that the gap 
>>> is getting really big again. @ash why not start releasing now and move 
>>> the remainder to 1.10.2? I dont think there are real blockers (although 
>>> we might find them).
>>> 
>>> 
 On 28 Oct 2018, at 15:35, airflowuser 
  wrote:
 
 I was really hoping that 
 https://github.com/apache/incubator-airflow/pull/4069 will be merged 
 into 1.10.1
 Deleting dags was a highly requested feature for 1.10 - this can fix 
 the problem with it.
 
 
 ‐‐‐ Original Message ‐‐‐
>> On Friday, 

Re: 1.10.1 Release?

2018-10-30 Thread David Klosowski
Hi Airflow Devs:

Is this timezone issue in Airflow version 1.10.0 or only in 1.10.1?

Thanks.

Regards,
David

On Tue, Oct 30, 2018 at 11:11 AM Bolke de Bruin  wrote:

> we specifically remove timezone info to determine the next schedule. Ie.
> cron sets exact date times so tz info should not make sense. I’m going to
> have a look now.
>
>
> > On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> >
> > I've done a bit more digging - the issue is of our tz-aware handling
> inside following_schedule (and previous schedule) - causing it to loop.
> >
> > This section of the croniter docs seems relevant
> https://github.com/kiorky/croniter#about-dst
> >
> >Be sure to init your croniter instance with a TZ aware datetime for
> this to work !:
>  local_date = tz.localize(datetime(2017, 3, 26))
>  val = croniter('0 0 * * *', local_date).get_next(datetime)
> >
> > I think the problem is that we are _not_ passing a TZ aware dag in and
> we should be.
> >
> >> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
> >>
> >> Oh that’s a great environment to start digging. Thanks. I’ll have a
> look.
> >>
> >> B.
> >>
> >> Verstuurd vanaf mijn iPad
> >>
> >>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het
> volgende geschreven:
> >>>
> >>> This line in airflow.jobs (line 874 in my checkout) is causing the
> loop:
> >>>
> >>>  last_run = dag.get_last_dagrun(session=session)
> >>>  if last_run and next_run_date:
> >>>  while next_run_date <= last_run.execution_date:
> >>>  next_run_date = dag.following_schedule(next_run_date)
> >>>
> >>>
> >>>
>  On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
> 
>  Hi, kaczors on gitter has produced a minmal reproduction case:
> https://github.com/kaczors/airflow_1_10_tz_bug
> 
>  Rough repro steps: In a VM, with time syncing disabled, and
> configured with system timezone of Europe/Zurich (or any other CEST one)
> run
> 
>  - `date 10280250.00`
>  - initdb, start scheduler, webserver, enable dag etc.
>  - `date 10280259.00`
>  - wait 5-10 mins for scheduler to catch up
>  - After the on-the-hour task run the scheduler will spin up another
> process to parse the dag... and it never returns.
> 
>  I've only just managed to reproduce it, so haven't dug in to why yet.
> A quick hacky debug print shows something is stuck in an infinite loop.
> 
>  -ash
> 
> > On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> >
> > Can this be confirmed? Then I can have a look at it. Preferably with
> dag definition code.
> >
> > On the licensing requirements:
> >
> > 1. Indeed licensing header for markdown documents. It was suggested
> to use html comments. I’m not sure how that renders with others like PDF
> though.
> > 2. The licensing notifications need to be tied to a specific version
> as licenses might change with versions.
> >
> > Cheers
> > Bolke
> >
> > Verstuurd vanaf mijn iPad
> >
> >> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor 
> het volgende geschreven:
> >>
> >> I was going to make a start on the release, but two people have
> reported that there might be an issue around non-UTC dags and the scheduler
> changing over from Summer time.
> >>
> >>> 08:45 Emmanuel> Hi there, we are currently experiencing a very
> strange issue : we have hourly DAGs with a start_date in a local timezone
> (not UTC) and since (Sunday) the last winter time change they don’t run
> anymore. Any idea ?
> >>> 09:41  it impacted all our DAG that had a run at 3am
> (Europe/Paris), the exact time of winter time change :(
> >>
> >> I am going to take a look at this today and see if I can get to the
> bottom of it.
> >>
> >> Bolke: are there any outstanding tasks/issues that you know of that
> might slow down the vote for a 1.10.1? (i.e. did we sort of out all the
> licensing issues that were asked of us? I thought I read something about
> license declarations in markdown files?)
> >>
> >> -ash
> >>
> >>> On 28 Oct 2018, at 14:46, Bolke de Bruin 
> wrote:
> >>>
> >>> I agree with that, but I would favor time based releases instead.
> We are again at the point that a release takes so much time that the gap is
> getting really big again. @ash why not start releasing now and move the
> remainder to 1.10.2? I dont think there are real blockers (although we
> might find them).
> >>>
> >>>
>  On 28 Oct 2018, at 15:35, airflowuser 
>  
> wrote:
> 
>  I was really hoping that
> https://github.com/apache/incubator-airflow/pull/4069 will be merged into
> 1.10.1
>  Deleting dags was a highly requested feature for 1.10 - this can
> fix the problem with it.
> 
> 
>  ‐‐‐ Original Message ‐‐‐
> >> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin <
> bdbr...@gmail.com> wrote:
> >
> > Hey 

Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
we specifically remove timezone info to determine the next schedule. Ie. cron 
sets exact date times so tz info should not make sense. I’m going to have a 
look now.


> On 30 Oct 2018, at 19:01, Ash Berlin-Taylor  wrote:
> 
> I've done a bit more digging - the issue is of our tz-aware handling inside 
> following_schedule (and previous schedule) - causing it to loop.
> 
> This section of the croniter docs seems relevant 
> https://github.com/kiorky/croniter#about-dst
> 
>Be sure to init your croniter instance with a TZ aware datetime for this 
> to work !:
 local_date = tz.localize(datetime(2017, 3, 26))
 val = croniter('0 0 * * *', local_date).get_next(datetime)
> 
> I think the problem is that we are _not_ passing a TZ aware dag in and we 
> should be.
> 
>> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
>> 
>> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
>> 
>> B.
>> 
>> Verstuurd vanaf mijn iPad
>> 
>>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
>>> volgende geschreven:
>>> 
>>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>>> 
>>>  last_run = dag.get_last_dagrun(session=session)
>>>  if last_run and next_run_date:
>>>  while next_run_date <= last_run.execution_date:
>>>  next_run_date = dag.following_schedule(next_run_date)
>>> 
>>> 
>>> 
 On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
 
 Hi, kaczors on gitter has produced a minmal reproduction case: 
 https://github.com/kaczors/airflow_1_10_tz_bug
 
 Rough repro steps: In a VM, with time syncing disabled, and configured 
 with system timezone of Europe/Zurich (or any other CEST one) run 
 
 - `date 10280250.00`
 - initdb, start scheduler, webserver, enable dag etc.
 - `date 10280259.00`
 - wait 5-10 mins for scheduler to catch up
 - After the on-the-hour task run the scheduler will spin up another 
 process to parse the dag... and it never returns.
 
 I've only just managed to reproduce it, so haven't dug in to why yet. A 
 quick hacky debug print shows something is stuck in an infinite loop.
 
 -ash
 
> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> 
> Can this be confirmed? Then I can have a look at it. Preferably with dag 
> definition code.
> 
> On the licensing requirements:
> 
> 1. Indeed licensing header for markdown documents. It was suggested to 
> use html comments. I’m not sure how that renders with others like PDF 
> though.
> 2. The licensing notifications need to be tied to a specific version as 
> licenses might change with versions.
> 
> Cheers
> Bolke
> 
> Verstuurd vanaf mijn iPad
> 
>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
>> volgende geschreven:
>> 
>> I was going to make a start on the release, but two people have reported 
>> that there might be an issue around non-UTC dags and the scheduler 
>> changing over from Summer time.
>> 
>>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
>>> issue : we have hourly DAGs with a start_date in a local timezone (not 
>>> UTC) and since (Sunday) the last winter time change they don’t run 
>>> anymore. Any idea ?
>>> 09:41  it impacted all our DAG that had a run at 3am 
>>> (Europe/Paris), the exact time of winter time change :(
>> 
>> I am going to take a look at this today and see if I can get to the 
>> bottom of it.
>> 
>> Bolke: are there any outstanding tasks/issues that you know of that 
>> might slow down the vote for a 1.10.1? (i.e. did we sort of out all the 
>> licensing issues that were asked of us? I thought I read something about 
>> license declarations in markdown files?)
>> 
>> -ash
>> 
>>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
>>> 
>>> I agree with that, but I would favor time based releases instead. We 
>>> are again at the point that a release takes so much time that the gap 
>>> is getting really big again. @ash why not start releasing now and move 
>>> the remainder to 1.10.2? I dont think there are real blockers (although 
>>> we might find them).
>>> 
>>> 
 On 28 Oct 2018, at 15:35, airflowuser 
  wrote:
 
 I was really hoping that 
 https://github.com/apache/incubator-airflow/pull/4069 will be merged 
 into 1.10.1
 Deleting dags was a highly requested feature for 1.10 - this can fix 
 the problem with it.
 
 
 ‐‐‐ Original Message ‐‐‐
>> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin 
>>  wrote:
> 
> Hey Ash,
> 
> I was wondering if you are picking up the 1.10.1 release? Master is 
> speeding ahead and you were tracking fixes for 1.10.1 right?

Re: 1.10.1 Release?

2018-10-30 Thread Ash Berlin-Taylor
I've done a bit more digging - the issue is of our tz-aware handling inside 
following_schedule (and previous schedule) - causing it to loop.

This section of the croniter docs seems relevant 
https://github.com/kiorky/croniter#about-dst

Be sure to init your croniter instance with a TZ aware datetime for this to 
work !:
>>> local_date = tz.localize(datetime(2017, 3, 26))
>>> val = croniter('0 0 * * *', local_date).get_next(datetime)

I think the problem is that we are _not_ passing a TZ aware dag in and we 
should be.

> On 30 Oct 2018, at 17:35, Bolke de Bruin  wrote:
> 
> Oh that’s a great environment to start digging. Thanks. I’ll have a look.
> 
> B.
> 
> Verstuurd vanaf mijn iPad
> 
>> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
>> volgende geschreven:
>> 
>> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
>> 
>>   last_run = dag.get_last_dagrun(session=session)
>>   if last_run and next_run_date:
>>   while next_run_date <= last_run.execution_date:
>>   next_run_date = dag.following_schedule(next_run_date)
>> 
>> 
>> 
>>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
>>> 
>>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>>> https://github.com/kaczors/airflow_1_10_tz_bug
>>> 
>>> Rough repro steps: In a VM, with time syncing disabled, and configured with 
>>> system timezone of Europe/Zurich (or any other CEST one) run 
>>> 
>>> - `date 10280250.00`
>>> - initdb, start scheduler, webserver, enable dag etc.
>>> - `date 10280259.00`
>>> - wait 5-10 mins for scheduler to catch up
>>> - After the on-the-hour task run the scheduler will spin up another process 
>>> to parse the dag... and it never returns.
>>> 
>>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>>> quick hacky debug print shows something is stuck in an infinite loop.
>>> 
>>> -ash
>>> 
 On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
 
 Can this be confirmed? Then I can have a look at it. Preferably with dag 
 definition code.
 
 On the licensing requirements:
 
 1. Indeed licensing header for markdown documents. It was suggested to use 
 html comments. I’m not sure how that renders with others like PDF though.
 2. The licensing notifications need to be tied to a specific version as 
 licenses might change with versions.
 
 Cheers
 Bolke
 
 Verstuurd vanaf mijn iPad
 
> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> I was going to make a start on the release, but two people have reported 
> that there might be an issue around non-UTC dags and the scheduler 
> changing over from Summer time.
> 
>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
>> issue : we have hourly DAGs with a start_date in a local timezone (not 
>> UTC) and since (Sunday) the last winter time change they don’t run 
>> anymore. Any idea ?
>> 09:41  it impacted all our DAG that had a run at 3am 
>> (Europe/Paris), the exact time of winter time change :(
> 
> I am going to take a look at this today and see if I can get to the 
> bottom of it.
> 
> Bolke: are there any outstanding tasks/issues that you know of that might 
> slow down the vote for a 1.10.1? (i.e. did we sort of out all the 
> licensing issues that were asked of us? I thought I read something about 
> license declarations in markdown files?)
> 
> -ash
> 
>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
>> 
>> I agree with that, but I would favor time based releases instead. We are 
>> again at the point that a release takes so much time that the gap is 
>> getting really big again. @ash why not start releasing now and move the 
>> remainder to 1.10.2? I dont think there are real blockers (although we 
>> might find them).
>> 
>> 
>>> On 28 Oct 2018, at 15:35, airflowuser 
>>>  wrote:
>>> 
>>> I was really hoping that 
>>> https://github.com/apache/incubator-airflow/pull/4069 will be merged 
>>> into 1.10.1
>>> Deleting dags was a highly requested feature for 1.10 - this can fix 
>>> the problem with it.
>>> 
>>> 
>>> ‐‐‐ Original Message ‐‐‐
> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin 
>  wrote:
 
 Hey Ash,
 
 I was wondering if you are picking up the 1.10.1 release? Master is 
 speeding ahead and you were tracking fixes for 1.10.1 right?
 
 B.
> 
>>> 
>> 



Re: 1.10.1 Release?

2018-10-30 Thread Bolke de Bruin
Oh that’s a great environment to start digging. Thanks. I’ll have a look.

B.

Verstuurd vanaf mijn iPad

> Op 30 okt. 2018 om 18:25 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> This line in airflow.jobs (line 874 in my checkout) is causing the loop:
> 
>last_run = dag.get_last_dagrun(session=session)
>if last_run and next_run_date:
>while next_run_date <= last_run.execution_date:
>next_run_date = dag.following_schedule(next_run_date)
> 
> 
> 
>> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
>> 
>> Hi, kaczors on gitter has produced a minmal reproduction case: 
>> https://github.com/kaczors/airflow_1_10_tz_bug
>> 
>> Rough repro steps: In a VM, with time syncing disabled, and configured with 
>> system timezone of Europe/Zurich (or any other CEST one) run 
>> 
>> - `date 10280250.00`
>> - initdb, start scheduler, webserver, enable dag etc.
>> - `date 10280259.00`
>> - wait 5-10 mins for scheduler to catch up
>> - After the on-the-hour task run the scheduler will spin up another process 
>> to parse the dag... and it never returns.
>> 
>> I've only just managed to reproduce it, so haven't dug in to why yet. A 
>> quick hacky debug print shows something is stuck in an infinite loop.
>> 
>> -ash
>> 
>>> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
>>> 
>>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>>> definition code.
>>> 
>>> On the licensing requirements:
>>> 
>>> 1. Indeed licensing header for markdown documents. It was suggested to use 
>>> html comments. I’m not sure how that renders with others like PDF though.
>>> 2. The licensing notifications need to be tied to a specific version as 
>>> licenses might change with versions.
>>> 
>>> Cheers
>>> Bolke
>>> 
>>> Verstuurd vanaf mijn iPad
>>> 
 Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
 volgende geschreven:
 
 I was going to make a start on the release, but two people have reported 
 that there might be an issue around non-UTC dags and the scheduler 
 changing over from Summer time.
 
> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
> issue : we have hourly DAGs with a start_date in a local timezone (not 
> UTC) and since (Sunday) the last winter time change they don’t run 
> anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am 
> (Europe/Paris), the exact time of winter time change :(
 
 I am going to take a look at this today and see if I can get to the bottom 
 of it.
 
 Bolke: are there any outstanding tasks/issues that you know of that might 
 slow down the vote for a 1.10.1? (i.e. did we sort of out all the 
 licensing issues that were asked of us? I thought I read something about 
 license declarations in markdown files?)
 
 -ash
 
> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> 
> I agree with that, but I would favor time based releases instead. We are 
> again at the point that a release takes so much time that the gap is 
> getting really big again. @ash why not start releasing now and move the 
> remainder to 1.10.2? I dont think there are real blockers (although we 
> might find them).
> 
> 
>> On 28 Oct 2018, at 15:35, airflowuser 
>>  wrote:
>> 
>> I was really hoping that 
>> https://github.com/apache/incubator-airflow/pull/4069 will be merged 
>> into 1.10.1
>> Deleting dags was a highly requested feature for 1.10 - this can fix the 
>> problem with it.
>> 
>> 
>> ‐‐‐ Original Message ‐‐‐
 On Friday, October 26, 2018 6:12 PM, Bolke de Bruin 
  wrote:
>>> 
>>> Hey Ash,
>>> 
>>> I was wondering if you are picking up the 1.10.1 release? Master is 
>>> speeding ahead and you were tracking fixes for 1.10.1 right?
>>> 
>>> B.
 
>> 
> 


Re: 1.10.1 Release?

2018-10-30 Thread Ash Berlin-Taylor
This line in airflow.jobs (line 874 in my checkout) is causing the loop:

last_run = dag.get_last_dagrun(session=session)
if last_run and next_run_date:
while next_run_date <= last_run.execution_date:
next_run_date = dag.following_schedule(next_run_date)



> On 30 Oct 2018, at 17:20, Ash Berlin-Taylor  wrote:
> 
> Hi, kaczors on gitter has produced a minmal reproduction case: 
> https://github.com/kaczors/airflow_1_10_tz_bug
> 
> Rough repro steps: In a VM, with time syncing disabled, and configured with 
> system timezone of Europe/Zurich (or any other CEST one) run 
> 
> - `date 10280250.00`
> - initdb, start scheduler, webserver, enable dag etc.
> - `date 10280259.00`
> - wait 5-10 mins for scheduler to catch up
> - After the on-the-hour task run the scheduler will spin up another process 
> to parse the dag... and it never returns.
> 
> I've only just managed to reproduce it, so haven't dug in to why yet. A quick 
> hacky debug print shows something is stuck in an infinite loop.
> 
> -ash
> 
>> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
>> 
>> Can this be confirmed? Then I can have a look at it. Preferably with dag 
>> definition code.
>> 
>> On the licensing requirements:
>> 
>> 1. Indeed licensing header for markdown documents. It was suggested to use 
>> html comments. I’m not sure how that renders with others like PDF though.
>> 2. The licensing notifications need to be tied to a specific version as 
>> licenses might change with versions.
>> 
>> Cheers
>> Bolke
>> 
>> Verstuurd vanaf mijn iPad
>> 
>>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
>>> volgende geschreven:
>>> 
>>> I was going to make a start on the release, but two people have reported 
>>> that there might be an issue around non-UTC dags and the scheduler changing 
>>> over from Summer time.
>>> 
 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
 issue : we have hourly DAGs with a start_date in a local timezone (not 
 UTC) and since (Sunday) the last winter time change they don’t run 
 anymore. Any idea ?
 09:41  it impacted all our DAG that had a run at 3am 
 (Europe/Paris), the exact time of winter time change :(
>>> 
>>> I am going to take a look at this today and see if I can get to the bottom 
>>> of it.
>>> 
>>> Bolke: are there any outstanding tasks/issues that you know of that might 
>>> slow down the vote for a 1.10.1? (i.e. did we sort of out all the licensing 
>>> issues that were asked of us? I thought I read something about license 
>>> declarations in markdown files?)
>>> 
>>> -ash
>>> 
 On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
 
 I agree with that, but I would favor time based releases instead. We are 
 again at the point that a release takes so much time that the gap is 
 getting really big again. @ash why not start releasing now and move the 
 remainder to 1.10.2? I dont think there are real blockers (although we 
 might find them).
 
 
> On 28 Oct 2018, at 15:35, airflowuser 
>  wrote:
> 
> I was really hoping that 
> https://github.com/apache/incubator-airflow/pull/4069 will be merged into 
> 1.10.1
> Deleting dags was a highly requested feature for 1.10 - this can fix the 
> problem with it.
> 
> 
> ‐‐‐ Original Message ‐‐‐
>>> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  
>>> wrote:
>> 
>> Hey Ash,
>> 
>> I was wondering if you are picking up the 1.10.1 release? Master is 
>> speeding ahead and you were tracking fixes for 1.10.1 right?
>> 
>> B.
>>> 
> 



Re: 1.10.1 Release?

2018-10-30 Thread Ash Berlin-Taylor
Hi, kaczors on gitter has produced a minmal reproduction case: 
https://github.com/kaczors/airflow_1_10_tz_bug

Rough repro steps: In a VM, with time syncing disabled, and configured with 
system timezone of Europe/Zurich (or any other CEST one) run 

- `date 10280250.00`
- initdb, start scheduler, webserver, enable dag etc.
- `date 10280259.00`
- wait 5-10 mins for scheduler to catch up
- After the on-the-hour task run the scheduler will spin up another process to 
parse the dag... and it never returns.

I've only just managed to reproduce it, so haven't dug in to why yet. A quick 
hacky debug print shows something is stuck in an infinite loop.

-ash

> On 29 Oct 2018, at 17:59, Bolke de Bruin  wrote:
> 
> Can this be confirmed? Then I can have a look at it. Preferably with dag 
> definition code.
> 
> On the licensing requirements:
> 
> 1. Indeed licensing header for markdown documents. It was suggested to use 
> html comments. I’m not sure how that renders with others like PDF though.
> 2. The licensing notifications need to be tied to a specific version as 
> licenses might change with versions.
> 
> Cheers
> Bolke
> 
> Verstuurd vanaf mijn iPad
> 
>> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
>> volgende geschreven:
>> 
>> I was going to make a start on the release, but two people have reported 
>> that there might be an issue around non-UTC dags and the scheduler changing 
>> over from Summer time.
>> 
>>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange 
>>> issue : we have hourly DAGs with a start_date in a local timezone (not UTC) 
>>> and since (Sunday) the last winter time change they don’t run anymore. Any 
>>> idea ?
>>> 09:41  it impacted all our DAG that had a run at 3am 
>>> (Europe/Paris), the exact time of winter time change :(
>> 
>> I am going to take a look at this today and see if I can get to the bottom 
>> of it.
>> 
>> Bolke: are there any outstanding tasks/issues that you know of that might 
>> slow down the vote for a 1.10.1? (i.e. did we sort of out all the licensing 
>> issues that were asked of us? I thought I read something about license 
>> declarations in markdown files?)
>> 
>> -ash
>> 
>>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
>>> 
>>> I agree with that, but I would favor time based releases instead. We are 
>>> again at the point that a release takes so much time that the gap is 
>>> getting really big again. @ash why not start releasing now and move the 
>>> remainder to 1.10.2? I dont think there are real blockers (although we 
>>> might find them).
>>> 
>>> 
 On 28 Oct 2018, at 15:35, airflowuser  
 wrote:
 
 I was really hoping that 
 https://github.com/apache/incubator-airflow/pull/4069 will be merged into 
 1.10.1
 Deleting dags was a highly requested feature for 1.10 - this can fix the 
 problem with it.
 
 
 ‐‐‐ Original Message ‐‐‐
>> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  
>> wrote:
> 
> Hey Ash,
> 
> I was wondering if you are picking up the 1.10.1 release? Master is 
> speeding ahead and you were tracking fixes for 1.10.1 right?
> 
> B.
>> 



Re: 1.10.1 Release?

2018-10-29 Thread Bolke de Bruin
Can this be confirmed? Then I can have a look at it. Preferably with dag 
definition code.

On the licensing requirements:

1. Indeed licensing header for markdown documents. It was suggested to use html 
comments. I’m not sure how that renders with others like PDF though.
2. The licensing notifications need to be tied to a specific version as 
licenses might change with versions.

Cheers
Bolke

Verstuurd vanaf mijn iPad

> Op 29 okt. 2018 om 12:39 heeft Ash Berlin-Taylor  het 
> volgende geschreven:
> 
> I was going to make a start on the release, but two people have reported that 
> there might be an issue around non-UTC dags and the scheduler changing over 
> from Summer time.
> 
>> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
>> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
>> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
>> 09:41  it impacted all our DAG that had a run at 3am 
>> (Europe/Paris), the exact time of winter time change :(
> 
> I am going to take a look at this today and see if I can get to the bottom of 
> it.
> 
> Bolke: are there any outstanding tasks/issues that you know of that might 
> slow down the vote for a 1.10.1? (i.e. did we sort of out all the licensing 
> issues that were asked of us? I thought I read something about license 
> declarations in markdown files?)
> 
> -ash
> 
>> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
>> 
>> I agree with that, but I would favor time based releases instead. We are 
>> again at the point that a release takes so much time that the gap is getting 
>> really big again. @ash why not start releasing now and move the remainder to 
>> 1.10.2? I dont think there are real blockers (although we might find them).
>> 
>> 
>>> On 28 Oct 2018, at 15:35, airflowuser  
>>> wrote:
>>> 
>>> I was really hoping that 
>>> https://github.com/apache/incubator-airflow/pull/4069 will be merged into 
>>> 1.10.1
>>> Deleting dags was a highly requested feature for 1.10 - this can fix the 
>>> problem with it.
>>> 
>>> 
>>> ‐‐‐ Original Message ‐‐‐
> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  
> wrote:
 
 Hey Ash,
 
 I was wondering if you are picking up the 1.10.1 release? Master is 
 speeding ahead and you were tracking fixes for 1.10.1 right?
 
 B.
> 


Re: 1.10.1 Release?

2018-10-29 Thread Ash Berlin-Taylor
I was going to make a start on the release, but two people have reported that 
there might be an issue around non-UTC dags and the scheduler changing over 
from Summer time.

> 08:45 Emmanuel> Hi there, we are currently experiencing a very strange issue 
> : we have hourly DAGs with a start_date in a local timezone (not UTC) and 
> since (Sunday) the last winter time change they don’t run anymore. Any idea ?
> 09:41  it impacted all our DAG that had a run at 3am 
> (Europe/Paris), the exact time of winter time change :(

I am going to take a look at this today and see if I can get to the bottom of 
it.

Bolke: are there any outstanding tasks/issues that you know of that might slow 
down the vote for a 1.10.1? (i.e. did we sort of out all the licensing issues 
that were asked of us? I thought I read something about license declarations in 
markdown files?)

-ash

> On 28 Oct 2018, at 14:46, Bolke de Bruin  wrote:
> 
> I agree with that, but I would favor time based releases instead. We are 
> again at the point that a release takes so much time that the gap is getting 
> really big again. @ash why not start releasing now and move the remainder to 
> 1.10.2? I dont think there are real blockers (although we might find them).
> 
> 
>> On 28 Oct 2018, at 15:35, airflowuser  
>> wrote:
>> 
>> I was really hoping that 
>> https://github.com/apache/incubator-airflow/pull/4069 will be merged into 
>> 1.10.1
>> Deleting dags was a highly requested feature for 1.10 - this can fix the 
>> problem with it.
>> 
>> 
>> ‐‐‐ Original Message ‐‐‐
>> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  
>> wrote:
>> 
>>> Hey Ash,
>>> 
>>> I was wondering if you are picking up the 1.10.1 release? Master is 
>>> speeding ahead and you were tracking fixes for 1.10.1 right?
>>> 
>>> B.
>> 
>> 
> 



Re: 1.10.1 Release?

2018-10-28 Thread Bolke de Bruin
I agree with that, but I would favor time based releases instead. We are again 
at the point that a release takes so much time that the gap is getting really 
big again. @ash why not start releasing now and move the remainder to 1.10.2? I 
dont think there are real blockers (although we might find them).


> On 28 Oct 2018, at 15:35, airflowuser  
> wrote:
> 
> I was really hoping that 
> https://github.com/apache/incubator-airflow/pull/4069 will be merged into 
> 1.10.1
> Deleting dags was a highly requested feature for 1.10 - this can fix the 
> problem with it.
> 
> 
> ‐‐‐ Original Message ‐‐‐
> On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  wrote:
> 
>> Hey Ash,
>> 
>> I was wondering if you are picking up the 1.10.1 release? Master is speeding 
>> ahead and you were tracking fixes for 1.10.1 right?
>> 
>> B.
> 
> 



Re: 1.10.1 Release?

2018-10-28 Thread airflowuser
I was really hoping that https://github.com/apache/incubator-airflow/pull/4069 
will be merged into 1.10.1
Deleting dags was a highly requested feature for 1.10 - this can fix the 
problem with it.


‐‐‐ Original Message ‐‐‐
On Friday, October 26, 2018 6:12 PM, Bolke de Bruin  wrote:

> Hey Ash,
>
> I was wondering if you are picking up the 1.10.1 release? Master is speeding 
> ahead and you were tracking fixes for 1.10.1 right?
>
> B.




Re: 1.10.1 Release?

2018-10-26 Thread Ash Berlin-Taylor
Hey, yeah I've been working (a bit slowly) on it - we had a few test failures 
on the v1-10-test branch (conflicts/bad cherry-picks) to unravel to get green 
builds, but we're there now.

A rough summary of where we are (wide columns, won't look good if it has to 
line-wrap, https://gist.github.com/ashb/4285c5a6c6b3be616495b1181d3fd63e to see 
it in another format):

(airflow) (themisto python/incuba…irflow v1-10-test:+)% ./dev/airflow-jira 
compare 1.10.1
ISSUE ID  |TYPE||PRIORITY  ||STATUS|DESCRIPTION 
  |MERGED|PR|COMMIT
AIRFLOW-3238  |Bug ||Major ||Resolved  |Dags, removed from the 
filesystem, are not deactiv|1 |#na   
|1eeb0a4a24fa8541763a67f84ec9f4b034f66475
AIRFLOW-3237  |Improvement ||Major ||Resolved  |Refactor example DAGs   
  |1 |#na   
|fdfb359e4b95dfadfa3973d43025f61f4aa3b96a
AIRFLOW-3232  |Improvement ||Trivial   ||Resolved  |Make documentation for 
GCF Functions operator more|1 |#na   
|d4dff076a6eaf169424822c0010c802f4af80c6a
AIRFLOW-3203  |Bug ||Critical  ||Closed|Bugs in DockerOperator 
& Some operator test script|1 |#na   
|3dfc9562d3df127ca8337edb600ab6a0259521ac
AIRFLOW-3197  |Improvement ||Minor ||Resolved  |EMR Hook is missing 
some parameters to valid on th|1 |#na   
|079b0ee95e4a1d37bdb31c477b531db264242bf7
AIRFLOW-3195  |Improvement ||Trivial   ||Resolved  |Druid Hook: Log 
ingestion spec and task id|1 |#na   
|8e55b499b2dcc6e4dc4d86f8225d1424f6886a0c
AIRFLOW-3187  |Wish||Major ||Resolved  |Update airflow.gif file 
with a slower version |0 |- |-
AIRFLOW-3183  |Bug ||Minor ||Resolved  |Potential Bug in 
utils/dag_processing/DagFileProce|1 |#na   
|0e98c60268c805639be8cd8c1cdf9a8909f966bb
AIRFLOW-3178  |Bug ||Major ||Resolved  |`airflow run` config 
doens't cope with % in config|1 |#na   
|5bc4cfa4a7908818877828ed3db1090b71e65b93
AIRFLOW-3177  |Improvement ||Minor ||Resolved  |Change 
scheduler_heartbeat metric from gauge to co|1 |#na   
|cab121bc036be24bcb1e397f48ca1672e1f73692
AIRFLOW-3173  |Improvement ||Major ||Resolved  |Add _cmd options for 
password config options  |1 |#na   
|e2a238c3a55be5fcb17b5ec64e7165dd954a4bc0
AIRFLOW-3172  |Bug ||Major ||Open  |AttributeError: 
'DagModel' object has no attribute|0 |- |-
AIRFLOW-3162  |Bug ||Minor ||Resolved  |HttpHook fails to parse 
URL when port is specified|1 |#na   
|040707b5d830acb96fc6c9a367039ed96aba7231
AIRFLOW-3147  |New Feature ||Major ||Resolved  |Update Flask-AppBuilder 
version   |1 |#na   
|35f996b65a2e68f4e168e3f466f3a939e4fb904a
AIRFLOW-3138  |Bug ||Major ||Resolved  |Migration cc1e65623dc7 
creates issues with postgre|1 |#na   
|1234af2995780327517c770bd09daf684563527d
AIRFLOW-3137  |Improvement ||Trivial   ||Resolved  |Make ProxyFix 
middleware optional |0 |- |-
AIRFLOW-3127  |Improvement ||Minor ||Resolved  |Celery SSL 
Documentation is out-dated |0 |- |-
AIRFLOW-3124  |Bug ||Minor ||Resolved  |Broken webserver debug 
mode (RBAC)|1 |#na   
|f9988147999b73f10978f74d37f704dae8e4f012
AIRFLOW-3119  |Improvement ||Minor ||Resolved  |Enable loglevel on 
celery worker and inherit from |1 |#na   
|23be2a30cd28f236722433a07c536cb17c8515ee
AIRFLOW-3114  |Improvement ||Minor ||Open  |Add feature to create 
External BigQuery Table for |0 |- |-
AIRFLOW-3112  |Improvement ||Minor ||Resolved  |Align SFTP hook with 
SSH hook |1 |#na   
|35969a426822e5fc9bc2b941b4fe5c5098a5ee52
AIRFLOW-3111  |Bug ||Minor ||Resolved  |Confusing comments and 
instructions for log templa|0 |- |-
AIRFLOW-3109  |Bug ||Major ||Resolved  |Default user permission 
should contain 'can_clear'|0 |- |-
AIRFLOW-3104  |Improvement ||Minor ||Resolved  |.airflowignore feature 
is not mentioned at all in |0 |- |-
AIRFLOW-3099  |Bug ||Minor ||Resolved  |Errors raised when some 
blocs are missing in airfl|1 |#na   
|25bb0dff64687948f73b3fb86fae4e8476f4f9ce
AIRFLOW-3090  |Wish||Minor ||Closed|INFO logs are too 
verbose |0 |- |-
AIRFLOW-3089  |Bug ||Minor ||Resolved  |Google auth doesn't 
work under http   |1 |#na   
|bd7510094de25f2c2bce2a49a3febae28899f3b6
AIRFLOW-3079  |Bug ||Major ||Resolved  |Improve initdb to 
support MSSQL Server|0 |- |-
AIRFLOW-3078  |New Feature ||Trivial   ||Resolved  |Basic operators for 
Google Compute Engine |1 |#na   
|6aeda0fcb2ecbba647a6cede992bf23556c1b05c
AIRFLOW-3072  |Bug ||Major