Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Robert Metzger
Hi,

@Kostas: Yes, it is a VOTE release candidate.

@Chesnay: I agree. That's something we can fix with the first bugfix
release.

@Till, regarding FLINK-6646: This is definitively an ugly issue, because it
breaks HA in many cases.
I agree that it would be really nice to have this fixed asap. On the other
hand, it was broken like this in 1.2. and probably 1.1 as well.
So I assume in practice no user is affected by this (I assume because
everybody on YARN uses the "per job YARN cluster" function).
I would propose not to cancel RC2 because of this, but include the fix as
soon as we have to cancel RC2. (I'll try to review it asap so that it
mergable).



Regards,
Robert


On Wed, May 24, 2017 at 5:56 PM, Till Rohrmann  wrote:

> I think that https://issues.apache.org/jira/browse/FLINK-6646 might
> qualify
> as a blocker issue.
>
> The problem is that the yarn application files are managed by the
> ClusterClient. Due to that the ClusterClient deletes them if it shuts down.
> The ClusterClient shut down is also executed in case of a connection loss
> to the JobManager (e.g. in case of a JM failure). Due to the file deletion,
> Yarn is then no longer able to restart the JobManager in HA mode which
> effectively thwarts Flink's HA capabilities.
>
> The problem is that the files should not be deleted by the ClusterClient
> once the Yarn application has been started. Instead the ApplicationMaster
> should be responsible for the deletion.
>
> Cheers,
> Till
>
> On Wed, May 24, 2017 at 4:53 PM, Timo Walther  wrote:
>
> > I tested the quickstarts, the SBT build, the PlanVisualizer, and the
> > HistoryServer. I could not find any serious problems. However, we have to
> > update the quickstart scripts, once 1.3 is released.
> >
> > Timo
> >
> >
> > Am 24.05.17 um 16:05 schrieb Chesnay Schepler:
> >
> > I've found a small problem in the yarn user-jar handling. We recently
> >> added a switch to disable the user-jar inclusion in the class path,
> which
> >> isn't working.
> >>
> >> PR is open, but i wouldn't consider this a release-blocker.
> >> https://github.com/apache/flink/pull/3979
> >>
> >> On 24.05.2017 15:45, Kostas Kloudas wrote:
> >>
> >>> Thanks Robert for pushing this.
> >>> This is going to be a voting RC?
> >>>
> >>> Kostas
> >>>
> >>> On May 24, 2017, at 3:16 PM, Robert Metzger 
> wrote:
> 
>  Great!
>  The fixes are in, and I'm now building the RC2 (There are currently no
>  blockers in our JIRA)
> 
>  On Tue, May 23, 2017 at 9:37 PM, Stefan Richter <
>  s.rich...@data-artisans.com
> 
> > wrote:
> > I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will
> > merge
> > tomorrow.
> >
> > Am 23.05.2017 um 21:05 schrieb Chesnay Schepler  >:
> >>
> >> It appears that rescaling is broken, I've filed
> >>
> > https://issues.apache.org/jira/browse/FLINK-6690 for that.
> >
> >> needless to say, this is a release blocker.
> >>
> >> On 23.05.2017 20:55, Robert Metzger wrote:
> >>
> >>> I know I'm talking to myself here :) Anyways, I was running into
> some
> >>> issues while creating the release (I was using master instead of
> the
> >>> release-1.3 branch, which lead to some issues with the scala 2.10 /
> >>> 2.11
> >>> switch).
> >>>
> >>> The RC2 is basically ready, however, there's at least one new
> >>> blocker:
> >>> https://issues.apache.org/jira/browse/FLINK-6685 which needs
> >>> addressing
> >>> first.
> >>>
> >>> Let me know if you want me to publish the new RC2. Otherwise, I'll
> >>>
> >> re-do it
> >
> >> with the fix included.
> >>>
> >>> On Tue, May 23, 2017 at 10:35 AM, Robert Metzger <
> >>> rmetz...@apache.org>
> >>> wrote:
> >>>
> >>> I've started building the RC.
> 
>  On Mon, May 22, 2017 at 6:01 PM, Robert Metzger <
>  rmetz...@apache.org>
>  wrote:
> 
>  Gordon's PR has been merged. I forgot one blocking issue. Till
> >
>  created a
> >
> >> PR for it: https://issues.apache.org/jira/browse/FLINK-6328
> > Once travis has passed, I'll merge that one and then do the RC.
> >
> > On Mon, May 22, 2017 at 10:36 AM, Robert Metzger <
> > rmetz...@apache.org
> > wrote:
> >
> > Thanks a lot for doing the legal checks for the release.
> >>
> >> I'll create the first voting release candidate once
> >> https://github.com/apache/flink/pull/3937 is merged.
> >>
> >> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang <
> >> xiaow...@gmail.com>
> >> wrote:
> >>
> >> Hi Robert,
> >>>
> >>> I did the following checks and found no issues:
> >>>
> >>>   - Check if checksums and GPG files match the corresponding
> >>> release
> >>> files
> >>>   - Verify that the source archiv

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Till Rohrmann
I think that https://issues.apache.org/jira/browse/FLINK-6646 might qualify
as a blocker issue.

The problem is that the yarn application files are managed by the
ClusterClient. Due to that the ClusterClient deletes them if it shuts down.
The ClusterClient shut down is also executed in case of a connection loss
to the JobManager (e.g. in case of a JM failure). Due to the file deletion,
Yarn is then no longer able to restart the JobManager in HA mode which
effectively thwarts Flink's HA capabilities.

The problem is that the files should not be deleted by the ClusterClient
once the Yarn application has been started. Instead the ApplicationMaster
should be responsible for the deletion.

Cheers,
Till

On Wed, May 24, 2017 at 4:53 PM, Timo Walther  wrote:

> I tested the quickstarts, the SBT build, the PlanVisualizer, and the
> HistoryServer. I could not find any serious problems. However, we have to
> update the quickstart scripts, once 1.3 is released.
>
> Timo
>
>
> Am 24.05.17 um 16:05 schrieb Chesnay Schepler:
>
> I've found a small problem in the yarn user-jar handling. We recently
>> added a switch to disable the user-jar inclusion in the class path, which
>> isn't working.
>>
>> PR is open, but i wouldn't consider this a release-blocker.
>> https://github.com/apache/flink/pull/3979
>>
>> On 24.05.2017 15:45, Kostas Kloudas wrote:
>>
>>> Thanks Robert for pushing this.
>>> This is going to be a voting RC?
>>>
>>> Kostas
>>>
>>> On May 24, 2017, at 3:16 PM, Robert Metzger  wrote:

 Great!
 The fixes are in, and I'm now building the RC2 (There are currently no
 blockers in our JIRA)

 On Tue, May 23, 2017 at 9:37 PM, Stefan Richter <
 s.rich...@data-artisans.com

> wrote:
> I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will
> merge
> tomorrow.
>
> Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :
>>
>> It appears that rescaling is broken, I've filed
>>
> https://issues.apache.org/jira/browse/FLINK-6690 for that.
>
>> needless to say, this is a release blocker.
>>
>> On 23.05.2017 20:55, Robert Metzger wrote:
>>
>>> I know I'm talking to myself here :) Anyways, I was running into some
>>> issues while creating the release (I was using master instead of the
>>> release-1.3 branch, which lead to some issues with the scala 2.10 /
>>> 2.11
>>> switch).
>>>
>>> The RC2 is basically ready, however, there's at least one new
>>> blocker:
>>> https://issues.apache.org/jira/browse/FLINK-6685 which needs
>>> addressing
>>> first.
>>>
>>> Let me know if you want me to publish the new RC2. Otherwise, I'll
>>>
>> re-do it
>
>> with the fix included.
>>>
>>> On Tue, May 23, 2017 at 10:35 AM, Robert Metzger <
>>> rmetz...@apache.org>
>>> wrote:
>>>
>>> I've started building the RC.

 On Mon, May 22, 2017 at 6:01 PM, Robert Metzger <
 rmetz...@apache.org>
 wrote:

 Gordon's PR has been merged. I forgot one blocking issue. Till
>
 created a
>
>> PR for it: https://issues.apache.org/jira/browse/FLINK-6328
> Once travis has passed, I'll merge that one and then do the RC.
>
> On Mon, May 22, 2017 at 10:36 AM, Robert Metzger <
> rmetz...@apache.org
> wrote:
>
> Thanks a lot for doing the legal checks for the release.
>>
>> I'll create the first voting release candidate once
>> https://github.com/apache/flink/pull/3937 is merged.
>>
>> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang <
>> xiaow...@gmail.com>
>> wrote:
>>
>> Hi Robert,
>>>
>>> I did the following checks and found no issues:
>>>
>>>   - Check if checksums and GPG files match the corresponding
>>> release
>>> files
>>>   - Verify that the source archives do not contain any binaries
>>>   - Check if the source release is building properly with Maven
>>> (including
>>> license header check and checkstyle). Also the tests should be
>>>
>> executed
>
>> (mvn clean verify).
>>>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3,
>>>
>> 2.7.2)
>
>>   - Check build for Scala 2.11
>>>   - Check that the README.md file is meaningful
>>>
>>> thanks
>>> Xiaowei
>>>
>>> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler <
>>>
>> ches...@apache.org>
>
>> wrote:
>>>
>>> Whoops, this is the PR for enabling the test:
 https://github.com/apache/flink/pull/3844


 On 19.05.2017 12:14, Robert Metzger wrote:

 Thank you for all your input.
>
> @Chesnay, in your email 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Timo Walther
I tested the quickstarts, the SBT build, the PlanVisualizer, and the 
HistoryServer. I could not find any serious problems. However, we have 
to update the quickstart scripts, once 1.3 is released.


Timo


Am 24.05.17 um 16:05 schrieb Chesnay Schepler:
I've found a small problem in the yarn user-jar handling. We recently 
added a switch to disable the user-jar inclusion in the class path, 
which isn't working.


PR is open, but i wouldn't consider this a release-blocker. 
https://github.com/apache/flink/pull/3979


On 24.05.2017 15:45, Kostas Kloudas wrote:

Thanks Robert for pushing this.
This is going to be a voting RC?

Kostas

On May 24, 2017, at 3:16 PM, Robert Metzger  
wrote:


Great!
The fixes are in, and I'm now building the RC2 (There are currently no
blockers in our JIRA)

On Tue, May 23, 2017 at 9:37 PM, Stefan Richter 

wrote:
I have fixes ready for both, FLINK-6690 and FLINK-6685, which I 
will merge

tomorrow.


Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :

It appears that rescaling is broken, I've filed

https://issues.apache.org/jira/browse/FLINK-6690 for that.

needless to say, this is a release blocker.

On 23.05.2017 20:55, Robert Metzger wrote:
I know I'm talking to myself here :) Anyways, I was running into 
some

issues while creating the release (I was using master instead of the
release-1.3 branch, which lead to some issues with the scala 2.10 
/ 2.11

switch).

The RC2 is basically ready, however, there's at least one new 
blocker:
https://issues.apache.org/jira/browse/FLINK-6685 which needs 
addressing

first.

Let me know if you want me to publish the new RC2. Otherwise, I'll

re-do it

with the fix included.

On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 


wrote:


I've started building the RC.

On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 


wrote:


Gordon's PR has been merged. I forgot one blocking issue. Till

created a

PR for it: https://issues.apache.org/jira/browse/FLINK-6328
Once travis has passed, I'll merge that one and then do the RC.

On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 

wrote:


Thanks a lot for doing the legal checks for the release.

I'll create the first voting release candidate once
https://github.com/apache/flink/pull/3937 is merged.

On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 


wrote:


Hi Robert,

I did the following checks and found no issues:

  - Check if checksums and GPG files match the corresponding 
release

files
  - Verify that the source archives do not contain any binaries
  - Check if the source release is building properly with Maven
(including
license header check and checkstyle). Also the tests should be

executed

(mvn clean verify).
  - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3,

2.7.2)

  - Check build for Scala 2.11
  - Check that the README.md file is meaningful

thanks
Xiaowei

On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler <

ches...@apache.org>

wrote:


Whoops, this is the PR for enabling the test:
https://github.com/apache/flink/pull/3844


On 19.05.2017 12:14, Robert Metzger wrote:


Thank you for all your input.

@Chesnay, in your email you are pointing to the same PR twice:
This PR fixes the compilation on Windows: (reviewed once, most

recent

changes not reviewed)
https://github.com/apache/flink/pull/3854
This PR enables a test for savepoint compatibility: (nice 
to have,

easy to

review)
https://github.com/apache/flink/pull/3854

Also the "should define more than one task slot" thing is not

important

IMO.

I think the "empty path on windows" thing is not a release

blocker.

--

These are the issues mentioned in the thread that are still 
open

and

blockers:
- Add nested serializers to config snapshots of composite

serializers:

https://github.com/apache/flink/pull/3937 has no review yet
- FLINK-6610 


WebServer
could not be created,when set the 
"jobmanager.web.submit.enable"

to

false
- FLINK-6629 


ClusterClient
cannot submit jobs to HA cluster if address not set in

configuration



On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <

trohrm...@apache.org>

wrote:

I might have found another blocker:

https://issues.apache.org/jira/browse/FLINK-6629.

The issue is that the ClusterClient only allows to submit 
jobs to

an HA

cluster if you have specified the JobManager's address in the
flink-conf.yaml or via the command line options. If no 
address is

set,

then
it fails completely. If the wrong address is set, which can

easily

happen

in an HA setting, then we are not able to find the proper

connecting

address for the ActorSystem. This basically voids Flink's HA
capabilities.

Cheers,
Till

On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <

ches...@apache.org>

wrote:

The test document says that the default flink-conf.yml 
"should

define

more

than one task slot", but it currently configures exact

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Chesnay Schepler
I've found a small problem in the yarn user-jar handling. We recently 
added a switch to disable the user-jar inclusion in the class path, 
which isn't working.


PR is open, but i wouldn't consider this a release-blocker. 
https://github.com/apache/flink/pull/3979


On 24.05.2017 15:45, Kostas Kloudas wrote:

Thanks Robert for pushing this.
This is going to be a voting RC?

Kostas


On May 24, 2017, at 3:16 PM, Robert Metzger  wrote:

Great!
The fixes are in, and I'm now building the RC2 (There are currently no
blockers in our JIRA)

On Tue, May 23, 2017 at 9:37 PM, Stefan Richter 
wrote:
I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will merge
tomorrow.


Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :

It appears that rescaling is broken, I've filed

https://issues.apache.org/jira/browse/FLINK-6690 for that.

needless to say, this is a release blocker.

On 23.05.2017 20:55, Robert Metzger wrote:

I know I'm talking to myself here :) Anyways, I was running into some
issues while creating the release (I was using master instead of the
release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
switch).

The RC2 is basically ready, however, there's at least one new blocker:
https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
first.

Let me know if you want me to publish the new RC2. Otherwise, I'll

re-do it

with the fix included.

On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
wrote:


I've started building the RC.

On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
wrote:


Gordon's PR has been merged. I forgot one blocking issue. Till

created a

PR for it: https://issues.apache.org/jira/browse/FLINK-6328
Once travis has passed, I'll merge that one and then do the RC.

On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
Thanks a lot for doing the legal checks for the release.

I'll create the first voting release candidate once
https://github.com/apache/flink/pull/3937 is merged.

On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
wrote:


Hi Robert,

I did the following checks and found no issues:

  - Check if checksums and GPG files match the corresponding release
files
  - Verify that the source archives do not contain any binaries
  - Check if the source release is building properly with Maven
(including
license header check and checkstyle). Also the tests should be

executed

(mvn clean verify).
  - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3,

2.7.2)

  - Check build for Scala 2.11
  - Check that the README.md file is meaningful

thanks
Xiaowei

On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler <

ches...@apache.org>

wrote:


Whoops, this is the PR for enabling the test:
https://github.com/apache/flink/pull/3844


On 19.05.2017 12:14, Robert Metzger wrote:


Thank you for all your input.

@Chesnay, in your email you are pointing to the same PR twice:
This PR fixes the compilation on Windows:  (reviewed once, most

recent

changes not reviewed)
https://github.com/apache/flink/pull/3854
This PR enables a test for savepoint compatibility: (nice to have,

easy to

review)
https://github.com/apache/flink/pull/3854

Also the "should define more than one task slot" thing is not

important

IMO.

I think the "empty path on windows" thing is not a release

blocker.

--

These are the issues mentioned in the thread that are still open

and

blockers:
- Add nested serializers to config snapshots of composite

serializers:

https://github.com/apache/flink/pull/3937 has no review yet
- FLINK-6610 

WebServer

could not be created,when set the "jobmanager.web.submit.enable"

to

false

- FLINK-6629 
ClusterClient
cannot submit jobs to HA cluster if address not set in

configuration



On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <

trohrm...@apache.org>

wrote:

I might have found another blocker:

https://issues.apache.org/jira/browse/FLINK-6629.

The issue is that the ClusterClient only allows to submit jobs to

an HA

cluster if you have specified the JobManager's address in the
flink-conf.yaml or via the command line options. If no address is

set,

then
it fails completely. If the wrong address is set, which can

easily

happen

in an HA setting, then we are not able to find the proper

connecting

address for the ActorSystem. This basically voids Flink's HA
capabilities.

Cheers,
Till

On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <

ches...@apache.org>

wrote:

The test document says that the default flink-conf.yml "should

define

more


than one task slot", but it currently configures exactly 1 task

slot.

Not
sure if it is a typo in the doc though.


On 18.05.2017 22:10, Chesnay Schepler wrote:

The start-cluster.sh script failed for me on Windows when

executed

in a

directory containing spaces.

On 18.05.2017 20:47, Chesnay Schepler wrote:

FLINK-6610 should also be 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Kostas Kloudas
Thanks Robert for pushing this.
This is going to be a voting RC?

Kostas

> On May 24, 2017, at 3:16 PM, Robert Metzger  wrote:
> 
> Great!
> The fixes are in, and I'm now building the RC2 (There are currently no
> blockers in our JIRA)
> 
> On Tue, May 23, 2017 at 9:37 PM, Stefan Richter > wrote:
> 
>> I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will merge
>> tomorrow.
>> 
>>> Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :
>>> 
>>> It appears that rescaling is broken, I've filed
>> https://issues.apache.org/jira/browse/FLINK-6690 for that.
>>> 
>>> needless to say, this is a release blocker.
>>> 
>>> On 23.05.2017 20:55, Robert Metzger wrote:
 I know I'm talking to myself here :) Anyways, I was running into some
 issues while creating the release (I was using master instead of the
 release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
 switch).
 
 The RC2 is basically ready, however, there's at least one new blocker:
 https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
 first.
 
 Let me know if you want me to publish the new RC2. Otherwise, I'll
>> re-do it
 with the fix included.
 
 On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
 wrote:
 
> I've started building the RC.
> 
> On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
> wrote:
> 
>> Gordon's PR has been merged. I forgot one blocking issue. Till
>> created a
>> PR for it: https://issues.apache.org/jira/browse/FLINK-6328
>> Once travis has passed, I'll merge that one and then do the RC.
>> 
>> On Mon, May 22, 2017 at 10:36 AM, Robert Metzger >> 
>> wrote:
>> 
>>> Thanks a lot for doing the legal checks for the release.
>>> 
>>> I'll create the first voting release candidate once
>>> https://github.com/apache/flink/pull/3937 is merged.
>>> 
>>> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
>>> wrote:
>>> 
 Hi Robert,
 
 I did the following checks and found no issues:
 
  - Check if checksums and GPG files match the corresponding release
 files
  - Verify that the source archives do not contain any binaries
  - Check if the source release is building properly with Maven
 (including
 license header check and checkstyle). Also the tests should be
>> executed
 (mvn clean verify).
  - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3,
>> 2.7.2)
  - Check build for Scala 2.11
  - Check that the README.md file is meaningful
 
 thanks
 Xiaowei
 
 On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler <
>> ches...@apache.org>
 wrote:
 
> Whoops, this is the PR for enabling the test:
> https://github.com/apache/flink/pull/3844
> 
> 
> On 19.05.2017 12:14, Robert Metzger wrote:
> 
>> Thank you for all your input.
>> 
>> @Chesnay, in your email you are pointing to the same PR twice:
>> This PR fixes the compilation on Windows:  (reviewed once, most
 recent
>> changes not reviewed)
>> https://github.com/apache/flink/pull/3854
>> This PR enables a test for savepoint compatibility: (nice to have,
 easy to
>> review)
>> https://github.com/apache/flink/pull/3854
>> 
>> Also the "should define more than one task slot" thing is not
 important
>> IMO.
>> 
>> I think the "empty path on windows" thing is not a release
>> blocker.
>> 
>> --
>> 
>> These are the issues mentioned in the thread that are still open
>> and
>> blockers:
>> - Add nested serializers to config snapshots of composite
 serializers:
>> https://github.com/apache/flink/pull/3937 has no review yet
>> - FLINK-6610 
 WebServer
>> could not be created,when set the "jobmanager.web.submit.enable"
>> to
 false
>> - FLINK-6629 
>> ClusterClient
>> cannot submit jobs to HA cluster if address not set in
>> configuration
>> 
>> 
>> 
>> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <
 trohrm...@apache.org>
>> wrote:
>> 
>> I might have found another blocker:
>>> https://issues.apache.org/jira/browse/FLINK-6629.
>>> 
>>> The issue is that the ClusterClient only allows to submit jobs to
 an HA
>>> cluster if you have specified the JobManager's address in the
>>> flink-conf.yaml or via the command line options. If no address is
 set,
>>> then
>>> it fails co

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-24 Thread Robert Metzger
Great!
The fixes are in, and I'm now building the RC2 (There are currently no
blockers in our JIRA)

On Tue, May 23, 2017 at 9:37 PM, Stefan Richter  wrote:

> I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will merge
> tomorrow.
>
> > Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :
> >
> > It appears that rescaling is broken, I've filed
> https://issues.apache.org/jira/browse/FLINK-6690 for that.
> >
> > needless to say, this is a release blocker.
> >
> > On 23.05.2017 20:55, Robert Metzger wrote:
> >> I know I'm talking to myself here :) Anyways, I was running into some
> >> issues while creating the release (I was using master instead of the
> >> release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
> >> switch).
> >>
> >> The RC2 is basically ready, however, there's at least one new blocker:
> >> https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
> >> first.
> >>
> >> Let me know if you want me to publish the new RC2. Otherwise, I'll
> re-do it
> >> with the fix included.
> >>
> >> On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
> >> wrote:
> >>
> >>> I've started building the RC.
> >>>
> >>> On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
> >>> wrote:
> >>>
>  Gordon's PR has been merged. I forgot one blocking issue. Till
> created a
>  PR for it: https://issues.apache.org/jira/browse/FLINK-6328
>  Once travis has passed, I'll merge that one and then do the RC.
> 
>  On Mon, May 22, 2017 at 10:36 AM, Robert Metzger  >
>  wrote:
> 
> > Thanks a lot for doing the legal checks for the release.
> >
> > I'll create the first voting release candidate once
> > https://github.com/apache/flink/pull/3937 is merged.
> >
> > On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
> > wrote:
> >
> >> Hi Robert,
> >>
> >> I did the following checks and found no issues:
> >>
> >>   - Check if checksums and GPG files match the corresponding release
> >> files
> >>   - Verify that the source archives do not contain any binaries
> >>   - Check if the source release is building properly with Maven
> >> (including
> >> license header check and checkstyle). Also the tests should be
> executed
> >> (mvn clean verify).
> >>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3,
> 2.7.2)
> >>   - Check build for Scala 2.11
> >>   - Check that the README.md file is meaningful
> >>
> >> thanks
> >> Xiaowei
> >>
> >> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler <
> ches...@apache.org>
> >> wrote:
> >>
> >>> Whoops, this is the PR for enabling the test:
> >>> https://github.com/apache/flink/pull/3844
> >>>
> >>>
> >>> On 19.05.2017 12:14, Robert Metzger wrote:
> >>>
>  Thank you for all your input.
> 
>  @Chesnay, in your email you are pointing to the same PR twice:
>  This PR fixes the compilation on Windows:  (reviewed once, most
> >> recent
>  changes not reviewed)
>  https://github.com/apache/flink/pull/3854
>  This PR enables a test for savepoint compatibility: (nice to have,
> >> easy to
>  review)
>  https://github.com/apache/flink/pull/3854
> 
>  Also the "should define more than one task slot" thing is not
> >> important
>  IMO.
> 
>  I think the "empty path on windows" thing is not a release
> blocker.
> 
>  --
> 
>  These are the issues mentioned in the thread that are still open
> and
>  blockers:
>  - Add nested serializers to config snapshots of composite
> >> serializers:
>  https://github.com/apache/flink/pull/3937 has no review yet
>  - FLINK-6610 
> >> WebServer
>  could not be created,when set the "jobmanager.web.submit.enable"
> to
> >> false
>  - FLINK-6629 
>  ClusterClient
>  cannot submit jobs to HA cluster if address not set in
> configuration
> 
> 
> 
>  On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <
> >> trohrm...@apache.org>
>  wrote:
> 
>  I might have found another blocker:
> > https://issues.apache.org/jira/browse/FLINK-6629.
> >
> > The issue is that the ClusterClient only allows to submit jobs to
> >> an HA
> > cluster if you have specified the JobManager's address in the
> > flink-conf.yaml or via the command line options. If no address is
> >> set,
> > then
> > it fails completely. If the wrong address is set, which can
> easily
> >> happen
> > in an HA setting, then we are not able to find the proper
> connecting
> > address for the 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-23 Thread Stefan Richter
I have fixes ready for both, FLINK-6690 and FLINK-6685, which I will merge 
tomorrow.

> Am 23.05.2017 um 21:05 schrieb Chesnay Schepler :
> 
> It appears that rescaling is broken, I've filed 
> https://issues.apache.org/jira/browse/FLINK-6690 for that.
> 
> needless to say, this is a release blocker.
> 
> On 23.05.2017 20:55, Robert Metzger wrote:
>> I know I'm talking to myself here :) Anyways, I was running into some
>> issues while creating the release (I was using master instead of the
>> release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
>> switch).
>> 
>> The RC2 is basically ready, however, there's at least one new blocker:
>> https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
>> first.
>> 
>> Let me know if you want me to publish the new RC2. Otherwise, I'll re-do it
>> with the fix included.
>> 
>> On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
>> wrote:
>> 
>>> I've started building the RC.
>>> 
>>> On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
>>> wrote:
>>> 
 Gordon's PR has been merged. I forgot one blocking issue. Till created a
 PR for it: https://issues.apache.org/jira/browse/FLINK-6328
 Once travis has passed, I'll merge that one and then do the RC.
 
 On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
 wrote:
 
> Thanks a lot for doing the legal checks for the release.
> 
> I'll create the first voting release candidate once
> https://github.com/apache/flink/pull/3937 is merged.
> 
> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
> wrote:
> 
>> Hi Robert,
>> 
>> I did the following checks and found no issues:
>> 
>>   - Check if checksums and GPG files match the corresponding release
>> files
>>   - Verify that the source archives do not contain any binaries
>>   - Check if the source release is building properly with Maven
>> (including
>> license header check and checkstyle). Also the tests should be executed
>> (mvn clean verify).
>>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
>>   - Check build for Scala 2.11
>>   - Check that the README.md file is meaningful
>> 
>> thanks
>> Xiaowei
>> 
>> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
>> wrote:
>> 
>>> Whoops, this is the PR for enabling the test:
>>> https://github.com/apache/flink/pull/3844
>>> 
>>> 
>>> On 19.05.2017 12:14, Robert Metzger wrote:
>>> 
 Thank you for all your input.
 
 @Chesnay, in your email you are pointing to the same PR twice:
 This PR fixes the compilation on Windows:  (reviewed once, most
>> recent
 changes not reviewed)
 https://github.com/apache/flink/pull/3854
 This PR enables a test for savepoint compatibility: (nice to have,
>> easy to
 review)
 https://github.com/apache/flink/pull/3854
 
 Also the "should define more than one task slot" thing is not
>> important
 IMO.
 
 I think the "empty path on windows" thing is not a release blocker.
 
 --
 
 These are the issues mentioned in the thread that are still open and
 blockers:
 - Add nested serializers to config snapshots of composite
>> serializers:
 https://github.com/apache/flink/pull/3937 has no review yet
 - FLINK-6610 
>> WebServer
 could not be created,when set the "jobmanager.web.submit.enable" to
>> false
 - FLINK-6629 
 ClusterClient
 cannot submit jobs to HA cluster if address not set in configuration
 
 
 
 On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <
>> trohrm...@apache.org>
 wrote:
 
 I might have found another blocker:
> https://issues.apache.org/jira/browse/FLINK-6629.
> 
> The issue is that the ClusterClient only allows to submit jobs to
>> an HA
> cluster if you have specified the JobManager's address in the
> flink-conf.yaml or via the command line options. If no address is
>> set,
> then
> it fails completely. If the wrong address is set, which can easily
>> happen
> in an HA setting, then we are not able to find the proper connecting
> address for the ActorSystem. This basically voids Flink's HA
> capabilities.
> 
> Cheers,
> Till
> 
> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <
>> ches...@apache.org>
> wrote:
> 
> The test document says that the default flink-conf.yml "should
>> define
> more
> 
>> than one task slot", but it currently configures e

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-23 Thread Chesnay Schepler
It appears that rescaling is broken, I've filed 
https://issues.apache.org/jira/browse/FLINK-6690 for that.


needless to say, this is a release blocker.

On 23.05.2017 20:55, Robert Metzger wrote:

I know I'm talking to myself here :) Anyways, I was running into some
issues while creating the release (I was using master instead of the
release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
switch).

The RC2 is basically ready, however, there's at least one new blocker:
https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
first.

Let me know if you want me to publish the new RC2. Otherwise, I'll re-do it
with the fix included.

On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
wrote:


I've started building the RC.

On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
wrote:


Gordon's PR has been merged. I forgot one blocking issue. Till created a
PR for it: https://issues.apache.org/jira/browse/FLINK-6328
Once travis has passed, I'll merge that one and then do the RC.

On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
wrote:


Thanks a lot for doing the legal checks for the release.

I'll create the first voting release candidate once
https://github.com/apache/flink/pull/3937 is merged.

On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
wrote:


Hi Robert,

I did the following checks and found no issues:

   - Check if checksums and GPG files match the corresponding release
files
   - Verify that the source archives do not contain any binaries
   - Check if the source release is building properly with Maven
(including
license header check and checkstyle). Also the tests should be executed
(mvn clean verify).
   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
   - Check build for Scala 2.11
   - Check that the README.md file is meaningful

thanks
Xiaowei

On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
wrote:


Whoops, this is the PR for enabling the test:
https://github.com/apache/flink/pull/3844


On 19.05.2017 12:14, Robert Metzger wrote:


Thank you for all your input.

@Chesnay, in your email you are pointing to the same PR twice:
This PR fixes the compilation on Windows:  (reviewed once, most

recent

changes not reviewed)
https://github.com/apache/flink/pull/3854
This PR enables a test for savepoint compatibility: (nice to have,

easy to

review)
https://github.com/apache/flink/pull/3854

Also the "should define more than one task slot" thing is not

important

IMO.

I think the "empty path on windows" thing is not a release blocker.

--

These are the issues mentioned in the thread that are still open and
blockers:
- Add nested serializers to config snapshots of composite

serializers:

https://github.com/apache/flink/pull/3937 has no review yet
- FLINK-6610 

WebServer

could not be created,when set the "jobmanager.web.submit.enable" to

false

- FLINK-6629 
ClusterClient
cannot submit jobs to HA cluster if address not set in configuration



On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <

trohrm...@apache.org>

wrote:

I might have found another blocker:

https://issues.apache.org/jira/browse/FLINK-6629.

The issue is that the ClusterClient only allows to submit jobs to

an HA

cluster if you have specified the JobManager's address in the
flink-conf.yaml or via the command line options. If no address is

set,

then
it fails completely. If the wrong address is set, which can easily

happen

in an HA setting, then we are not able to find the proper connecting
address for the ActorSystem. This basically voids Flink's HA
capabilities.

Cheers,
Till

On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <

ches...@apache.org>

wrote:

The test document says that the default flink-conf.yml "should

define

more


than one task slot", but it currently configures exactly 1 task

slot.

Not
sure if it is a typo in the doc though.


On 18.05.2017 22:10, Chesnay Schepler wrote:

The start-cluster.sh script failed for me on Windows when executed

in a

directory containing spaces.

On 18.05.2017 20:47, Chesnay Schepler wrote:

FLINK-6610 should also be fixed; it is currently not possible to
disable

web-submissions.

On 18.05.2017 18:13, jincheng sun wrote:

Hi Robert,

I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done

soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :

The following tickets for 1.3.0 have a PR in need of review:


[FLINK-6582] [docs] Project from maven archetype is not

buildable by

default
[FLINK-6616] [docs] Clarify provenance of official Docker

images


On May 18, 2017, at 5:40 AM, Fabian Hueske 


wrote:

I have a couple of PRs ready with bugfixes that I'll try to get in

as

well.

Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann <

trohrm...@apache.org>:

I'd like to get a fix 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-23 Thread Robert Metzger
I know I'm talking to myself here :) Anyways, I was running into some
issues while creating the release (I was using master instead of the
release-1.3 branch, which lead to some issues with the scala 2.10 / 2.11
switch).

The RC2 is basically ready, however, there's at least one new blocker:
https://issues.apache.org/jira/browse/FLINK-6685 which needs addressing
first.

Let me know if you want me to publish the new RC2. Otherwise, I'll re-do it
with the fix included.

On Tue, May 23, 2017 at 10:35 AM, Robert Metzger 
wrote:

> I've started building the RC.
>
> On Mon, May 22, 2017 at 6:01 PM, Robert Metzger 
> wrote:
>
>> Gordon's PR has been merged. I forgot one blocking issue. Till created a
>> PR for it: https://issues.apache.org/jira/browse/FLINK-6328
>> Once travis has passed, I'll merge that one and then do the RC.
>>
>> On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
>> wrote:
>>
>>> Thanks a lot for doing the legal checks for the release.
>>>
>>> I'll create the first voting release candidate once
>>> https://github.com/apache/flink/pull/3937 is merged.
>>>
>>> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
>>> wrote:
>>>
 Hi Robert,

 I did the following checks and found no issues:

   - Check if checksums and GPG files match the corresponding release
 files
   - Verify that the source archives do not contain any binaries
   - Check if the source release is building properly with Maven
 (including
 license header check and checkstyle). Also the tests should be executed
 (mvn clean verify).
   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
   - Check build for Scala 2.11
   - Check that the README.md file is meaningful

 thanks
 Xiaowei

 On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
 wrote:

 > Whoops, this is the PR for enabling the test:
 > https://github.com/apache/flink/pull/3844
 >
 >
 > On 19.05.2017 12:14, Robert Metzger wrote:
 >
 >> Thank you for all your input.
 >>
 >> @Chesnay, in your email you are pointing to the same PR twice:
 >> This PR fixes the compilation on Windows:  (reviewed once, most
 recent
 >> changes not reviewed)
 >> https://github.com/apache/flink/pull/3854
 >> This PR enables a test for savepoint compatibility: (nice to have,
 easy to
 >> review)
 >> https://github.com/apache/flink/pull/3854
 >>
 >> Also the "should define more than one task slot" thing is not
 important
 >> IMO.
 >>
 >> I think the "empty path on windows" thing is not a release blocker.
 >>
 >> --
 >>
 >> These are the issues mentioned in the thread that are still open and
 >> blockers:
 >> - Add nested serializers to config snapshots of composite
 serializers:
 >> https://github.com/apache/flink/pull/3937 has no review yet
 >> - FLINK-6610 
 WebServer
 >> could not be created,when set the "jobmanager.web.submit.enable" to
 false
 >> - FLINK-6629 
 >> ClusterClient
 >> cannot submit jobs to HA cluster if address not set in configuration
 >>
 >>
 >>
 >> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann <
 trohrm...@apache.org>
 >> wrote:
 >>
 >> I might have found another blocker:
 >>> https://issues.apache.org/jira/browse/FLINK-6629.
 >>>
 >>> The issue is that the ClusterClient only allows to submit jobs to
 an HA
 >>> cluster if you have specified the JobManager's address in the
 >>> flink-conf.yaml or via the command line options. If no address is
 set,
 >>> then
 >>> it fails completely. If the wrong address is set, which can easily
 happen
 >>> in an HA setting, then we are not able to find the proper connecting
 >>> address for the ActorSystem. This basically voids Flink's HA
 >>> capabilities.
 >>>
 >>> Cheers,
 >>> Till
 >>>
 >>> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <
 ches...@apache.org>
 >>> wrote:
 >>>
 >>> The test document says that the default flink-conf.yml "should
 define
 
 >>> more
 >>>
  than one task slot", but it currently configures exactly 1 task
 slot.
  Not
  sure if it is a typo in the doc though.
 
 
  On 18.05.2017 22:10, Chesnay Schepler wrote:
 
  The start-cluster.sh script failed for me on Windows when executed
 in a
 > directory containing spaces.
 >
 > On 18.05.2017 20:47, Chesnay Schepler wrote:
 >
 > FLINK-6610 should also be fixed; it is currently not possible to
 >>
 > disable
 >>>
  web-submissions.
 >>
 >> On 18.05.2017 18:13, jincheng sun w

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-23 Thread Robert Metzger
I've started building the RC.

On Mon, May 22, 2017 at 6:01 PM, Robert Metzger  wrote:

> Gordon's PR has been merged. I forgot one blocking issue. Till created a
> PR for it: https://issues.apache.org/jira/browse/FLINK-6328
> Once travis has passed, I'll merge that one and then do the RC.
>
> On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
> wrote:
>
>> Thanks a lot for doing the legal checks for the release.
>>
>> I'll create the first voting release candidate once
>> https://github.com/apache/flink/pull/3937 is merged.
>>
>> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang 
>> wrote:
>>
>>> Hi Robert,
>>>
>>> I did the following checks and found no issues:
>>>
>>>   - Check if checksums and GPG files match the corresponding release
>>> files
>>>   - Verify that the source archives do not contain any binaries
>>>   - Check if the source release is building properly with Maven
>>> (including
>>> license header check and checkstyle). Also the tests should be executed
>>> (mvn clean verify).
>>>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
>>>   - Check build for Scala 2.11
>>>   - Check that the README.md file is meaningful
>>>
>>> thanks
>>> Xiaowei
>>>
>>> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
>>> wrote:
>>>
>>> > Whoops, this is the PR for enabling the test:
>>> > https://github.com/apache/flink/pull/3844
>>> >
>>> >
>>> > On 19.05.2017 12:14, Robert Metzger wrote:
>>> >
>>> >> Thank you for all your input.
>>> >>
>>> >> @Chesnay, in your email you are pointing to the same PR twice:
>>> >> This PR fixes the compilation on Windows:  (reviewed once, most recent
>>> >> changes not reviewed)
>>> >> https://github.com/apache/flink/pull/3854
>>> >> This PR enables a test for savepoint compatibility: (nice to have,
>>> easy to
>>> >> review)
>>> >> https://github.com/apache/flink/pull/3854
>>> >>
>>> >> Also the "should define more than one task slot" thing is not
>>> important
>>> >> IMO.
>>> >>
>>> >> I think the "empty path on windows" thing is not a release blocker.
>>> >>
>>> >> --
>>> >>
>>> >> These are the issues mentioned in the thread that are still open and
>>> >> blockers:
>>> >> - Add nested serializers to config snapshots of composite serializers:
>>> >> https://github.com/apache/flink/pull/3937 has no review yet
>>> >> - FLINK-6610 
>>> WebServer
>>> >> could not be created,when set the "jobmanager.web.submit.enable" to
>>> false
>>> >> - FLINK-6629 
>>> >> ClusterClient
>>> >> cannot submit jobs to HA cluster if address not set in configuration
>>> >>
>>> >>
>>> >>
>>> >> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann >> >
>>> >> wrote:
>>> >>
>>> >> I might have found another blocker:
>>> >>> https://issues.apache.org/jira/browse/FLINK-6629.
>>> >>>
>>> >>> The issue is that the ClusterClient only allows to submit jobs to an
>>> HA
>>> >>> cluster if you have specified the JobManager's address in the
>>> >>> flink-conf.yaml or via the command line options. If no address is
>>> set,
>>> >>> then
>>> >>> it fails completely. If the wrong address is set, which can easily
>>> happen
>>> >>> in an HA setting, then we are not able to find the proper connecting
>>> >>> address for the ActorSystem. This basically voids Flink's HA
>>> >>> capabilities.
>>> >>>
>>> >>> Cheers,
>>> >>> Till
>>> >>>
>>> >>> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <
>>> ches...@apache.org>
>>> >>> wrote:
>>> >>>
>>> >>> The test document says that the default flink-conf.yml "should define
>>> 
>>> >>> more
>>> >>>
>>>  than one task slot", but it currently configures exactly 1 task
>>> slot.
>>>  Not
>>>  sure if it is a typo in the doc though.
>>> 
>>> 
>>>  On 18.05.2017 22:10, Chesnay Schepler wrote:
>>> 
>>>  The start-cluster.sh script failed for me on Windows when executed
>>> in a
>>> > directory containing spaces.
>>> >
>>> > On 18.05.2017 20:47, Chesnay Schepler wrote:
>>> >
>>> > FLINK-6610 should also be fixed; it is currently not possible to
>>> >>
>>> > disable
>>> >>>
>>>  web-submissions.
>>> >>
>>> >> On 18.05.2017 18:13, jincheng sun wrote:
>>> >>
>>> >> Hi Robert,
>>> >>> I have some checks to do and some test improve PRs (
>>> >>> https://issues.apache.org/jira/browse/FLINK-6619) need be done
>>> soon.
>>> >>>
>>> >>> Best,
>>> >>> SunJincheng
>>> >>>
>>> >>> 2017-05-18 22:17 GMT+08:00 Greg Hogan :
>>> >>>
>>> >>> The following tickets for 1.3.0 have a PR in need of review:
>>> >>>
>>>  [FLINK-6582] [docs] Project from maven archetype is not
>>> buildable by
>>>  default
>>>  [FLINK-6616] [docs] Clarify provenance of official Docker images
>>> 
>>> 
>>>  On May 18, 2017, at 5:40 AM, Fabian Hueske 
>>> 
>>> >>> wrote:

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-22 Thread Robert Metzger
Gordon's PR has been merged. I forgot one blocking issue. Till created a PR
for it: https://issues.apache.org/jira/browse/FLINK-6328
Once travis has passed, I'll merge that one and then do the RC.

On Mon, May 22, 2017 at 10:36 AM, Robert Metzger 
wrote:

> Thanks a lot for doing the legal checks for the release.
>
> I'll create the first voting release candidate once https://github.com/
> apache/flink/pull/3937 is merged.
>
> On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang  wrote:
>
>> Hi Robert,
>>
>> I did the following checks and found no issues:
>>
>>   - Check if checksums and GPG files match the corresponding release files
>>   - Verify that the source archives do not contain any binaries
>>   - Check if the source release is building properly with Maven (including
>> license header check and checkstyle). Also the tests should be executed
>> (mvn clean verify).
>>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
>>   - Check build for Scala 2.11
>>   - Check that the README.md file is meaningful
>>
>> thanks
>> Xiaowei
>>
>> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
>> wrote:
>>
>> > Whoops, this is the PR for enabling the test:
>> > https://github.com/apache/flink/pull/3844
>> >
>> >
>> > On 19.05.2017 12:14, Robert Metzger wrote:
>> >
>> >> Thank you for all your input.
>> >>
>> >> @Chesnay, in your email you are pointing to the same PR twice:
>> >> This PR fixes the compilation on Windows:  (reviewed once, most recent
>> >> changes not reviewed)
>> >> https://github.com/apache/flink/pull/3854
>> >> This PR enables a test for savepoint compatibility: (nice to have,
>> easy to
>> >> review)
>> >> https://github.com/apache/flink/pull/3854
>> >>
>> >> Also the "should define more than one task slot" thing is not important
>> >> IMO.
>> >>
>> >> I think the "empty path on windows" thing is not a release blocker.
>> >>
>> >> --
>> >>
>> >> These are the issues mentioned in the thread that are still open and
>> >> blockers:
>> >> - Add nested serializers to config snapshots of composite serializers:
>> >> https://github.com/apache/flink/pull/3937 has no review yet
>> >> - FLINK-6610 
>> WebServer
>> >> could not be created,when set the "jobmanager.web.submit.enable" to
>> false
>> >> - FLINK-6629 
>> >> ClusterClient
>> >> cannot submit jobs to HA cluster if address not set in configuration
>> >>
>> >>
>> >>
>> >> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann 
>> >> wrote:
>> >>
>> >> I might have found another blocker:
>> >>> https://issues.apache.org/jira/browse/FLINK-6629.
>> >>>
>> >>> The issue is that the ClusterClient only allows to submit jobs to an
>> HA
>> >>> cluster if you have specified the JobManager's address in the
>> >>> flink-conf.yaml or via the command line options. If no address is set,
>> >>> then
>> >>> it fails completely. If the wrong address is set, which can easily
>> happen
>> >>> in an HA setting, then we are not able to find the proper connecting
>> >>> address for the ActorSystem. This basically voids Flink's HA
>> >>> capabilities.
>> >>>
>> >>> Cheers,
>> >>> Till
>> >>>
>> >>> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler <
>> ches...@apache.org>
>> >>> wrote:
>> >>>
>> >>> The test document says that the default flink-conf.yml "should define
>> 
>> >>> more
>> >>>
>>  than one task slot", but it currently configures exactly 1 task slot.
>>  Not
>>  sure if it is a typo in the doc though.
>> 
>> 
>>  On 18.05.2017 22:10, Chesnay Schepler wrote:
>> 
>>  The start-cluster.sh script failed for me on Windows when executed
>> in a
>> > directory containing spaces.
>> >
>> > On 18.05.2017 20:47, Chesnay Schepler wrote:
>> >
>> > FLINK-6610 should also be fixed; it is currently not possible to
>> >>
>> > disable
>> >>>
>>  web-submissions.
>> >>
>> >> On 18.05.2017 18:13, jincheng sun wrote:
>> >>
>> >> Hi Robert,
>> >>> I have some checks to do and some test improve PRs (
>> >>> https://issues.apache.org/jira/browse/FLINK-6619) need be done
>> soon.
>> >>>
>> >>> Best,
>> >>> SunJincheng
>> >>>
>> >>> 2017-05-18 22:17 GMT+08:00 Greg Hogan :
>> >>>
>> >>> The following tickets for 1.3.0 have a PR in need of review:
>> >>>
>>  [FLINK-6582] [docs] Project from maven archetype is not
>> buildable by
>>  default
>>  [FLINK-6616] [docs] Clarify provenance of official Docker images
>> 
>> 
>>  On May 18, 2017, at 5:40 AM, Fabian Hueske 
>> 
>> >>> wrote:
>> >>>
>>  I have a couple of PRs ready with bugfixes that I'll try to get in
>> >
>>  as
>> >>>
>>  well.
>> 
>>  Should be done soon.
>> >
>> > 2017-05-18 11:24 GMT+02:00 Till Rohrmann > >:
>> 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-22 Thread Robert Metzger
Thanks a lot for doing the legal checks for the release.

I'll create the first voting release candidate once
https://github.com/apache/flink/pull/3937 is merged.

On Fri, May 19, 2017 at 4:45 PM, Xiaowei Jiang  wrote:

> Hi Robert,
>
> I did the following checks and found no issues:
>
>   - Check if checksums and GPG files match the corresponding release files
>   - Verify that the source archives do not contain any binaries
>   - Check if the source release is building properly with Maven (including
> license header check and checkstyle). Also the tests should be executed
> (mvn clean verify).
>   - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
>   - Check build for Scala 2.11
>   - Check that the README.md file is meaningful
>
> thanks
> Xiaowei
>
> On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
> wrote:
>
> > Whoops, this is the PR for enabling the test:
> > https://github.com/apache/flink/pull/3844
> >
> >
> > On 19.05.2017 12:14, Robert Metzger wrote:
> >
> >> Thank you for all your input.
> >>
> >> @Chesnay, in your email you are pointing to the same PR twice:
> >> This PR fixes the compilation on Windows:  (reviewed once, most recent
> >> changes not reviewed)
> >> https://github.com/apache/flink/pull/3854
> >> This PR enables a test for savepoint compatibility: (nice to have, easy
> to
> >> review)
> >> https://github.com/apache/flink/pull/3854
> >>
> >> Also the "should define more than one task slot" thing is not important
> >> IMO.
> >>
> >> I think the "empty path on windows" thing is not a release blocker.
> >>
> >> --
> >>
> >> These are the issues mentioned in the thread that are still open and
> >> blockers:
> >> - Add nested serializers to config snapshots of composite serializers:
> >> https://github.com/apache/flink/pull/3937 has no review yet
> >> - FLINK-6610 
> WebServer
> >> could not be created,when set the "jobmanager.web.submit.enable" to
> false
> >> - FLINK-6629 
> >> ClusterClient
> >> cannot submit jobs to HA cluster if address not set in configuration
> >>
> >>
> >>
> >> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann 
> >> wrote:
> >>
> >> I might have found another blocker:
> >>> https://issues.apache.org/jira/browse/FLINK-6629.
> >>>
> >>> The issue is that the ClusterClient only allows to submit jobs to an HA
> >>> cluster if you have specified the JobManager's address in the
> >>> flink-conf.yaml or via the command line options. If no address is set,
> >>> then
> >>> it fails completely. If the wrong address is set, which can easily
> happen
> >>> in an HA setting, then we are not able to find the proper connecting
> >>> address for the ActorSystem. This basically voids Flink's HA
> >>> capabilities.
> >>>
> >>> Cheers,
> >>> Till
> >>>
> >>> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler  >
> >>> wrote:
> >>>
> >>> The test document says that the default flink-conf.yml "should define
> 
> >>> more
> >>>
>  than one task slot", but it currently configures exactly 1 task slot.
>  Not
>  sure if it is a typo in the doc though.
> 
> 
>  On 18.05.2017 22:10, Chesnay Schepler wrote:
> 
>  The start-cluster.sh script failed for me on Windows when executed in
> a
> > directory containing spaces.
> >
> > On 18.05.2017 20:47, Chesnay Schepler wrote:
> >
> > FLINK-6610 should also be fixed; it is currently not possible to
> >>
> > disable
> >>>
>  web-submissions.
> >>
> >> On 18.05.2017 18:13, jincheng sun wrote:
> >>
> >> Hi Robert,
> >>> I have some checks to do and some test improve PRs (
> >>> https://issues.apache.org/jira/browse/FLINK-6619) need be done
> soon.
> >>>
> >>> Best,
> >>> SunJincheng
> >>>
> >>> 2017-05-18 22:17 GMT+08:00 Greg Hogan :
> >>>
> >>> The following tickets for 1.3.0 have a PR in need of review:
> >>>
>  [FLINK-6582] [docs] Project from maven archetype is not buildable
> by
>  default
>  [FLINK-6616] [docs] Clarify provenance of official Docker images
> 
> 
>  On May 18, 2017, at 5:40 AM, Fabian Hueske 
> 
> >>> wrote:
> >>>
>  I have a couple of PRs ready with bugfixes that I'll try to get in
> >
>  as
> >>>
>  well.
> 
>  Should be done soon.
> >
> > 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
> >
> > I'd like to get a fix in for
> >
> >> https://issues.apache.org/jira/browse/FLINK-6612. This can
> >>
> > basically
> >>>
>  thwart
> >> Flink's recovery capabilities.
> >>
> >> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler <
> >> ches...@apache.org>
> >> wrote:
> >>
> >> This PR reduces logging noise a bit: (got +1 to 

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-19 Thread Xiaowei Jiang
Hi Robert,

I did the following checks and found no issues:

  - Check if checksums and GPG files match the corresponding release files
  - Verify that the source archives do not contain any binaries
  - Check if the source release is building properly with Maven (including
license header check and checkstyle). Also the tests should be executed
(mvn clean verify).
  - Check build for custom Hadoop version (2.3.0, 2.4.1, 2.6.3, 2.7.2)
  - Check build for Scala 2.11
  - Check that the README.md file is meaningful

thanks
Xiaowei

On Fri, May 19, 2017 at 6:29 PM, Chesnay Schepler 
wrote:

> Whoops, this is the PR for enabling the test:
> https://github.com/apache/flink/pull/3844
>
>
> On 19.05.2017 12:14, Robert Metzger wrote:
>
>> Thank you for all your input.
>>
>> @Chesnay, in your email you are pointing to the same PR twice:
>> This PR fixes the compilation on Windows:  (reviewed once, most recent
>> changes not reviewed)
>> https://github.com/apache/flink/pull/3854
>> This PR enables a test for savepoint compatibility: (nice to have, easy to
>> review)
>> https://github.com/apache/flink/pull/3854
>>
>> Also the "should define more than one task slot" thing is not important
>> IMO.
>>
>> I think the "empty path on windows" thing is not a release blocker.
>>
>> --
>>
>> These are the issues mentioned in the thread that are still open and
>> blockers:
>> - Add nested serializers to config snapshots of composite serializers:
>> https://github.com/apache/flink/pull/3937 has no review yet
>> - FLINK-6610  WebServer
>> could not be created,when set the "jobmanager.web.submit.enable" to false
>> - FLINK-6629 
>> ClusterClient
>> cannot submit jobs to HA cluster if address not set in configuration
>>
>>
>>
>> On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann 
>> wrote:
>>
>> I might have found another blocker:
>>> https://issues.apache.org/jira/browse/FLINK-6629.
>>>
>>> The issue is that the ClusterClient only allows to submit jobs to an HA
>>> cluster if you have specified the JobManager's address in the
>>> flink-conf.yaml or via the command line options. If no address is set,
>>> then
>>> it fails completely. If the wrong address is set, which can easily happen
>>> in an HA setting, then we are not able to find the proper connecting
>>> address for the ActorSystem. This basically voids Flink's HA
>>> capabilities.
>>>
>>> Cheers,
>>> Till
>>>
>>> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler 
>>> wrote:
>>>
>>> The test document says that the default flink-conf.yml "should define

>>> more
>>>
 than one task slot", but it currently configures exactly 1 task slot.
 Not
 sure if it is a typo in the doc though.


 On 18.05.2017 22:10, Chesnay Schepler wrote:

 The start-cluster.sh script failed for me on Windows when executed in a
> directory containing spaces.
>
> On 18.05.2017 20:47, Chesnay Schepler wrote:
>
> FLINK-6610 should also be fixed; it is currently not possible to
>>
> disable
>>>
 web-submissions.
>>
>> On 18.05.2017 18:13, jincheng sun wrote:
>>
>> Hi Robert,
>>> I have some checks to do and some test improve PRs (
>>> https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.
>>>
>>> Best,
>>> SunJincheng
>>>
>>> 2017-05-18 22:17 GMT+08:00 Greg Hogan :
>>>
>>> The following tickets for 1.3.0 have a PR in need of review:
>>>
 [FLINK-6582] [docs] Project from maven archetype is not buildable by
 default
 [FLINK-6616] [docs] Clarify provenance of official Docker images


 On May 18, 2017, at 5:40 AM, Fabian Hueske 

>>> wrote:
>>>
 I have a couple of PRs ready with bugfixes that I'll try to get in
>
 as
>>>
 well.

 Should be done soon.
>
> 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
>
> I'd like to get a fix in for
>
>> https://issues.apache.org/jira/browse/FLINK-6612. This can
>>
> basically
>>>
 thwart
>> Flink's recovery capabilities.
>>
>> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler <
>> ches...@apache.org>
>> wrote:
>>
>> This PR reduces logging noise a bit: (got +1 to merge)
>>
>>> https://github.com/apache/flink/pull/3917
>>>
>>> This PR fixes the compilation on Windows:  (reviewed once, most
>>> recent
>>> changes not reviewed)
>>> https://github.com/apache/flink/pull/3854
>>>
>>> This PR enables a test for savepoint compatibility: (nice to
>>> have,
>>> easy
>>>
>>> to
>>
>> review)
>>> https://github.com/apache/flink/pull/3854
>>>
>>>

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-19 Thread Chesnay Schepler
Whoops, this is the PR for enabling the test: 
https://github.com/apache/flink/pull/3844


On 19.05.2017 12:14, Robert Metzger wrote:

Thank you for all your input.

@Chesnay, in your email you are pointing to the same PR twice:
This PR fixes the compilation on Windows:  (reviewed once, most recent
changes not reviewed)
https://github.com/apache/flink/pull/3854
This PR enables a test for savepoint compatibility: (nice to have, easy to
review)
https://github.com/apache/flink/pull/3854

Also the "should define more than one task slot" thing is not important IMO.

I think the "empty path on windows" thing is not a release blocker.

--

These are the issues mentioned in the thread that are still open and
blockers:
- Add nested serializers to config snapshots of composite serializers:
https://github.com/apache/flink/pull/3937 has no review yet
- FLINK-6610  WebServer
could not be created,when set the "jobmanager.web.submit.enable" to false
- FLINK-6629  ClusterClient
cannot submit jobs to HA cluster if address not set in configuration



On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann 
wrote:


I might have found another blocker:
https://issues.apache.org/jira/browse/FLINK-6629.

The issue is that the ClusterClient only allows to submit jobs to an HA
cluster if you have specified the JobManager's address in the
flink-conf.yaml or via the command line options. If no address is set, then
it fails completely. If the wrong address is set, which can easily happen
in an HA setting, then we are not able to find the proper connecting
address for the ActorSystem. This basically voids Flink's HA capabilities.

Cheers,
Till

On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler 
wrote:


The test document says that the default flink-conf.yml "should define

more

than one task slot", but it currently configures exactly 1 task slot. Not
sure if it is a typo in the doc though.


On 18.05.2017 22:10, Chesnay Schepler wrote:


The start-cluster.sh script failed for me on Windows when executed in a
directory containing spaces.

On 18.05.2017 20:47, Chesnay Schepler wrote:


FLINK-6610 should also be fixed; it is currently not possible to

disable

web-submissions.

On 18.05.2017 18:13, jincheng sun wrote:


Hi Robert,
I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :

The following tickets for 1.3.0 have a PR in need of review:

[FLINK-6582] [docs] Project from maven archetype is not buildable by
default
[FLINK-6616] [docs] Clarify provenance of official Docker images


On May 18, 2017, at 5:40 AM, Fabian Hueske 

wrote:

I have a couple of PRs ready with bugfixes that I'll try to get in

as

well.


Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann :

I'd like to get a fix in for

https://issues.apache.org/jira/browse/FLINK-6612. This can

basically

thwart
Flink's recovery capabilities.

On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler <
ches...@apache.org>
wrote:

This PR reduces logging noise a bit: (got +1 to merge)

https://github.com/apache/flink/pull/3917

This PR fixes the compilation on Windows:  (reviewed once, most
recent
changes not reviewed)
https://github.com/apache/flink/pull/3854

This PR enables a test for savepoint compatibility: (nice to have,
easy


to


review)
https://github.com/apache/flink/pull/3854

These 2 PRs fix minor issues with metrics: (trivial review, both
one-liners)
https://github.com/apache/flink/pull/3906
https://github.com/apache/flink/pull/3907


On 18.05.2017 10:52, Robert Metzger wrote:

I will.

Actually I had it already on my radar because its one of the

three

remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for

the

other

blockers, I think its fine to release without having them fixed.

Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright <

eronwri...@gmail.com

wrote:

Robert, please add FLINK-6606 to the list of JIRAs that you're


tracking,

thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger <
rmetz...@apache.org
wrote:

I totally forgot to post a document with testing tasks in the

RC0

thread,

so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-
Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the


document,

I'll

write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <


rmetz...@apache.org>

wrote:

Thanks for the pointer. I'll keep an eye on the JIRA.


I've gone through the JIRAs tagged with 1.3.0 yesterday to
create a

list

of new features in 1.3. Feel free to add more / change it in

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-19 Thread Robert Metzger
Thank you for all your input.

@Chesnay, in your email you are pointing to the same PR twice:
This PR fixes the compilation on Windows:  (reviewed once, most recent
changes not reviewed)
https://github.com/apache/flink/pull/3854
This PR enables a test for savepoint compatibility: (nice to have, easy to
review)
https://github.com/apache/flink/pull/3854

Also the "should define more than one task slot" thing is not important IMO.

I think the "empty path on windows" thing is not a release blocker.

--

These are the issues mentioned in the thread that are still open and
blockers:
- Add nested serializers to config snapshots of composite serializers:
https://github.com/apache/flink/pull/3937 has no review yet
- FLINK-6610  WebServer
could not be created,when set the "jobmanager.web.submit.enable" to false
- FLINK-6629  ClusterClient
cannot submit jobs to HA cluster if address not set in configuration



On Fri, May 19, 2017 at 12:17 AM, Till Rohrmann 
wrote:

> I might have found another blocker:
> https://issues.apache.org/jira/browse/FLINK-6629.
>
> The issue is that the ClusterClient only allows to submit jobs to an HA
> cluster if you have specified the JobManager's address in the
> flink-conf.yaml or via the command line options. If no address is set, then
> it fails completely. If the wrong address is set, which can easily happen
> in an HA setting, then we are not able to find the proper connecting
> address for the ActorSystem. This basically voids Flink's HA capabilities.
>
> Cheers,
> Till
>
> On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler 
> wrote:
>
> > The test document says that the default flink-conf.yml "should define
> more
> > than one task slot", but it currently configures exactly 1 task slot. Not
> > sure if it is a typo in the doc though.
> >
> >
> > On 18.05.2017 22:10, Chesnay Schepler wrote:
> >
> >> The start-cluster.sh script failed for me on Windows when executed in a
> >> directory containing spaces.
> >>
> >> On 18.05.2017 20:47, Chesnay Schepler wrote:
> >>
> >>> FLINK-6610 should also be fixed; it is currently not possible to
> disable
> >>> web-submissions.
> >>>
> >>> On 18.05.2017 18:13, jincheng sun wrote:
> >>>
>  Hi Robert,
>  I have some checks to do and some test improve PRs (
>  https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.
> 
>  Best,
>  SunJincheng
> 
>  2017-05-18 22:17 GMT+08:00 Greg Hogan :
> 
>  The following tickets for 1.3.0 have a PR in need of review:
> >
> > [FLINK-6582] [docs] Project from maven archetype is not buildable by
> > default
> > [FLINK-6616] [docs] Clarify provenance of official Docker images
> >
> >
> > On May 18, 2017, at 5:40 AM, Fabian Hueske 
> wrote:
> >>
> >> I have a couple of PRs ready with bugfixes that I'll try to get in
> as
> >>
> > well.
> >
> >> Should be done soon.
> >>
> >> 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
> >>
> >> I'd like to get a fix in for
> >>> https://issues.apache.org/jira/browse/FLINK-6612. This can
> basically
> >>> thwart
> >>> Flink's recovery capabilities.
> >>>
> >>> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler <
> >>> ches...@apache.org>
> >>> wrote:
> >>>
> >>> This PR reduces logging noise a bit: (got +1 to merge)
>  https://github.com/apache/flink/pull/3917
> 
>  This PR fixes the compilation on Windows:  (reviewed once, most
>  recent
>  changes not reviewed)
>  https://github.com/apache/flink/pull/3854
> 
>  This PR enables a test for savepoint compatibility: (nice to have,
>  easy
> 
> >>> to
> >>>
>  review)
>  https://github.com/apache/flink/pull/3854
> 
>  These 2 PRs fix minor issues with metrics: (trivial review, both
>  one-liners)
>  https://github.com/apache/flink/pull/3906
>  https://github.com/apache/flink/pull/3907
> 
> 
>  On 18.05.2017 10:52, Robert Metzger wrote:
> 
>  I will.
> > Actually I had it already on my radar because its one of the
> three
> > remaining blockers.
> >
> > Your JIRA has already a PR so I guess its on a good track, for
> the
> >
>  other
> >
> >> blockers, I think its fine to release without having them fixed.
> > Is there anything else we need to get into the 1.3.0 release?
> > Otherwise, I would soon create the first voting RC.
> >
> >
> >
> > On Wed, May 17, 2017 at 8:49 PM, Eron Wright <
> eronwri...@gmail.com
> > >
> > wrote:
> >
> > Robert, please add FLINK-6606 to the list of JIRAs that you're
> >
> >>>

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Till Rohrmann
I might have found another blocker:
https://issues.apache.org/jira/browse/FLINK-6629.

The issue is that the ClusterClient only allows to submit jobs to an HA
cluster if you have specified the JobManager's address in the
flink-conf.yaml or via the command line options. If no address is set, then
it fails completely. If the wrong address is set, which can easily happen
in an HA setting, then we are not able to find the proper connecting
address for the ActorSystem. This basically voids Flink's HA capabilities.

Cheers,
Till

On Thu, May 18, 2017 at 10:23 PM, Chesnay Schepler 
wrote:

> The test document says that the default flink-conf.yml "should define more
> than one task slot", but it currently configures exactly 1 task slot. Not
> sure if it is a typo in the doc though.
>
>
> On 18.05.2017 22:10, Chesnay Schepler wrote:
>
>> The start-cluster.sh script failed for me on Windows when executed in a
>> directory containing spaces.
>>
>> On 18.05.2017 20:47, Chesnay Schepler wrote:
>>
>>> FLINK-6610 should also be fixed; it is currently not possible to disable
>>> web-submissions.
>>>
>>> On 18.05.2017 18:13, jincheng sun wrote:
>>>
 Hi Robert,
 I have some checks to do and some test improve PRs (
 https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

 Best,
 SunJincheng

 2017-05-18 22:17 GMT+08:00 Greg Hogan :

 The following tickets for 1.3.0 have a PR in need of review:
>
> [FLINK-6582] [docs] Project from maven archetype is not buildable by
> default
> [FLINK-6616] [docs] Clarify provenance of official Docker images
>
>
> On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:
>>
>> I have a couple of PRs ready with bugfixes that I'll try to get in as
>>
> well.
>
>> Should be done soon.
>>
>> 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
>>
>> I'd like to get a fix in for
>>> https://issues.apache.org/jira/browse/FLINK-6612. This can basically
>>> thwart
>>> Flink's recovery capabilities.
>>>
>>> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler <
>>> ches...@apache.org>
>>> wrote:
>>>
>>> This PR reduces logging noise a bit: (got +1 to merge)
 https://github.com/apache/flink/pull/3917

 This PR fixes the compilation on Windows:  (reviewed once, most
 recent
 changes not reviewed)
 https://github.com/apache/flink/pull/3854

 This PR enables a test for savepoint compatibility: (nice to have,
 easy

>>> to
>>>
 review)
 https://github.com/apache/flink/pull/3854

 These 2 PRs fix minor issues with metrics: (trivial review, both
 one-liners)
 https://github.com/apache/flink/pull/3906
 https://github.com/apache/flink/pull/3907


 On 18.05.2017 10:52, Robert Metzger wrote:

 I will.
> Actually I had it already on my radar because its one of the three
> remaining blockers.
>
> Your JIRA has already a PR so I guess its on a good track, for the
>
 other
>
>> blockers, I think its fine to release without having them fixed.
> Is there anything else we need to get into the 1.3.0 release?
> Otherwise, I would soon create the first voting RC.
>
>
>
> On Wed, May 17, 2017 at 8:49 PM, Eron Wright  >
> wrote:
>
> Robert, please add FLINK-6606 to the list of JIRAs that you're
>
 tracking,
>
>> thanks.
>>
>> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger <
>> rmetz...@apache.org
>> wrote:
>>
>> I totally forgot to post a document with testing tasks in the RC0
>>
> thread,
>>>
 so I'll do it in the RC1 thread.
>>>
>>> Please use this document:
>>> https://docs.google.com/document/d/11WCfV15VwQNF-
>>> Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
>>>
>>> If I have the feeling that not enough people are seeing the
>>>
>> document,
>
>> I'll
>>
>> write a dedicated email to user@ and dev@ :)
>>>
>>>
>>> On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <
>>>
>> rmetz...@apache.org>
>
>> wrote:
>>>
>>> Thanks for the pointer. I'll keep an eye on the JIRA.
>>>
 I've gone through the JIRAs tagged with 1.3.0 yesterday to
 create a

 list
>>> of new features in 1.3. Feel free to add more / change it in the
>>>
>> wiki:
>
>> https://cwiki.apache.org/confluence/display/FLINK/
 Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3


 On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra <

Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Chesnay Schepler
The test document says that the default flink-conf.yml "should define 
more than one task slot", but it currently configures exactly 1 task 
slot. Not sure if it is a typo in the doc though.


On 18.05.2017 22:10, Chesnay Schepler wrote:
The start-cluster.sh script failed for me on Windows when executed in 
a directory containing spaces.


On 18.05.2017 20:47, Chesnay Schepler wrote:
FLINK-6610 should also be fixed; it is currently not possible to 
disable web-submissions.


On 18.05.2017 18:13, jincheng sun wrote:

Hi Robert,
I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :


The following tickets for 1.3.0 have a PR in need of review:

[FLINK-6582] [docs] Project from maven archetype is not buildable by
default
[FLINK-6616] [docs] Clarify provenance of official Docker images



On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:

I have a couple of PRs ready with bugfixes that I'll try to get in as

well.

Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann :


I'd like to get a fix in for
https://issues.apache.org/jira/browse/FLINK-6612. This can basically
thwart
Flink's recovery capabilities.

On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 


wrote:


This PR reduces logging noise a bit: (got +1 to merge)
https://github.com/apache/flink/pull/3917

This PR fixes the compilation on Windows:  (reviewed once, most 
recent

changes not reviewed)
https://github.com/apache/flink/pull/3854

This PR enables a test for savepoint compatibility: (nice to 
have, easy

to

review)
https://github.com/apache/flink/pull/3854

These 2 PRs fix minor issues with metrics: (trivial review, both
one-liners)
https://github.com/apache/flink/pull/3906
https://github.com/apache/flink/pull/3907


On 18.05.2017 10:52, Robert Metzger wrote:


I will.
Actually I had it already on my radar because its one of the three
remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for the

other

blockers, I think its fine to release without having them fixed.
Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright 


wrote:

Robert, please add FLINK-6606 to the list of JIRAs that you're

tracking,

thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 

wrote:

I totally forgot to post a document with testing tasks in the RC0

thread,

so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-
Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the

document,

I'll


write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <

rmetz...@apache.org>

wrote:

Thanks for the pointer. I'll keep an eye on the JIRA.
I've gone through the JIRAs tagged with 1.3.0 yesterday to 
create a



list
of new features in 1.3. Feel free to add more / change it in the

wiki:

https://cwiki.apache.org/confluence/display/FLINK/
Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3 



On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 

wrote:


Thanks Robert,

Just for the record I think there are still some problems with


incremental
snapshots, I think Stefan is still working on it.

I added some comments to https://issues.apache.org/


jira/browse/FLINK-6537
Gyula

Robert Metzger  ezt írta (időpont: 2017.

máj.

15.,

H,

19:41):

Hi Devs,

This is the second non-voting RC. The last RC had some big

issues,

making

it hard to start Flink locally. I hope this RC proves to 
be more



stable.

I hope to create the first voting RC by end of this week.




-

The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85

The artifacts are located here:
http://people.apache.org/~rmetzger/flink-1.3.0-rc1/

The maven staging repository is here:
https://repository.apache.org/content/repositories/


orgapacheflink-1119



-

Happy testing!

Regards,
Robert














Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Chesnay Schepler
The start-cluster.sh script failed for me on Windows when executed in a 
directory containing spaces.


On 18.05.2017 20:47, Chesnay Schepler wrote:
FLINK-6610 should also be fixed; it is currently not possible to 
disable web-submissions.


On 18.05.2017 18:13, jincheng sun wrote:

Hi Robert,
I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :


The following tickets for 1.3.0 have a PR in need of review:

[FLINK-6582] [docs] Project from maven archetype is not buildable by
default
[FLINK-6616] [docs] Clarify provenance of official Docker images



On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:

I have a couple of PRs ready with bugfixes that I'll try to get in as

well.

Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann :


I'd like to get a fix in for
https://issues.apache.org/jira/browse/FLINK-6612. This can basically
thwart
Flink's recovery capabilities.

On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 


wrote:


This PR reduces logging noise a bit: (got +1 to merge)
https://github.com/apache/flink/pull/3917

This PR fixes the compilation on Windows:  (reviewed once, most 
recent

changes not reviewed)
https://github.com/apache/flink/pull/3854

This PR enables a test for savepoint compatibility: (nice to 
have, easy

to

review)
https://github.com/apache/flink/pull/3854

These 2 PRs fix minor issues with metrics: (trivial review, both
one-liners)
https://github.com/apache/flink/pull/3906
https://github.com/apache/flink/pull/3907


On 18.05.2017 10:52, Robert Metzger wrote:


I will.
Actually I had it already on my radar because its one of the three
remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for the

other

blockers, I think its fine to release without having them fixed.
Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
wrote:

Robert, please add FLINK-6606 to the list of JIRAs that you're

tracking,

thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 

wrote:

I totally forgot to post a document with testing tasks in the RC0

thread,

so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-
Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the

document,

I'll


write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <

rmetz...@apache.org>

wrote:

Thanks for the pointer. I'll keep an eye on the JIRA.
I've gone through the JIRAs tagged with 1.3.0 yesterday to 
create a



list
of new features in 1.3. Feel free to add more / change it in the

wiki:

https://cwiki.apache.org/confluence/display/FLINK/
Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3 



On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 

wrote:


Thanks Robert,

Just for the record I think there are still some problems with


incremental
snapshots, I think Stefan is still working on it.

I added some comments to https://issues.apache.org/


jira/browse/FLINK-6537
Gyula

Robert Metzger  ezt írta (időpont: 2017.

máj.

15.,

H,

19:41):

Hi Devs,

This is the second non-voting RC. The last RC had some big

issues,

making

it hard to start Flink locally. I hope this RC proves to be 
more



stable.

I hope to create the first voting RC by end of this week.




-

The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85

The artifacts are located here:
http://people.apache.org/~rmetzger/flink-1.3.0-rc1/

The maven staging repository is here:
https://repository.apache.org/content/repositories/


orgapacheflink-1119



-

Happy testing!

Regards,
Robert











Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Chesnay Schepler
FLINK-6610 should also be fixed; it is currently not possible to disable 
web-submissions.


On 18.05.2017 18:13, jincheng sun wrote:

Hi Robert,
I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :


The following tickets for 1.3.0 have a PR in need of review:

[FLINK-6582] [docs] Project from maven archetype is not buildable by
default
[FLINK-6616] [docs] Clarify provenance of official Docker images



On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:

I have a couple of PRs ready with bugfixes that I'll try to get in as

well.

Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann :


I'd like to get a fix in for
https://issues.apache.org/jira/browse/FLINK-6612. This can basically
thwart
Flink's recovery capabilities.

On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 
wrote:


This PR reduces logging noise a bit: (got +1 to merge)
https://github.com/apache/flink/pull/3917

This PR fixes the compilation on Windows:  (reviewed once, most recent
changes not reviewed)
https://github.com/apache/flink/pull/3854

This PR enables a test for savepoint compatibility: (nice to have, easy

to

review)
https://github.com/apache/flink/pull/3854

These 2 PRs fix minor issues with metrics: (trivial review, both
one-liners)
https://github.com/apache/flink/pull/3906
https://github.com/apache/flink/pull/3907


On 18.05.2017 10:52, Robert Metzger wrote:


I will.
Actually I had it already on my radar because its one of the three
remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for the

other

blockers, I think its fine to release without having them fixed.
Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
wrote:

Robert, please add FLINK-6606 to the list of JIRAs that you're

tracking,

thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
thread,

so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-
Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the

document,

I'll


write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <

rmetz...@apache.org>

wrote:

Thanks for the pointer. I'll keep an eye on the JIRA.

I've gone through the JIRAs tagged with 1.3.0 yesterday to create a


list
of new features in 1.3. Feel free to add more / change it in the

wiki:

https://cwiki.apache.org/confluence/display/FLINK/
Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3

On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
wrote:


Thanks Robert,

Just for the record I think there are still some problems with


incremental
snapshots, I think Stefan is still working on it.

I added some comments to https://issues.apache.org/


jira/browse/FLINK-6537
Gyula

Robert Metzger  ezt írta (időpont: 2017.

máj.

15.,

H,

19:41):

Hi Devs,

This is the second non-voting RC. The last RC had some big

issues,

making


it hard to start Flink locally. I hope this RC proves to be more


stable.

I hope to create the first voting RC by end of this week.




-

The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85

The artifacts are located here:
http://people.apache.org/~rmetzger/flink-1.3.0-rc1/

The maven staging repository is here:
https://repository.apache.org/content/repositories/


orgapacheflink-1119



-

Happy testing!

Regards,
Robert








Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread jincheng sun
Hi Robert,
I have some checks to do and some test improve PRs (
https://issues.apache.org/jira/browse/FLINK-6619) need be done soon.

Best,
SunJincheng

2017-05-18 22:17 GMT+08:00 Greg Hogan :

> The following tickets for 1.3.0 have a PR in need of review:
>
> [FLINK-6582] [docs] Project from maven archetype is not buildable by
> default
> [FLINK-6616] [docs] Clarify provenance of official Docker images
>
>
> > On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:
> >
> > I have a couple of PRs ready with bugfixes that I'll try to get in as
> well.
> > Should be done soon.
> >
> > 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
> >
> >> I'd like to get a fix in for
> >> https://issues.apache.org/jira/browse/FLINK-6612. This can basically
> >> thwart
> >> Flink's recovery capabilities.
> >>
> >> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 
> >> wrote:
> >>
> >>> This PR reduces logging noise a bit: (got +1 to merge)
> >>> https://github.com/apache/flink/pull/3917
> >>>
> >>> This PR fixes the compilation on Windows:  (reviewed once, most recent
> >>> changes not reviewed)
> >>> https://github.com/apache/flink/pull/3854
> >>>
> >>> This PR enables a test for savepoint compatibility: (nice to have, easy
> >> to
> >>> review)
> >>> https://github.com/apache/flink/pull/3854
> >>>
> >>> These 2 PRs fix minor issues with metrics: (trivial review, both
> >>> one-liners)
> >>> https://github.com/apache/flink/pull/3906
> >>> https://github.com/apache/flink/pull/3907
> >>>
> >>>
> >>> On 18.05.2017 10:52, Robert Metzger wrote:
> >>>
>  I will.
>  Actually I had it already on my radar because its one of the three
>  remaining blockers.
> 
>  Your JIRA has already a PR so I guess its on a good track, for the
> other
>  blockers, I think its fine to release without having them fixed.
>  Is there anything else we need to get into the 1.3.0 release?
>  Otherwise, I would soon create the first voting RC.
> 
> 
> 
>  On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
>  wrote:
> 
>  Robert, please add FLINK-6606 to the list of JIRAs that you're
> tracking,
> > thanks.
> >
> > On Tue, May 16, 2017 at 8:30 AM, Robert Metzger  >
> > wrote:
> >
> > I totally forgot to post a document with testing tasks in the RC0
> >> thread,
> >> so I'll do it in the RC1 thread.
> >>
> >> Please use this document:
> >> https://docs.google.com/document/d/11WCfV15VwQNF-
> >> Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
> >>
> >> If I have the feeling that not enough people are seeing the
> document,
> >>
> > I'll
> >
> >> write a dedicated email to user@ and dev@ :)
> >>
> >>
> >> On Tue, May 16, 2017 at 9:26 AM, Robert Metzger <
> rmetz...@apache.org>
> >> wrote:
> >>
> >> Thanks for the pointer. I'll keep an eye on the JIRA.
> >>>
> >>> I've gone through the JIRAs tagged with 1.3.0 yesterday to create a
> >>>
> >> list
> >
> >> of new features in 1.3. Feel free to add more / change it in the
> wiki:
> >>> https://cwiki.apache.org/confluence/display/FLINK/
> >>> Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
> >>>
> >>> On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra  >
> >>>
> >> wrote:
> >>
> >>> Thanks Robert,
> 
>  Just for the record I think there are still some problems with
> 
> >>> incremental
> >>
> >>> snapshots, I think Stefan is still working on it.
> 
>  I added some comments to https://issues.apache.org/
> 
> >>> jira/browse/FLINK-6537
> >>
> >>> Gyula
> 
>  Robert Metzger  ezt írta (időpont: 2017.
> máj.
> 
> >>> 15.,
> >
> >> H,
>  19:41):
> 
>  Hi Devs,
> >
> > This is the second non-voting RC. The last RC had some big
> issues,
> >
>  making
> 
> > it hard to start Flink locally. I hope this RC proves to be more
> >
>  stable.
> >>
> >>> I hope to create the first voting RC by end of this week.
> >
> > 
> >
>  -
> >
> >> The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> >
> > The artifacts are located here:
> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> >
> > The maven staging repository is here:
> > https://repository.apache.org/content/repositories/
> >
>  orgapacheflink-1119
> >>
> >>> 
> >
>  -
> >
> >> Happy testing!
> >
> > Regards,
> > Robert
> >
> >
> >>>
> >>>
> >>
>
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Greg Hogan
The following tickets for 1.3.0 have a PR in need of review:

[FLINK-6582] [docs] Project from maven archetype is not buildable by default
[FLINK-6616] [docs] Clarify provenance of official Docker images


> On May 18, 2017, at 5:40 AM, Fabian Hueske  wrote:
> 
> I have a couple of PRs ready with bugfixes that I'll try to get in as well.
> Should be done soon.
> 
> 2017-05-18 11:24 GMT+02:00 Till Rohrmann :
> 
>> I'd like to get a fix in for
>> https://issues.apache.org/jira/browse/FLINK-6612. This can basically
>> thwart
>> Flink's recovery capabilities.
>> 
>> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 
>> wrote:
>> 
>>> This PR reduces logging noise a bit: (got +1 to merge)
>>> https://github.com/apache/flink/pull/3917
>>> 
>>> This PR fixes the compilation on Windows:  (reviewed once, most recent
>>> changes not reviewed)
>>> https://github.com/apache/flink/pull/3854
>>> 
>>> This PR enables a test for savepoint compatibility: (nice to have, easy
>> to
>>> review)
>>> https://github.com/apache/flink/pull/3854
>>> 
>>> These 2 PRs fix minor issues with metrics: (trivial review, both
>>> one-liners)
>>> https://github.com/apache/flink/pull/3906
>>> https://github.com/apache/flink/pull/3907
>>> 
>>> 
>>> On 18.05.2017 10:52, Robert Metzger wrote:
>>> 
 I will.
 Actually I had it already on my radar because its one of the three
 remaining blockers.
 
 Your JIRA has already a PR so I guess its on a good track, for the other
 blockers, I think its fine to release without having them fixed.
 Is there anything else we need to get into the 1.3.0 release?
 Otherwise, I would soon create the first voting RC.
 
 
 
 On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
 wrote:
 
 Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
> thanks.
> 
> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
> wrote:
> 
> I totally forgot to post a document with testing tasks in the RC0
>> thread,
>> so I'll do it in the RC1 thread.
>> 
>> Please use this document:
>> https://docs.google.com/document/d/11WCfV15VwQNF-
>> Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
>> 
>> If I have the feeling that not enough people are seeing the document,
>> 
> I'll
> 
>> write a dedicated email to user@ and dev@ :)
>> 
>> 
>> On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
>> wrote:
>> 
>> Thanks for the pointer. I'll keep an eye on the JIRA.
>>> 
>>> I've gone through the JIRAs tagged with 1.3.0 yesterday to create a
>>> 
>> list
> 
>> of new features in 1.3. Feel free to add more / change it in the wiki:
>>> https://cwiki.apache.org/confluence/display/FLINK/
>>> Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
>>> 
>>> On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
>>> 
>> wrote:
>> 
>>> Thanks Robert,
 
 Just for the record I think there are still some problems with
 
>>> incremental
>> 
>>> snapshots, I think Stefan is still working on it.
 
 I added some comments to https://issues.apache.org/
 
>>> jira/browse/FLINK-6537
>> 
>>> Gyula
 
 Robert Metzger  ezt írta (időpont: 2017. máj.
 
>>> 15.,
> 
>> H,
 19:41):
 
 Hi Devs,
> 
> This is the second non-voting RC. The last RC had some big issues,
> 
 making
 
> it hard to start Flink locally. I hope this RC proves to be more
> 
 stable.
>> 
>>> I hope to create the first voting RC by end of this week.
> 
> 
> 
 -
> 
>> The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> 
> The artifacts are located here:
> http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> 
> The maven staging repository is here:
> https://repository.apache.org/content/repositories/
> 
 orgapacheflink-1119
>> 
>>> 
> 
 -
> 
>> Happy testing!
> 
> Regards,
> Robert
> 
> 
>>> 
>>> 
>> 



Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Fabian Hueske
I have a couple of PRs ready with bugfixes that I'll try to get in as well.
Should be done soon.

2017-05-18 11:24 GMT+02:00 Till Rohrmann :

> I'd like to get a fix in for
> https://issues.apache.org/jira/browse/FLINK-6612. This can basically
> thwart
> Flink's recovery capabilities.
>
> On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 
> wrote:
>
> > This PR reduces logging noise a bit: (got +1 to merge)
> > https://github.com/apache/flink/pull/3917
> >
> > This PR fixes the compilation on Windows:  (reviewed once, most recent
> > changes not reviewed)
> > https://github.com/apache/flink/pull/3854
> >
> > This PR enables a test for savepoint compatibility: (nice to have, easy
> to
> > review)
> > https://github.com/apache/flink/pull/3854
> >
> > These 2 PRs fix minor issues with metrics: (trivial review, both
> > one-liners)
> > https://github.com/apache/flink/pull/3906
> > https://github.com/apache/flink/pull/3907
> >
> >
> > On 18.05.2017 10:52, Robert Metzger wrote:
> >
> >> I will.
> >> Actually I had it already on my radar because its one of the three
> >> remaining blockers.
> >>
> >> Your JIRA has already a PR so I guess its on a good track, for the other
> >> blockers, I think its fine to release without having them fixed.
> >> Is there anything else we need to get into the 1.3.0 release?
> >> Otherwise, I would soon create the first voting RC.
> >>
> >>
> >>
> >> On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
> >> wrote:
> >>
> >> Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
> >>> thanks.
> >>>
> >>> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
> >>> wrote:
> >>>
> >>> I totally forgot to post a document with testing tasks in the RC0
> thread,
>  so I'll do it in the RC1 thread.
> 
>  Please use this document:
>  https://docs.google.com/document/d/11WCfV15VwQNF-
>  Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
> 
>  If I have the feeling that not enough people are seeing the document,
> 
> >>> I'll
> >>>
>  write a dedicated email to user@ and dev@ :)
> 
> 
>  On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
>  wrote:
> 
>  Thanks for the pointer. I'll keep an eye on the JIRA.
> >
> > I've gone through the JIRAs tagged with 1.3.0 yesterday to create a
> >
>  list
> >>>
>  of new features in 1.3. Feel free to add more / change it in the wiki:
> > https://cwiki.apache.org/confluence/display/FLINK/
> > Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
> >
> > On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
> >
>  wrote:
> 
> > Thanks Robert,
> >>
> >> Just for the record I think there are still some problems with
> >>
> > incremental
> 
> > snapshots, I think Stefan is still working on it.
> >>
> >> I added some comments to https://issues.apache.org/
> >>
> > jira/browse/FLINK-6537
> 
> > Gyula
> >>
> >> Robert Metzger  ezt írta (időpont: 2017. máj.
> >>
> > 15.,
> >>>
>  H,
> >> 19:41):
> >>
> >> Hi Devs,
> >>>
> >>> This is the second non-voting RC. The last RC had some big issues,
> >>>
> >> making
> >>
> >>> it hard to start Flink locally. I hope this RC proves to be more
> >>>
> >> stable.
> 
> > I hope to create the first voting RC by end of this week.
> >>>
> >>> 
> >>>
> >> -
> >>>
>  The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> >>>
> >>> The artifacts are located here:
> >>> http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> >>>
> >>> The maven staging repository is here:
> >>> https://repository.apache.org/content/repositories/
> >>>
> >> orgapacheflink-1119
> 
> > 
> >>>
> >> -
> >>>
>  Happy testing!
> >>>
> >>> Regards,
> >>> Robert
> >>>
> >>>
> >
> >
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Till Rohrmann
I'd like to get a fix in for
https://issues.apache.org/jira/browse/FLINK-6612. This can basically thwart
Flink's recovery capabilities.

On Thu, May 18, 2017 at 11:13 AM, Chesnay Schepler 
wrote:

> This PR reduces logging noise a bit: (got +1 to merge)
> https://github.com/apache/flink/pull/3917
>
> This PR fixes the compilation on Windows:  (reviewed once, most recent
> changes not reviewed)
> https://github.com/apache/flink/pull/3854
>
> This PR enables a test for savepoint compatibility: (nice to have, easy to
> review)
> https://github.com/apache/flink/pull/3854
>
> These 2 PRs fix minor issues with metrics: (trivial review, both
> one-liners)
> https://github.com/apache/flink/pull/3906
> https://github.com/apache/flink/pull/3907
>
>
> On 18.05.2017 10:52, Robert Metzger wrote:
>
>> I will.
>> Actually I had it already on my radar because its one of the three
>> remaining blockers.
>>
>> Your JIRA has already a PR so I guess its on a good track, for the other
>> blockers, I think its fine to release without having them fixed.
>> Is there anything else we need to get into the 1.3.0 release?
>> Otherwise, I would soon create the first voting RC.
>>
>>
>>
>> On Wed, May 17, 2017 at 8:49 PM, Eron Wright 
>> wrote:
>>
>> Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
>>> thanks.
>>>
>>> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
>>> wrote:
>>>
>>> I totally forgot to post a document with testing tasks in the RC0 thread,
 so I'll do it in the RC1 thread.

 Please use this document:
 https://docs.google.com/document/d/11WCfV15VwQNF-
 Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

 If I have the feeling that not enough people are seeing the document,

>>> I'll
>>>
 write a dedicated email to user@ and dev@ :)


 On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
 wrote:

 Thanks for the pointer. I'll keep an eye on the JIRA.
>
> I've gone through the JIRAs tagged with 1.3.0 yesterday to create a
>
 list
>>>
 of new features in 1.3. Feel free to add more / change it in the wiki:
> https://cwiki.apache.org/confluence/display/FLINK/
> Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
>
> On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
>
 wrote:

> Thanks Robert,
>>
>> Just for the record I think there are still some problems with
>>
> incremental

> snapshots, I think Stefan is still working on it.
>>
>> I added some comments to https://issues.apache.org/
>>
> jira/browse/FLINK-6537

> Gyula
>>
>> Robert Metzger  ezt írta (időpont: 2017. máj.
>>
> 15.,
>>>
 H,
>> 19:41):
>>
>> Hi Devs,
>>>
>>> This is the second non-voting RC. The last RC had some big issues,
>>>
>> making
>>
>>> it hard to start Flink locally. I hope this RC proves to be more
>>>
>> stable.

> I hope to create the first voting RC by end of this week.
>>>
>>> 
>>>
>> -
>>>
 The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
>>>
>>> The artifacts are located here:
>>> http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
>>>
>>> The maven staging repository is here:
>>> https://repository.apache.org/content/repositories/
>>>
>> orgapacheflink-1119

> 
>>>
>> -
>>>
 Happy testing!
>>>
>>> Regards,
>>> Robert
>>>
>>>
>
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Chesnay Schepler

This PR reduces logging noise a bit: (got +1 to merge)
https://github.com/apache/flink/pull/3917

This PR fixes the compilation on Windows:  (reviewed once, most recent 
changes not reviewed)

https://github.com/apache/flink/pull/3854

This PR enables a test for savepoint compatibility: (nice to have, easy 
to review)

https://github.com/apache/flink/pull/3854

These 2 PRs fix minor issues with metrics: (trivial review, both one-liners)
https://github.com/apache/flink/pull/3906
https://github.com/apache/flink/pull/3907

On 18.05.2017 10:52, Robert Metzger wrote:

I will.
Actually I had it already on my radar because its one of the three
remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for the other
blockers, I think its fine to release without having them fixed.
Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright  wrote:


Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
wrote:


I totally forgot to post a document with testing tasks in the RC0 thread,
so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-
Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the document,

I'll

write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
wrote:


Thanks for the pointer. I'll keep an eye on the JIRA.

I've gone through the JIRAs tagged with 1.3.0 yesterday to create a

list

of new features in 1.3. Feel free to add more / change it in the wiki:
https://cwiki.apache.org/confluence/display/FLINK/
Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3

On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 

wrote:

Thanks Robert,

Just for the record I think there are still some problems with

incremental

snapshots, I think Stefan is still working on it.

I added some comments to https://issues.apache.org/

jira/browse/FLINK-6537

Gyula

Robert Metzger  ezt írta (időpont: 2017. máj.

15.,

H,
19:41):


Hi Devs,

This is the second non-voting RC. The last RC had some big issues,

making

it hard to start Flink locally. I hope this RC proves to be more

stable.

I hope to create the first voting RC by end of this week.



-

The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85

The artifacts are located here:
http://people.apache.org/~rmetzger/flink-1.3.0-rc1/

The maven staging repository is here:
https://repository.apache.org/content/repositories/

orgapacheflink-1119



-

Happy testing!

Regards,
Robert







Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Tzu-Li (Gordon) Tai
Hi Robert,

There is one last pending fix for the serializer upgrades feature: 
https://issues.apache.org/jira/browse/FLINK-6482.
Pending PR: https://github.com/apache/flink/pull/3937.

I can’t say its a complete blocker, but since it will affect serialization 
format of checkpoints, it would be best to get it in now for the release 
instead of afterwards which would then require more code for backwards 
compatibility with 1.3.0. I think its better to try to avoid that if possible.

Thanks,
Gordon

On 18 May 2017 at 4:52:24 PM, Robert Metzger (rmetz...@apache.org) wrote:

I will.  
Actually I had it already on my radar because its one of the three  
remaining blockers.  

Your JIRA has already a PR so I guess its on a good track, for the other  
blockers, I think its fine to release without having them fixed.  
Is there anything else we need to get into the 1.3.0 release?  
Otherwise, I would soon create the first voting RC.  



On Wed, May 17, 2017 at 8:49 PM, Eron Wright  wrote:  

> Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,  
> thanks.  
>  
> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger   
> wrote:  
>  
> > I totally forgot to post a document with testing tasks in the RC0 thread,  
> > so I'll do it in the RC1 thread.  
> >  
> > Please use this document:  
> > https://docs.google.com/document/d/11WCfV15VwQNF-  
> > Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#  
> >  
> > If I have the feeling that not enough people are seeing the document,  
> I'll  
> > write a dedicated email to user@ and dev@ :)  
> >  
> >  
> > On Tue, May 16, 2017 at 9:26 AM, Robert Metzger   
> > wrote:  
> >  
> > > Thanks for the pointer. I'll keep an eye on the JIRA.  
> > >  
> > > I've gone through the JIRAs tagged with 1.3.0 yesterday to create a  
> list  
> > > of new features in 1.3. Feel free to add more / change it in the wiki:  
> > > https://cwiki.apache.org/confluence/display/FLINK/  
> > > Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3  
> > >  
> > > On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra   
> > wrote:  
> > >  
> > >> Thanks Robert,  
> > >>  
> > >> Just for the record I think there are still some problems with  
> > incremental  
> > >> snapshots, I think Stefan is still working on it.  
> > >>  
> > >> I added some comments to https://issues.apache.org/  
> > jira/browse/FLINK-6537  
> > >>  
> > >> Gyula  
> > >>  
> > >> Robert Metzger  ezt írta (időpont: 2017. máj.  
> 15.,  
> > >> H,  
> > >> 19:41):  
> > >>  
> > >> > Hi Devs,  
> > >> >  
> > >> > This is the second non-voting RC. The last RC had some big issues,  
> > >> making  
> > >> > it hard to start Flink locally. I hope this RC proves to be more  
> > stable.  
> > >> >  
> > >> > I hope to create the first voting RC by end of this week.  
> > >> >  
> > >> >   
> -  
> > >> >  
> > >> > The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85  
> > >> >  
> > >> > The artifacts are located here:  
> > >> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/  
> > >> >  
> > >> > The maven staging repository is here:  
> > >> > https://repository.apache.org/content/repositories/  
> > orgapacheflink-1119  
> > >> >  
> > >> >   
> -  
> > >> >  
> > >> > Happy testing!  
> > >> >  
> > >> > Regards,  
> > >> > Robert  
> > >> >  
> > >>  
> > >  
> > >  
> >  
>  


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-18 Thread Robert Metzger
I will.
Actually I had it already on my radar because its one of the three
remaining blockers.

Your JIRA has already a PR so I guess its on a good track, for the other
blockers, I think its fine to release without having them fixed.
Is there anything else we need to get into the 1.3.0 release?
Otherwise, I would soon create the first voting RC.



On Wed, May 17, 2017 at 8:49 PM, Eron Wright  wrote:

> Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
> thanks.
>
> On Tue, May 16, 2017 at 8:30 AM, Robert Metzger 
> wrote:
>
> > I totally forgot to post a document with testing tasks in the RC0 thread,
> > so I'll do it in the RC1 thread.
> >
> > Please use this document:
> > https://docs.google.com/document/d/11WCfV15VwQNF-
> > Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
> >
> > If I have the feeling that not enough people are seeing the document,
> I'll
> > write a dedicated email to user@ and dev@ :)
> >
> >
> > On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
> > wrote:
> >
> > > Thanks for the pointer. I'll keep an eye on the JIRA.
> > >
> > > I've gone through the JIRAs tagged with 1.3.0 yesterday to create a
> list
> > > of new features in 1.3. Feel free to add more / change it in the wiki:
> > > https://cwiki.apache.org/confluence/display/FLINK/
> > > Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
> > >
> > > On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
> > wrote:
> > >
> > >> Thanks Robert,
> > >>
> > >> Just for the record I think there are still some problems with
> > incremental
> > >> snapshots, I think Stefan is still working on it.
> > >>
> > >> I added some comments to https://issues.apache.org/
> > jira/browse/FLINK-6537
> > >>
> > >> Gyula
> > >>
> > >> Robert Metzger  ezt írta (időpont: 2017. máj.
> 15.,
> > >> H,
> > >> 19:41):
> > >>
> > >> > Hi Devs,
> > >> >
> > >> > This is the second non-voting RC. The last RC had some big issues,
> > >> making
> > >> > it hard to start Flink locally. I hope this RC proves to be more
> > stable.
> > >> >
> > >> > I hope to create the first voting RC by end of this week.
> > >> >
> > >> > 
> -
> > >> >
> > >> > The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> > >> >
> > >> > The artifacts are located here:
> > >> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> > >> >
> > >> > The maven staging repository is here:
> > >> > https://repository.apache.org/content/repositories/
> > orgapacheflink-1119
> > >> >
> > >> > 
> -
> > >> >
> > >> > Happy testing!
> > >> >
> > >> > Regards,
> > >> > Robert
> > >> >
> > >>
> > >
> > >
> >
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-17 Thread Eron Wright
Robert, please add FLINK-6606 to the list of JIRAs that you're tracking,
thanks.

On Tue, May 16, 2017 at 8:30 AM, Robert Metzger  wrote:

> I totally forgot to post a document with testing tasks in the RC0 thread,
> so I'll do it in the RC1 thread.
>
> Please use this document:
> https://docs.google.com/document/d/11WCfV15VwQNF-
> Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#
>
> If I have the feeling that not enough people are seeing the document, I'll
> write a dedicated email to user@ and dev@ :)
>
>
> On Tue, May 16, 2017 at 9:26 AM, Robert Metzger 
> wrote:
>
> > Thanks for the pointer. I'll keep an eye on the JIRA.
> >
> > I've gone through the JIRAs tagged with 1.3.0 yesterday to create a list
> > of new features in 1.3. Feel free to add more / change it in the wiki:
> > https://cwiki.apache.org/confluence/display/FLINK/
> > Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
> >
> > On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra 
> wrote:
> >
> >> Thanks Robert,
> >>
> >> Just for the record I think there are still some problems with
> incremental
> >> snapshots, I think Stefan is still working on it.
> >>
> >> I added some comments to https://issues.apache.org/
> jira/browse/FLINK-6537
> >>
> >> Gyula
> >>
> >> Robert Metzger  ezt írta (időpont: 2017. máj. 15.,
> >> H,
> >> 19:41):
> >>
> >> > Hi Devs,
> >> >
> >> > This is the second non-voting RC. The last RC had some big issues,
> >> making
> >> > it hard to start Flink locally. I hope this RC proves to be more
> stable.
> >> >
> >> > I hope to create the first voting RC by end of this week.
> >> >
> >> > -
> >> >
> >> > The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> >> >
> >> > The artifacts are located here:
> >> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> >> >
> >> > The maven staging repository is here:
> >> > https://repository.apache.org/content/repositories/
> orgapacheflink-1119
> >> >
> >> > -
> >> >
> >> > Happy testing!
> >> >
> >> > Regards,
> >> > Robert
> >> >
> >>
> >
> >
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-16 Thread Robert Metzger
I totally forgot to post a document with testing tasks in the RC0 thread,
so I'll do it in the RC1 thread.

Please use this document:
https://docs.google.com/document/d/11WCfV15VwQNF-Rar4E0RtWiZw1ddEbg5WWf4RFSQ_2Q/edit#

If I have the feeling that not enough people are seeing the document, I'll
write a dedicated email to user@ and dev@ :)


On Tue, May 16, 2017 at 9:26 AM, Robert Metzger  wrote:

> Thanks for the pointer. I'll keep an eye on the JIRA.
>
> I've gone through the JIRAs tagged with 1.3.0 yesterday to create a list
> of new features in 1.3. Feel free to add more / change it in the wiki:
> https://cwiki.apache.org/confluence/display/FLINK/
> Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3
>
> On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra  wrote:
>
>> Thanks Robert,
>>
>> Just for the record I think there are still some problems with incremental
>> snapshots, I think Stefan is still working on it.
>>
>> I added some comments to https://issues.apache.org/jira/browse/FLINK-6537
>>
>> Gyula
>>
>> Robert Metzger  ezt írta (időpont: 2017. máj. 15.,
>> H,
>> 19:41):
>>
>> > Hi Devs,
>> >
>> > This is the second non-voting RC. The last RC had some big issues,
>> making
>> > it hard to start Flink locally. I hope this RC proves to be more stable.
>> >
>> > I hope to create the first voting RC by end of this week.
>> >
>> > -
>> >
>> > The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
>> >
>> > The artifacts are located here:
>> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
>> >
>> > The maven staging repository is here:
>> > https://repository.apache.org/content/repositories/orgapacheflink-1119
>> >
>> > -
>> >
>> > Happy testing!
>> >
>> > Regards,
>> > Robert
>> >
>>
>
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-16 Thread Robert Metzger
Thanks for the pointer. I'll keep an eye on the JIRA.

I've gone through the JIRAs tagged with 1.3.0 yesterday to create a list of
new features in 1.3. Feel free to add more / change it in the wiki:
https://cwiki.apache.org/confluence/display/FLINK/Flink+Release+and+Feature+Plan#FlinkReleaseandFeaturePlan-Flink1.3

On Mon, May 15, 2017 at 10:29 PM, Gyula Fóra  wrote:

> Thanks Robert,
>
> Just for the record I think there are still some problems with incremental
> snapshots, I think Stefan is still working on it.
>
> I added some comments to https://issues.apache.org/jira/browse/FLINK-6537
>
> Gyula
>
> Robert Metzger  ezt írta (időpont: 2017. máj. 15., H,
> 19:41):
>
> > Hi Devs,
> >
> > This is the second non-voting RC. The last RC had some big issues, making
> > it hard to start Flink locally. I hope this RC proves to be more stable.
> >
> > I hope to create the first voting RC by end of this week.
> >
> > -
> >
> > The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
> >
> > The artifacts are located here:
> > http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
> >
> > The maven staging repository is here:
> > https://repository.apache.org/content/repositories/orgapacheflink-1119
> >
> > -
> >
> > Happy testing!
> >
> > Regards,
> > Robert
> >
>


Re: [DISCUSS] Release 1.3.0 RC1 (Non voting, testing release candidate)

2017-05-15 Thread Gyula Fóra
Thanks Robert,

Just for the record I think there are still some problems with incremental
snapshots, I think Stefan is still working on it.

I added some comments to https://issues.apache.org/jira/browse/FLINK-6537

Gyula

Robert Metzger  ezt írta (időpont: 2017. máj. 15., H,
19:41):

> Hi Devs,
>
> This is the second non-voting RC. The last RC had some big issues, making
> it hard to start Flink locally. I hope this RC proves to be more stable.
>
> I hope to create the first voting RC by end of this week.
>
> -
>
> The release commit is 3659a82f553fedf8afe8b5fae75922075fe17e85
>
> The artifacts are located here:
> http://people.apache.org/~rmetzger/flink-1.3.0-rc1/
>
> The maven staging repository is here:
> https://repository.apache.org/content/repositories/orgapacheflink-1119
>
> -
>
> Happy testing!
>
> Regards,
> Robert
>