You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by Robert Metzger <rm...@apache.org> on 2016/02/25 11:34:54 UTC

[VOTE] Release Apache Flink 1.0.0 (RC1)

Dear Flink community,

Please vote on releasing the following candidate as Apache Flink version 1.0
.0.

I've set user@flink.apache.org on CC because users are encouraged to help
testing Flink 1.0.0 for their specific use cases. Please report issues (and
successful tests!) on dev@flink.apache.org.


The commit to be voted on (
http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
e4d308d64057e5f94bec8bbca8f67aab0ea78faa

Branch:
release-1.0.0-rc1 (see
https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
)

The release artifacts to be voted on can be found at:
http://people.apache.org/~rmetzger/flink-1.0.0-rc1/

The release artifacts are signed with the key with fingerprint D9839159:
http://www.apache.org/dist/flink/KEYS

The staging repository for this release can be found at:
https://repository.apache.org/content/repositories/orgapacheflink-1063

-------------------------------------------------------------

The vote is open until Tuesday and passes if a majority of at least three
+1 PMC votes are cast.

The vote ends on Tuesday, March 1, 12:00 CET.

[ ] +1 Release this package as Apache Flink 1.0.0
[ ] -1 Do not release this package because ...

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Ufuk Celebi <uc...@apache.org>.
On Thu, Feb 25, 2016 at 5:23 PM, Vasiliki Kalavri
<va...@gmail.com> wrote:
> - HA: tested on a 6-node cluster with 2 masters.
> Issues:
> 1. After new leader election, the job history is cleaned up (at least in
> the WebUI). Is this on purpose?

Yes, the job history is part of the job manager.

> 2. After cluster restart, the jobmanager remembers and tries to re-submit
> previously failed resubmissions.
> This is one is a bit tricky:
> I had a batch job running and killed the master. After the new master took
> over, job resubmission failed because the HDFS output directory already
> existed. After re-starting the whole cluster and removing the HDFS
> directory, the new jobmanager re-submitted the previously failed batch job.

I think for this you have to set the write mode to overwrite at the moment.

> 3. Upon starting the cluster I get the following warning message "[WARNING]
> 1 instance(s) of jobmanager are already running", when jps shows no
> existing jobmanager process.

This is part of the bash script. It currently checks a PID file to
determine the running processes, but it does not actually check
whether the PIDs are valid or not. I think it's a good idea to
actually check this. Let me open an issue for this...

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Greg Hogan <co...@greghogan.com>.
Hi Vasia,

In the WebUI, the Subtasks and TaskManagers list the same operator
statistics but expand to show either per-subtask or per-TaskManager
statistics. Summarizing the statistics by TaskManager is valuable when
viewing larger clusters.

Greg

On Thu, Feb 25, 2016 at 11:23 AM, Vasiliki Kalavri <
vasilikikalavri@gmail.com> wrote:

> Hi squirrels,
>
> here's my testing outcome so far:
>
> - Examples: Ran all examples locally and on a cluster, both from CLI and
> web submission tool
> Issues:
> 1. PageRank example doesn't run without arguments anymore. I have a fix
> together with some doc fixes.
>
> - CLI: tested locally and on cluster
> Issues: None.
>
> - WebUI: tested locally and on cluster
> Issues:
> 1. Inside a job view, the "Subtasks" and "TaskManagers" tabs have the same
> content. Is this desired?
>
> - HA: tested on a 6-node cluster with 2 masters.
> Issues:
> 1. After new leader election, the job history is cleaned up (at least in
> the WebUI). Is this on purpose?
> 2. After cluster restart, the jobmanager remembers and tries to re-submit
> previously failed resubmissions.
> This is one is a bit tricky:
> I had a batch job running and killed the master. After the new master took
> over, job resubmission failed because the HDFS output directory already
> existed. After re-starting the whole cluster and removing the HDFS
> directory, the new jobmanager re-submitted the previously failed batch job.
> 3. Upon starting the cluster I get the following warning message "[WARNING]
> 1 instance(s) of jobmanager are already running", when jps shows no
> existing jobmanager process.
>
> Let me know if these are valid issues and I should open corresponding JIRAs
> or I'm misunderstanding something :)
>
> Cheers,
> -Vasia.
>
>
> On 25 February 2016 at 17:04, Stephan Ewen <se...@apache.org> wrote:
>
> > Thanks Marton, the issue is quite serious, I agree.
> >
> > It is a bit tricky to solve, unfortunately. It seems very hard to make
> > experience inside the IDE, with Maven, and with SBT smooth.
> > Maven/SBT packaging needs "provided" dependencies, while IntelliJ need
> > "compile" dependencies.
> >
> > On Thu, Feb 25, 2016 at 3:47 PM, Márton Balassi <
> balassi.marton@gmail.com>
> > wrote:
> >
> > > @Stephan on PR #1685. Fair enough, 1.0.1 is fine. We will try to get it
> > in
> > > soon anyway.
> > >
> > > Please consider a build inconvenience that I have just reported to both
> > the
> > > mailing list and JIRA in the meantime. [1]
> > >
> > > [1] https://issues.apache.org/jira/browse/FLINK-3511
> > >
> > >
> > >
> > > On Thu, Feb 25, 2016 at 3:27 PM, Stephan Ewen <se...@apache.org>
> wrote:
> > >
> > > > Concerning the Pull Request mentioned by Marton:
> > > >
> > > > I think it is a transparent bugfix patch. It is not really affecting
> > > > behavior that we want users to make assumptions about (assignment of
> > keys
> > > > to partitions should not be hardwired into applications).
> > > > The only affected program I can think of that is affected is the
> > > "queryable
> > > > state" program (and similar programs), which is using internal API
> and
> > > > behavior that are always subject to change
> > > >
> > > > As such, I would think it is also okay for to be included in a bugfix
> > > > release.
> > > > So if the authors don't have the fix ready in time, it should still
> be
> > > fine
> > > > to make it part of 1.0.1 (which will probably come quite soon, as
> usual
> > > > after a full release).
> > > >
> > > >
> > > > On Thu, Feb 25, 2016 at 2:42 PM, Ufuk Celebi <uc...@apache.org> wrote:
> > > >
> > > > > Maybe you can check whether this is a problem, too:
> > > > > https://issues.apache.org/jira/browse/FLINK-3501
> > > > >
> > > > > Under the assumption that no major functionality changes, I will
> > > > > continue with the functional checks.
> > > > >
> > > > > On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <
> rmetzger@apache.org
> > >
> > > > > wrote:
> > > > > > Damn. I agree that this is a blocker.
> > > > > > I use the maven-enforcer-plugin to check for the right maven, but
> > the
> > > > > build
> > > > > > profile that runs the profile is only active during "deploy", not
> > > when
> > > > > > packaging the binaries.
> > > > > > That's why I didn't realize that I build the binaries with the
> > wrong
> > > > > maven
> > > > > > version.
> > > > > >
> > > > > > I suggest that we keep collecting problems until Friday afternoon
> > > > (CET).
> > > > > > Then I'll create the next release candidate.
> > > > > >
> > > > > > I'd also like to address this one:
> > > > > > https://issues.apache.org/jira/browse/FLINK-3509
> > > > > >
> > > > > >
> > > > > > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <
> fhueske@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > >> Hi folks,
> > > > > >>
> > > > > >> I think I found a release blocker.
> > > > > >> The flink-dist JAR file contains non-relocated classes of Google
> > > Guava
> > > > > and
> > > > > >> Apache HttpComponents.
> > > > > >>
> > > > > >> Fabian
> > > > > >>
> > > > > >> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <chesnay@apache.org
> >:
> > > > > >>
> > > > > >> > tested the RC on Windows:
> > > > > >> >
> > > > > >> > - source compiles
> > > > > >> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > > > >> > - start/stop scripts work in both cygwin and windows CMD
> > > > > >> > - ran several examples from batch/streaming/python
> > > > > >> > - scripts also work on paths containing spaces
> > > > > >> >
> > > > > >> >
> > > > > >> > On 25.02.2016 12:41, Robert Metzger wrote:
> > > > > >> >
> > > > > >> >> (I'm removing user@ from the discussion)
> > > > > >> >>
> > > > > >> >> Thank you for bringing the pull request to my attention
> > Marton. I
> > > > > have
> > > > > >> to
> > > > > >> >> admit that I didn't announce this RC properly in advance. In
> > the
> > > > RC0
> > > > > >> >> thread
> > > > > >> >> I said "early next week" and now its Thursday. I should have
> > said
> > > > > >> >> something
> > > > > >> >> in that thread.
> > > > > >> >> The "trigger" for creating the release was that the number of
> > > > > blocking
> > > > > >> >> issues is 0 now.
> > > > > >> >>
> > > > > >> >> I did a quick check of the open pull requests yesterday
> evening
> > > and
> > > > > >> found
> > > > > >> >> one [1] to be included into the RC as well. Since the PR you
> > > > > mentioned
> > > > > >> is
> > > > > >> >> marked with [WIP] I thought its not yet ready to be merged.
> > > > > >> >>
> > > > > >> >> I would like to find a solution that works for everyone
> here: I
> > > > would
> > > > > >> like
> > > > > >> >> to avoid delaying the release until tomorrow evening, and
> also
> > > the
> > > > > work
> > > > > >> it
> > > > > >> >> incurs for me create a release candidate.
> > > > > >> >> How about the following: We keep this vote open, test and
> check
> > > the
> > > > > >> >> release
> > > > > >> >> and you merge the change to master in the meantime.
> > > > > >> >> Most likely, the release gets cancelled anyways because we
> find
> > > > > >> something
> > > > > >> >> and then the next RC will contain your change.
> > > > > >> >>
> > > > > >> >> [1] https://github.com/apache/flink/pull/1706
> > > > > >> >>
> > > > > >> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > > > > >> >> balassi.marton@gmail.com>
> > > > > >> >> wrote:
> > > > > >> >>
> > > > > >> >> Thanks for creating the candidate Robert and for the
> heads-up,
> > > > Slim.
> > > > > >> >>>
> > > > > >> >>> I would like to get a PR [1] in before 1.0.0 as it breaks
> > > hashing
> > > > > >> >>> behavior
> > > > > >> >>> of DataStream.keyBy. The PR has the feature implemented and
> > the
> > > > java
> > > > > >> >>> tests
> > > > > >> >>> adopted, there is still a bit of outstanding fix for the
> scala
> > > > > tests.
> > > > > >> >>> Gábor
> > > > > >> >>> Horváth or myself will finish it by tomorrow evening.
> > > > > >> >>>
> > > > > >> >>> [1] https://github.com/apache/flink/pull/1685
> > > > > >> >>>
> > > > > >> >>> Best,
> > > > > >> >>>
> > > > > >> >>> Marton
> > > > > >> >>>
> > > > > >> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <
> > > > sbaltagi@gmail.com>
> > > > > >> >>> wrote:
> > > > > >> >>>
> > > > > >> >>> Dear Flink community
> > > > > >> >>>>
> > > > > >> >>>> It is great news that the vote for the first release
> > candidate
> > > > > (RC1)
> > > > > >> of
> > > > > >> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > > > > >> >>>> As a community, we need to double our efforts and make sure
> > > that
> > > > > Flink
> > > > > >> >>>> 1.0.0 is GA before these 2 upcoming major events:
> > > > > >> >>>>
> > > > > >> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31,
> > 2016*
> > > > > >> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14,
> 2016*
> > > > > >> >>>>
> > > > > >> >>>> This is one aspect of the ‘market dynamics’ that we need to
> > > take
> > > > > into
> > > > > >> >>>> account as a community.
> > > > > >> >>>>
> > > > > >> >>>> Good luck!
> > > > > >> >>>>
> > > > > >> >>>> Slim Baltagi
> > > > > >> >>>>
> > > > > >> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <
> > > rmetzger@apache.org
> > > > >
> > > > > >> >>>> wrote:
> > > > > >> >>>>
> > > > > >> >>>> Dear Flink community,
> > > > > >> >>>>
> > > > > >> >>>> Please vote on releasing the following candidate as Apache
> > > Flink
> > > > > >> version
> > > > > >> >>>> 1.0.0.
> > > > > >> >>>>
> > > > > >> >>>> I've set user@flink.apache.org on CC because users are
> > > > encouraged
> > > > > to
> > > > > >> >>>> help testing Flink 1.0.0 for their specific use cases.
> Please
> > > > > report
> > > > > >> >>>> issues
> > > > > >> >>>> (and successful tests!) on dev@flink.apache.org.
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >>>> The commit to be voted on (
> > > > > >> >>>>
> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6
> > )
> > > > > >> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > > > > >> >>>>
> > > > > >> >>>> Branch:
> > > > > >> >>>> release-1.0.0-rc1 (see
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > > > > >> >>>> )
> > > > > >> >>>>
> > > > > >> >>>> The release artifacts to be voted on can be found at:
> > > > > >> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > > > > >> >>>>
> > > > > >> >>>> The release artifacts are signed with the key with
> > fingerprint
> > > > > >> D9839159:
> > > > > >> >>>> http://www.apache.org/dist/flink/KEYS
> > > > > >> >>>>
> > > > > >> >>>> The staging repository for this release can be found at:
> > > > > >> >>>>
> > > > > >>
> > > >
> https://repository.apache.org/content/repositories/orgapacheflink-1063
> > > > > >> >>>>
> > > > > >> >>>>
> -------------------------------------------------------------
> > > > > >> >>>>
> > > > > >> >>>> The vote is open until Tuesday and passes if a majority of
> at
> > > > least
> > > > > >> >>>> three
> > > > > >> >>>> +1 PMC votes are cast.
> > > > > >> >>>>
> > > > > >> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > > > > >> >>>>
> > > > > >> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > > > > >> >>>> [ ] -1 Do not release this package because ...
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Vasiliki Kalavri <va...@gmail.com>.
Hi squirrels,

here's my testing outcome so far:

- Examples: Ran all examples locally and on a cluster, both from CLI and
web submission tool
Issues:
1. PageRank example doesn't run without arguments anymore. I have a fix
together with some doc fixes.

- CLI: tested locally and on cluster
Issues: None.

- WebUI: tested locally and on cluster
Issues:
1. Inside a job view, the "Subtasks" and "TaskManagers" tabs have the same
content. Is this desired?

- HA: tested on a 6-node cluster with 2 masters.
Issues:
1. After new leader election, the job history is cleaned up (at least in
the WebUI). Is this on purpose?
2. After cluster restart, the jobmanager remembers and tries to re-submit
previously failed resubmissions.
This is one is a bit tricky:
I had a batch job running and killed the master. After the new master took
over, job resubmission failed because the HDFS output directory already
existed. After re-starting the whole cluster and removing the HDFS
directory, the new jobmanager re-submitted the previously failed batch job.
3. Upon starting the cluster I get the following warning message "[WARNING]
1 instance(s) of jobmanager are already running", when jps shows no
existing jobmanager process.

Let me know if these are valid issues and I should open corresponding JIRAs
or I'm misunderstanding something :)

Cheers,
-Vasia.


On 25 February 2016 at 17:04, Stephan Ewen <se...@apache.org> wrote:

> Thanks Marton, the issue is quite serious, I agree.
>
> It is a bit tricky to solve, unfortunately. It seems very hard to make
> experience inside the IDE, with Maven, and with SBT smooth.
> Maven/SBT packaging needs "provided" dependencies, while IntelliJ need
> "compile" dependencies.
>
> On Thu, Feb 25, 2016 at 3:47 PM, Márton Balassi <ba...@gmail.com>
> wrote:
>
> > @Stephan on PR #1685. Fair enough, 1.0.1 is fine. We will try to get it
> in
> > soon anyway.
> >
> > Please consider a build inconvenience that I have just reported to both
> the
> > mailing list and JIRA in the meantime. [1]
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-3511
> >
> >
> >
> > On Thu, Feb 25, 2016 at 3:27 PM, Stephan Ewen <se...@apache.org> wrote:
> >
> > > Concerning the Pull Request mentioned by Marton:
> > >
> > > I think it is a transparent bugfix patch. It is not really affecting
> > > behavior that we want users to make assumptions about (assignment of
> keys
> > > to partitions should not be hardwired into applications).
> > > The only affected program I can think of that is affected is the
> > "queryable
> > > state" program (and similar programs), which is using internal API and
> > > behavior that are always subject to change
> > >
> > > As such, I would think it is also okay for to be included in a bugfix
> > > release.
> > > So if the authors don't have the fix ready in time, it should still be
> > fine
> > > to make it part of 1.0.1 (which will probably come quite soon, as usual
> > > after a full release).
> > >
> > >
> > > On Thu, Feb 25, 2016 at 2:42 PM, Ufuk Celebi <uc...@apache.org> wrote:
> > >
> > > > Maybe you can check whether this is a problem, too:
> > > > https://issues.apache.org/jira/browse/FLINK-3501
> > > >
> > > > Under the assumption that no major functionality changes, I will
> > > > continue with the functional checks.
> > > >
> > > > On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <rmetzger@apache.org
> >
> > > > wrote:
> > > > > Damn. I agree that this is a blocker.
> > > > > I use the maven-enforcer-plugin to check for the right maven, but
> the
> > > > build
> > > > > profile that runs the profile is only active during "deploy", not
> > when
> > > > > packaging the binaries.
> > > > > That's why I didn't realize that I build the binaries with the
> wrong
> > > > maven
> > > > > version.
> > > > >
> > > > > I suggest that we keep collecting problems until Friday afternoon
> > > (CET).
> > > > > Then I'll create the next release candidate.
> > > > >
> > > > > I'd also like to address this one:
> > > > > https://issues.apache.org/jira/browse/FLINK-3509
> > > > >
> > > > >
> > > > > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> > > > wrote:
> > > > >
> > > > >> Hi folks,
> > > > >>
> > > > >> I think I found a release blocker.
> > > > >> The flink-dist JAR file contains non-relocated classes of Google
> > Guava
> > > > and
> > > > >> Apache HttpComponents.
> > > > >>
> > > > >> Fabian
> > > > >>
> > > > >> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> > > > >>
> > > > >> > tested the RC on Windows:
> > > > >> >
> > > > >> > - source compiles
> > > > >> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > > >> > - start/stop scripts work in both cygwin and windows CMD
> > > > >> > - ran several examples from batch/streaming/python
> > > > >> > - scripts also work on paths containing spaces
> > > > >> >
> > > > >> >
> > > > >> > On 25.02.2016 12:41, Robert Metzger wrote:
> > > > >> >
> > > > >> >> (I'm removing user@ from the discussion)
> > > > >> >>
> > > > >> >> Thank you for bringing the pull request to my attention
> Marton. I
> > > > have
> > > > >> to
> > > > >> >> admit that I didn't announce this RC properly in advance. In
> the
> > > RC0
> > > > >> >> thread
> > > > >> >> I said "early next week" and now its Thursday. I should have
> said
> > > > >> >> something
> > > > >> >> in that thread.
> > > > >> >> The "trigger" for creating the release was that the number of
> > > > blocking
> > > > >> >> issues is 0 now.
> > > > >> >>
> > > > >> >> I did a quick check of the open pull requests yesterday evening
> > and
> > > > >> found
> > > > >> >> one [1] to be included into the RC as well. Since the PR you
> > > > mentioned
> > > > >> is
> > > > >> >> marked with [WIP] I thought its not yet ready to be merged.
> > > > >> >>
> > > > >> >> I would like to find a solution that works for everyone here: I
> > > would
> > > > >> like
> > > > >> >> to avoid delaying the release until tomorrow evening, and also
> > the
> > > > work
> > > > >> it
> > > > >> >> incurs for me create a release candidate.
> > > > >> >> How about the following: We keep this vote open, test and check
> > the
> > > > >> >> release
> > > > >> >> and you merge the change to master in the meantime.
> > > > >> >> Most likely, the release gets cancelled anyways because we find
> > > > >> something
> > > > >> >> and then the next RC will contain your change.
> > > > >> >>
> > > > >> >> [1] https://github.com/apache/flink/pull/1706
> > > > >> >>
> > > > >> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > > > >> >> balassi.marton@gmail.com>
> > > > >> >> wrote:
> > > > >> >>
> > > > >> >> Thanks for creating the candidate Robert and for the heads-up,
> > > Slim.
> > > > >> >>>
> > > > >> >>> I would like to get a PR [1] in before 1.0.0 as it breaks
> > hashing
> > > > >> >>> behavior
> > > > >> >>> of DataStream.keyBy. The PR has the feature implemented and
> the
> > > java
> > > > >> >>> tests
> > > > >> >>> adopted, there is still a bit of outstanding fix for the scala
> > > > tests.
> > > > >> >>> Gábor
> > > > >> >>> Horváth or myself will finish it by tomorrow evening.
> > > > >> >>>
> > > > >> >>> [1] https://github.com/apache/flink/pull/1685
> > > > >> >>>
> > > > >> >>> Best,
> > > > >> >>>
> > > > >> >>> Marton
> > > > >> >>>
> > > > >> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <
> > > sbaltagi@gmail.com>
> > > > >> >>> wrote:
> > > > >> >>>
> > > > >> >>> Dear Flink community
> > > > >> >>>>
> > > > >> >>>> It is great news that the vote for the first release
> candidate
> > > > (RC1)
> > > > >> of
> > > > >> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > > > >> >>>> As a community, we need to double our efforts and make sure
> > that
> > > > Flink
> > > > >> >>>> 1.0.0 is GA before these 2 upcoming major events:
> > > > >> >>>>
> > > > >> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31,
> 2016*
> > > > >> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > > > >> >>>>
> > > > >> >>>> This is one aspect of the ‘market dynamics’ that we need to
> > take
> > > > into
> > > > >> >>>> account as a community.
> > > > >> >>>>
> > > > >> >>>> Good luck!
> > > > >> >>>>
> > > > >> >>>> Slim Baltagi
> > > > >> >>>>
> > > > >> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <
> > rmetzger@apache.org
> > > >
> > > > >> >>>> wrote:
> > > > >> >>>>
> > > > >> >>>> Dear Flink community,
> > > > >> >>>>
> > > > >> >>>> Please vote on releasing the following candidate as Apache
> > Flink
> > > > >> version
> > > > >> >>>> 1.0.0.
> > > > >> >>>>
> > > > >> >>>> I've set user@flink.apache.org on CC because users are
> > > encouraged
> > > > to
> > > > >> >>>> help testing Flink 1.0.0 for their specific use cases. Please
> > > > report
> > > > >> >>>> issues
> > > > >> >>>> (and successful tests!) on dev@flink.apache.org.
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >>>> The commit to be voted on (
> > > > >> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6
> )
> > > > >> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > > > >> >>>>
> > > > >> >>>> Branch:
> > > > >> >>>> release-1.0.0-rc1 (see
> > > > >> >>>>
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > > > >> >>>> )
> > > > >> >>>>
> > > > >> >>>> The release artifacts to be voted on can be found at:
> > > > >> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > > > >> >>>>
> > > > >> >>>> The release artifacts are signed with the key with
> fingerprint
> > > > >> D9839159:
> > > > >> >>>> http://www.apache.org/dist/flink/KEYS
> > > > >> >>>>
> > > > >> >>>> The staging repository for this release can be found at:
> > > > >> >>>>
> > > > >>
> > > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > > > >> >>>>
> > > > >> >>>> -------------------------------------------------------------
> > > > >> >>>>
> > > > >> >>>> The vote is open until Tuesday and passes if a majority of at
> > > least
> > > > >> >>>> three
> > > > >> >>>> +1 PMC votes are cast.
> > > > >> >>>>
> > > > >> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > > > >> >>>>
> > > > >> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > > > >> >>>> [ ] -1 Do not release this package because ...
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >
> > > > >>
> > > >
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Stephan Ewen <se...@apache.org>.
Thanks Marton, the issue is quite serious, I agree.

It is a bit tricky to solve, unfortunately. It seems very hard to make
experience inside the IDE, with Maven, and with SBT smooth.
Maven/SBT packaging needs "provided" dependencies, while IntelliJ need
"compile" dependencies.

On Thu, Feb 25, 2016 at 3:47 PM, Márton Balassi <ba...@gmail.com>
wrote:

> @Stephan on PR #1685. Fair enough, 1.0.1 is fine. We will try to get it in
> soon anyway.
>
> Please consider a build inconvenience that I have just reported to both the
> mailing list and JIRA in the meantime. [1]
>
> [1] https://issues.apache.org/jira/browse/FLINK-3511
>
>
>
> On Thu, Feb 25, 2016 at 3:27 PM, Stephan Ewen <se...@apache.org> wrote:
>
> > Concerning the Pull Request mentioned by Marton:
> >
> > I think it is a transparent bugfix patch. It is not really affecting
> > behavior that we want users to make assumptions about (assignment of keys
> > to partitions should not be hardwired into applications).
> > The only affected program I can think of that is affected is the
> "queryable
> > state" program (and similar programs), which is using internal API and
> > behavior that are always subject to change
> >
> > As such, I would think it is also okay for to be included in a bugfix
> > release.
> > So if the authors don't have the fix ready in time, it should still be
> fine
> > to make it part of 1.0.1 (which will probably come quite soon, as usual
> > after a full release).
> >
> >
> > On Thu, Feb 25, 2016 at 2:42 PM, Ufuk Celebi <uc...@apache.org> wrote:
> >
> > > Maybe you can check whether this is a problem, too:
> > > https://issues.apache.org/jira/browse/FLINK-3501
> > >
> > > Under the assumption that no major functionality changes, I will
> > > continue with the functional checks.
> > >
> > > On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <rm...@apache.org>
> > > wrote:
> > > > Damn. I agree that this is a blocker.
> > > > I use the maven-enforcer-plugin to check for the right maven, but the
> > > build
> > > > profile that runs the profile is only active during "deploy", not
> when
> > > > packaging the binaries.
> > > > That's why I didn't realize that I build the binaries with the wrong
> > > maven
> > > > version.
> > > >
> > > > I suggest that we keep collecting problems until Friday afternoon
> > (CET).
> > > > Then I'll create the next release candidate.
> > > >
> > > > I'd also like to address this one:
> > > > https://issues.apache.org/jira/browse/FLINK-3509
> > > >
> > > >
> > > > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> > > wrote:
> > > >
> > > >> Hi folks,
> > > >>
> > > >> I think I found a release blocker.
> > > >> The flink-dist JAR file contains non-relocated classes of Google
> Guava
> > > and
> > > >> Apache HttpComponents.
> > > >>
> > > >> Fabian
> > > >>
> > > >> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> > > >>
> > > >> > tested the RC on Windows:
> > > >> >
> > > >> > - source compiles
> > > >> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > >> > - start/stop scripts work in both cygwin and windows CMD
> > > >> > - ran several examples from batch/streaming/python
> > > >> > - scripts also work on paths containing spaces
> > > >> >
> > > >> >
> > > >> > On 25.02.2016 12:41, Robert Metzger wrote:
> > > >> >
> > > >> >> (I'm removing user@ from the discussion)
> > > >> >>
> > > >> >> Thank you for bringing the pull request to my attention Marton. I
> > > have
> > > >> to
> > > >> >> admit that I didn't announce this RC properly in advance. In the
> > RC0
> > > >> >> thread
> > > >> >> I said "early next week" and now its Thursday. I should have said
> > > >> >> something
> > > >> >> in that thread.
> > > >> >> The "trigger" for creating the release was that the number of
> > > blocking
> > > >> >> issues is 0 now.
> > > >> >>
> > > >> >> I did a quick check of the open pull requests yesterday evening
> and
> > > >> found
> > > >> >> one [1] to be included into the RC as well. Since the PR you
> > > mentioned
> > > >> is
> > > >> >> marked with [WIP] I thought its not yet ready to be merged.
> > > >> >>
> > > >> >> I would like to find a solution that works for everyone here: I
> > would
> > > >> like
> > > >> >> to avoid delaying the release until tomorrow evening, and also
> the
> > > work
> > > >> it
> > > >> >> incurs for me create a release candidate.
> > > >> >> How about the following: We keep this vote open, test and check
> the
> > > >> >> release
> > > >> >> and you merge the change to master in the meantime.
> > > >> >> Most likely, the release gets cancelled anyways because we find
> > > >> something
> > > >> >> and then the next RC will contain your change.
> > > >> >>
> > > >> >> [1] https://github.com/apache/flink/pull/1706
> > > >> >>
> > > >> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > > >> >> balassi.marton@gmail.com>
> > > >> >> wrote:
> > > >> >>
> > > >> >> Thanks for creating the candidate Robert and for the heads-up,
> > Slim.
> > > >> >>>
> > > >> >>> I would like to get a PR [1] in before 1.0.0 as it breaks
> hashing
> > > >> >>> behavior
> > > >> >>> of DataStream.keyBy. The PR has the feature implemented and the
> > java
> > > >> >>> tests
> > > >> >>> adopted, there is still a bit of outstanding fix for the scala
> > > tests.
> > > >> >>> Gábor
> > > >> >>> Horváth or myself will finish it by tomorrow evening.
> > > >> >>>
> > > >> >>> [1] https://github.com/apache/flink/pull/1685
> > > >> >>>
> > > >> >>> Best,
> > > >> >>>
> > > >> >>> Marton
> > > >> >>>
> > > >> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <
> > sbaltagi@gmail.com>
> > > >> >>> wrote:
> > > >> >>>
> > > >> >>> Dear Flink community
> > > >> >>>>
> > > >> >>>> It is great news that the vote for the first release candidate
> > > (RC1)
> > > >> of
> > > >> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > > >> >>>> As a community, we need to double our efforts and make sure
> that
> > > Flink
> > > >> >>>> 1.0.0 is GA before these 2 upcoming major events:
> > > >> >>>>
> > > >> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > > >> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > > >> >>>>
> > > >> >>>> This is one aspect of the ‘market dynamics’ that we need to
> take
> > > into
> > > >> >>>> account as a community.
> > > >> >>>>
> > > >> >>>> Good luck!
> > > >> >>>>
> > > >> >>>> Slim Baltagi
> > > >> >>>>
> > > >> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <
> rmetzger@apache.org
> > >
> > > >> >>>> wrote:
> > > >> >>>>
> > > >> >>>> Dear Flink community,
> > > >> >>>>
> > > >> >>>> Please vote on releasing the following candidate as Apache
> Flink
> > > >> version
> > > >> >>>> 1.0.0.
> > > >> >>>>
> > > >> >>>> I've set user@flink.apache.org on CC because users are
> > encouraged
> > > to
> > > >> >>>> help testing Flink 1.0.0 for their specific use cases. Please
> > > report
> > > >> >>>> issues
> > > >> >>>> (and successful tests!) on dev@flink.apache.org.
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> The commit to be voted on (
> > > >> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > > >> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > > >> >>>>
> > > >> >>>> Branch:
> > > >> >>>> release-1.0.0-rc1 (see
> > > >> >>>>
> > > >> >>>>
> > > >>
> > >
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > > >> >>>> )
> > > >> >>>>
> > > >> >>>> The release artifacts to be voted on can be found at:
> > > >> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > > >> >>>>
> > > >> >>>> The release artifacts are signed with the key with fingerprint
> > > >> D9839159:
> > > >> >>>> http://www.apache.org/dist/flink/KEYS
> > > >> >>>>
> > > >> >>>> The staging repository for this release can be found at:
> > > >> >>>>
> > > >>
> > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > > >> >>>>
> > > >> >>>> -------------------------------------------------------------
> > > >> >>>>
> > > >> >>>> The vote is open until Tuesday and passes if a majority of at
> > least
> > > >> >>>> three
> > > >> >>>> +1 PMC votes are cast.
> > > >> >>>>
> > > >> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > > >> >>>>
> > > >> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > > >> >>>> [ ] -1 Do not release this package because ...
> > > >> >>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >
> > > >>
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Márton Balassi <ba...@gmail.com>.
@Stephan on PR #1685. Fair enough, 1.0.1 is fine. We will try to get it in
soon anyway.

Please consider a build inconvenience that I have just reported to both the
mailing list and JIRA in the meantime. [1]

[1] https://issues.apache.org/jira/browse/FLINK-3511



On Thu, Feb 25, 2016 at 3:27 PM, Stephan Ewen <se...@apache.org> wrote:

> Concerning the Pull Request mentioned by Marton:
>
> I think it is a transparent bugfix patch. It is not really affecting
> behavior that we want users to make assumptions about (assignment of keys
> to partitions should not be hardwired into applications).
> The only affected program I can think of that is affected is the "queryable
> state" program (and similar programs), which is using internal API and
> behavior that are always subject to change
>
> As such, I would think it is also okay for to be included in a bugfix
> release.
> So if the authors don't have the fix ready in time, it should still be fine
> to make it part of 1.0.1 (which will probably come quite soon, as usual
> after a full release).
>
>
> On Thu, Feb 25, 2016 at 2:42 PM, Ufuk Celebi <uc...@apache.org> wrote:
>
> > Maybe you can check whether this is a problem, too:
> > https://issues.apache.org/jira/browse/FLINK-3501
> >
> > Under the assumption that no major functionality changes, I will
> > continue with the functional checks.
> >
> > On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <rm...@apache.org>
> > wrote:
> > > Damn. I agree that this is a blocker.
> > > I use the maven-enforcer-plugin to check for the right maven, but the
> > build
> > > profile that runs the profile is only active during "deploy", not when
> > > packaging the binaries.
> > > That's why I didn't realize that I build the binaries with the wrong
> > maven
> > > version.
> > >
> > > I suggest that we keep collecting problems until Friday afternoon
> (CET).
> > > Then I'll create the next release candidate.
> > >
> > > I'd also like to address this one:
> > > https://issues.apache.org/jira/browse/FLINK-3509
> > >
> > >
> > > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> > wrote:
> > >
> > >> Hi folks,
> > >>
> > >> I think I found a release blocker.
> > >> The flink-dist JAR file contains non-relocated classes of Google Guava
> > and
> > >> Apache HttpComponents.
> > >>
> > >> Fabian
> > >>
> > >> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> > >>
> > >> > tested the RC on Windows:
> > >> >
> > >> > - source compiles
> > >> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > >> > - start/stop scripts work in both cygwin and windows CMD
> > >> > - ran several examples from batch/streaming/python
> > >> > - scripts also work on paths containing spaces
> > >> >
> > >> >
> > >> > On 25.02.2016 12:41, Robert Metzger wrote:
> > >> >
> > >> >> (I'm removing user@ from the discussion)
> > >> >>
> > >> >> Thank you for bringing the pull request to my attention Marton. I
> > have
> > >> to
> > >> >> admit that I didn't announce this RC properly in advance. In the
> RC0
> > >> >> thread
> > >> >> I said "early next week" and now its Thursday. I should have said
> > >> >> something
> > >> >> in that thread.
> > >> >> The "trigger" for creating the release was that the number of
> > blocking
> > >> >> issues is 0 now.
> > >> >>
> > >> >> I did a quick check of the open pull requests yesterday evening and
> > >> found
> > >> >> one [1] to be included into the RC as well. Since the PR you
> > mentioned
> > >> is
> > >> >> marked with [WIP] I thought its not yet ready to be merged.
> > >> >>
> > >> >> I would like to find a solution that works for everyone here: I
> would
> > >> like
> > >> >> to avoid delaying the release until tomorrow evening, and also the
> > work
> > >> it
> > >> >> incurs for me create a release candidate.
> > >> >> How about the following: We keep this vote open, test and check the
> > >> >> release
> > >> >> and you merge the change to master in the meantime.
> > >> >> Most likely, the release gets cancelled anyways because we find
> > >> something
> > >> >> and then the next RC will contain your change.
> > >> >>
> > >> >> [1] https://github.com/apache/flink/pull/1706
> > >> >>
> > >> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > >> >> balassi.marton@gmail.com>
> > >> >> wrote:
> > >> >>
> > >> >> Thanks for creating the candidate Robert and for the heads-up,
> Slim.
> > >> >>>
> > >> >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> > >> >>> behavior
> > >> >>> of DataStream.keyBy. The PR has the feature implemented and the
> java
> > >> >>> tests
> > >> >>> adopted, there is still a bit of outstanding fix for the scala
> > tests.
> > >> >>> Gábor
> > >> >>> Horváth or myself will finish it by tomorrow evening.
> > >> >>>
> > >> >>> [1] https://github.com/apache/flink/pull/1685
> > >> >>>
> > >> >>> Best,
> > >> >>>
> > >> >>> Marton
> > >> >>>
> > >> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <
> sbaltagi@gmail.com>
> > >> >>> wrote:
> > >> >>>
> > >> >>> Dear Flink community
> > >> >>>>
> > >> >>>> It is great news that the vote for the first release candidate
> > (RC1)
> > >> of
> > >> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > >> >>>> As a community, we need to double our efforts and make sure that
> > Flink
> > >> >>>> 1.0.0 is GA before these 2 upcoming major events:
> > >> >>>>
> > >> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > >> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > >> >>>>
> > >> >>>> This is one aspect of the ‘market dynamics’ that we need to take
> > into
> > >> >>>> account as a community.
> > >> >>>>
> > >> >>>> Good luck!
> > >> >>>>
> > >> >>>> Slim Baltagi
> > >> >>>>
> > >> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rmetzger@apache.org
> >
> > >> >>>> wrote:
> > >> >>>>
> > >> >>>> Dear Flink community,
> > >> >>>>
> > >> >>>> Please vote on releasing the following candidate as Apache Flink
> > >> version
> > >> >>>> 1.0.0.
> > >> >>>>
> > >> >>>> I've set user@flink.apache.org on CC because users are
> encouraged
> > to
> > >> >>>> help testing Flink 1.0.0 for their specific use cases. Please
> > report
> > >> >>>> issues
> > >> >>>> (and successful tests!) on dev@flink.apache.org.
> > >> >>>>
> > >> >>>>
> > >> >>>> The commit to be voted on (
> > >> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > >> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > >> >>>>
> > >> >>>> Branch:
> > >> >>>> release-1.0.0-rc1 (see
> > >> >>>>
> > >> >>>>
> > >>
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > >> >>>> )
> > >> >>>>
> > >> >>>> The release artifacts to be voted on can be found at:
> > >> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > >> >>>>
> > >> >>>> The release artifacts are signed with the key with fingerprint
> > >> D9839159:
> > >> >>>> http://www.apache.org/dist/flink/KEYS
> > >> >>>>
> > >> >>>> The staging repository for this release can be found at:
> > >> >>>>
> > >>
> https://repository.apache.org/content/repositories/orgapacheflink-1063
> > >> >>>>
> > >> >>>> -------------------------------------------------------------
> > >> >>>>
> > >> >>>> The vote is open until Tuesday and passes if a majority of at
> least
> > >> >>>> three
> > >> >>>> +1 PMC votes are cast.
> > >> >>>>
> > >> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > >> >>>>
> > >> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > >> >>>> [ ] -1 Do not release this package because ...
> > >> >>>>
> > >> >>>>
> > >> >>>>
> > >> >>>>
> > >> >
> > >>
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Stephan Ewen <se...@apache.org>.
Concerning the Pull Request mentioned by Marton:

I think it is a transparent bugfix patch. It is not really affecting
behavior that we want users to make assumptions about (assignment of keys
to partitions should not be hardwired into applications).
The only affected program I can think of that is affected is the "queryable
state" program (and similar programs), which is using internal API and
behavior that are always subject to change

As such, I would think it is also okay for to be included in a bugfix
release.
So if the authors don't have the fix ready in time, it should still be fine
to make it part of 1.0.1 (which will probably come quite soon, as usual
after a full release).


On Thu, Feb 25, 2016 at 2:42 PM, Ufuk Celebi <uc...@apache.org> wrote:

> Maybe you can check whether this is a problem, too:
> https://issues.apache.org/jira/browse/FLINK-3501
>
> Under the assumption that no major functionality changes, I will
> continue with the functional checks.
>
> On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <rm...@apache.org>
> wrote:
> > Damn. I agree that this is a blocker.
> > I use the maven-enforcer-plugin to check for the right maven, but the
> build
> > profile that runs the profile is only active during "deploy", not when
> > packaging the binaries.
> > That's why I didn't realize that I build the binaries with the wrong
> maven
> > version.
> >
> > I suggest that we keep collecting problems until Friday afternoon (CET).
> > Then I'll create the next release candidate.
> >
> > I'd also like to address this one:
> > https://issues.apache.org/jira/browse/FLINK-3509
> >
> >
> > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> wrote:
> >
> >> Hi folks,
> >>
> >> I think I found a release blocker.
> >> The flink-dist JAR file contains non-relocated classes of Google Guava
> and
> >> Apache HttpComponents.
> >>
> >> Fabian
> >>
> >> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> >>
> >> > tested the RC on Windows:
> >> >
> >> > - source compiles
> >> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> >> > - start/stop scripts work in both cygwin and windows CMD
> >> > - ran several examples from batch/streaming/python
> >> > - scripts also work on paths containing spaces
> >> >
> >> >
> >> > On 25.02.2016 12:41, Robert Metzger wrote:
> >> >
> >> >> (I'm removing user@ from the discussion)
> >> >>
> >> >> Thank you for bringing the pull request to my attention Marton. I
> have
> >> to
> >> >> admit that I didn't announce this RC properly in advance. In the RC0
> >> >> thread
> >> >> I said "early next week" and now its Thursday. I should have said
> >> >> something
> >> >> in that thread.
> >> >> The "trigger" for creating the release was that the number of
> blocking
> >> >> issues is 0 now.
> >> >>
> >> >> I did a quick check of the open pull requests yesterday evening and
> >> found
> >> >> one [1] to be included into the RC as well. Since the PR you
> mentioned
> >> is
> >> >> marked with [WIP] I thought its not yet ready to be merged.
> >> >>
> >> >> I would like to find a solution that works for everyone here: I would
> >> like
> >> >> to avoid delaying the release until tomorrow evening, and also the
> work
> >> it
> >> >> incurs for me create a release candidate.
> >> >> How about the following: We keep this vote open, test and check the
> >> >> release
> >> >> and you merge the change to master in the meantime.
> >> >> Most likely, the release gets cancelled anyways because we find
> >> something
> >> >> and then the next RC will contain your change.
> >> >>
> >> >> [1] https://github.com/apache/flink/pull/1706
> >> >>
> >> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> >> >> balassi.marton@gmail.com>
> >> >> wrote:
> >> >>
> >> >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> >> >>>
> >> >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> >> >>> behavior
> >> >>> of DataStream.keyBy. The PR has the feature implemented and the java
> >> >>> tests
> >> >>> adopted, there is still a bit of outstanding fix for the scala
> tests.
> >> >>> Gábor
> >> >>> Horváth or myself will finish it by tomorrow evening.
> >> >>>
> >> >>> [1] https://github.com/apache/flink/pull/1685
> >> >>>
> >> >>> Best,
> >> >>>
> >> >>> Marton
> >> >>>
> >> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
> >> >>> wrote:
> >> >>>
> >> >>> Dear Flink community
> >> >>>>
> >> >>>> It is great news that the vote for the first release candidate
> (RC1)
> >> of
> >> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> >> >>>> As a community, we need to double our efforts and make sure that
> Flink
> >> >>>> 1.0.0 is GA before these 2 upcoming major events:
> >> >>>>
> >> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> >> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> >> >>>>
> >> >>>> This is one aspect of the ‘market dynamics’ that we need to take
> into
> >> >>>> account as a community.
> >> >>>>
> >> >>>> Good luck!
> >> >>>>
> >> >>>> Slim Baltagi
> >> >>>>
> >> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> >> >>>> wrote:
> >> >>>>
> >> >>>> Dear Flink community,
> >> >>>>
> >> >>>> Please vote on releasing the following candidate as Apache Flink
> >> version
> >> >>>> 1.0.0.
> >> >>>>
> >> >>>> I've set user@flink.apache.org on CC because users are encouraged
> to
> >> >>>> help testing Flink 1.0.0 for their specific use cases. Please
> report
> >> >>>> issues
> >> >>>> (and successful tests!) on dev@flink.apache.org.
> >> >>>>
> >> >>>>
> >> >>>> The commit to be voted on (
> >> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> >> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> >> >>>>
> >> >>>> Branch:
> >> >>>> release-1.0.0-rc1 (see
> >> >>>>
> >> >>>>
> >>
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> >> >>>> )
> >> >>>>
> >> >>>> The release artifacts to be voted on can be found at:
> >> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> >> >>>>
> >> >>>> The release artifacts are signed with the key with fingerprint
> >> D9839159:
> >> >>>> http://www.apache.org/dist/flink/KEYS
> >> >>>>
> >> >>>> The staging repository for this release can be found at:
> >> >>>>
> >> https://repository.apache.org/content/repositories/orgapacheflink-1063
> >> >>>>
> >> >>>> -------------------------------------------------------------
> >> >>>>
> >> >>>> The vote is open until Tuesday and passes if a majority of at least
> >> >>>> three
> >> >>>> +1 PMC votes are cast.
> >> >>>>
> >> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> >> >>>>
> >> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> >> >>>> [ ] -1 Do not release this package because ...
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >
> >>
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Ufuk Celebi <uc...@apache.org>.
Maybe you can check whether this is a problem, too:
https://issues.apache.org/jira/browse/FLINK-3501

Under the assumption that no major functionality changes, I will
continue with the functional checks.

On Thu, Feb 25, 2016 at 2:32 PM, Robert Metzger <rm...@apache.org> wrote:
> Damn. I agree that this is a blocker.
> I use the maven-enforcer-plugin to check for the right maven, but the build
> profile that runs the profile is only active during "deploy", not when
> packaging the binaries.
> That's why I didn't realize that I build the binaries with the wrong maven
> version.
>
> I suggest that we keep collecting problems until Friday afternoon (CET).
> Then I'll create the next release candidate.
>
> I'd also like to address this one:
> https://issues.apache.org/jira/browse/FLINK-3509
>
>
> On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com> wrote:
>
>> Hi folks,
>>
>> I think I found a release blocker.
>> The flink-dist JAR file contains non-relocated classes of Google Guava and
>> Apache HttpComponents.
>>
>> Fabian
>>
>> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
>>
>> > tested the RC on Windows:
>> >
>> > - source compiles
>> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
>> > - start/stop scripts work in both cygwin and windows CMD
>> > - ran several examples from batch/streaming/python
>> > - scripts also work on paths containing spaces
>> >
>> >
>> > On 25.02.2016 12:41, Robert Metzger wrote:
>> >
>> >> (I'm removing user@ from the discussion)
>> >>
>> >> Thank you for bringing the pull request to my attention Marton. I have
>> to
>> >> admit that I didn't announce this RC properly in advance. In the RC0
>> >> thread
>> >> I said "early next week" and now its Thursday. I should have said
>> >> something
>> >> in that thread.
>> >> The "trigger" for creating the release was that the number of blocking
>> >> issues is 0 now.
>> >>
>> >> I did a quick check of the open pull requests yesterday evening and
>> found
>> >> one [1] to be included into the RC as well. Since the PR you mentioned
>> is
>> >> marked with [WIP] I thought its not yet ready to be merged.
>> >>
>> >> I would like to find a solution that works for everyone here: I would
>> like
>> >> to avoid delaying the release until tomorrow evening, and also the work
>> it
>> >> incurs for me create a release candidate.
>> >> How about the following: We keep this vote open, test and check the
>> >> release
>> >> and you merge the change to master in the meantime.
>> >> Most likely, the release gets cancelled anyways because we find
>> something
>> >> and then the next RC will contain your change.
>> >>
>> >> [1] https://github.com/apache/flink/pull/1706
>> >>
>> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
>> >> balassi.marton@gmail.com>
>> >> wrote:
>> >>
>> >> Thanks for creating the candidate Robert and for the heads-up, Slim.
>> >>>
>> >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
>> >>> behavior
>> >>> of DataStream.keyBy. The PR has the feature implemented and the java
>> >>> tests
>> >>> adopted, there is still a bit of outstanding fix for the scala tests.
>> >>> Gábor
>> >>> Horváth or myself will finish it by tomorrow evening.
>> >>>
>> >>> [1] https://github.com/apache/flink/pull/1685
>> >>>
>> >>> Best,
>> >>>
>> >>> Marton
>> >>>
>> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
>> >>> wrote:
>> >>>
>> >>> Dear Flink community
>> >>>>
>> >>>> It is great news that the vote for the first release candidate (RC1)
>> of
>> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
>> >>>> As a community, we need to double our efforts and make sure that Flink
>> >>>> 1.0.0 is GA before these 2 upcoming major events:
>> >>>>
>> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>> >>>>
>> >>>> This is one aspect of the ‘market dynamics’ that we need to take into
>> >>>> account as a community.
>> >>>>
>> >>>> Good luck!
>> >>>>
>> >>>> Slim Baltagi
>> >>>>
>> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
>> >>>> wrote:
>> >>>>
>> >>>> Dear Flink community,
>> >>>>
>> >>>> Please vote on releasing the following candidate as Apache Flink
>> version
>> >>>> 1.0.0.
>> >>>>
>> >>>> I've set user@flink.apache.org on CC because users are encouraged to
>> >>>> help testing Flink 1.0.0 for their specific use cases. Please report
>> >>>> issues
>> >>>> (and successful tests!) on dev@flink.apache.org.
>> >>>>
>> >>>>
>> >>>> The commit to be voted on (
>> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>> >>>>
>> >>>> Branch:
>> >>>> release-1.0.0-rc1 (see
>> >>>>
>> >>>>
>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
>> >>>> )
>> >>>>
>> >>>> The release artifacts to be voted on can be found at:
>> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>> >>>>
>> >>>> The release artifacts are signed with the key with fingerprint
>> D9839159:
>> >>>> http://www.apache.org/dist/flink/KEYS
>> >>>>
>> >>>> The staging repository for this release can be found at:
>> >>>>
>> https://repository.apache.org/content/repositories/orgapacheflink-1063
>> >>>>
>> >>>> -------------------------------------------------------------
>> >>>>
>> >>>> The vote is open until Tuesday and passes if a majority of at least
>> >>>> three
>> >>>> +1 PMC votes are cast.
>> >>>>
>> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
>> >>>>
>> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
>> >>>> [ ] -1 Do not release this package because ...
>> >>>>
>> >>>>
>> >>>>
>> >>>>
>> >
>>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Robert Metzger <rm...@apache.org>.
@Greg: I've merged the two PRs you've mentioned.

Since nobody objected, I'll create now a branch for "release-1.0" and I'll
update the version in master to 1.1-SNAPSHOT.

Please remember that I'm aiming for creating the next RC later today. I
keep an eye on blocking issues / open pull requests. If there's nothing
I'll start creating the RC.

On Thu, Feb 25, 2016 at 8:10 PM, Stephan Ewen <se...@apache.org> wrote:

> Just looked at Greg's object juggle PR - looks good for inclusion in the
> next release candidate.
>
> Have not tested the web UI Router fix, but the code looks good.
> Hi,
>
> I have two bugfix pull requests in the stack.
>
> [FLINK-3340] [runtime] Fix object juggling in drivers
>   https://github.com/apache/flink/pull/1626
>
> [FLINK-3437] [web-dashboard] Fix UI router state for job plan
>   https://github.com/apache/flink/pull/1661
>
> Greg
>
> On Thu, Feb 25, 2016 at 8:32 AM, Robert Metzger <rm...@apache.org>
> wrote:
>
> > Damn. I agree that this is a blocker.
> > I use the maven-enforcer-plugin to check for the right maven, but the
> build
> > profile that runs the profile is only active during "deploy", not when
> > packaging the binaries.
> > That's why I didn't realize that I build the binaries with the wrong
> maven
> > version.
> >
> > I suggest that we keep collecting problems until Friday afternoon (CET).
> > Then I'll create the next release candidate.
> >
> > I'd also like to address this one:
> > https://issues.apache.org/jira/browse/FLINK-3509
> >
> >
> > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> wrote:
> >
> > > Hi folks,
> > >
> > > I think I found a release blocker.
> > > The flink-dist JAR file contains non-relocated classes of Google Guava
> > and
> > > Apache HttpComponents.
> > >
> > > Fabian
> > >
> > > 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> > >
> > > > tested the RC on Windows:
> > > >
> > > > - source compiles
> > > > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > > - start/stop scripts work in both cygwin and windows CMD
> > > > - ran several examples from batch/streaming/python
> > > > - scripts also work on paths containing spaces
> > > >
> > > >
> > > > On 25.02.2016 12:41, Robert Metzger wrote:
> > > >
> > > >> (I'm removing user@ from the discussion)
> > > >>
> > > >> Thank you for bringing the pull request to my attention Marton. I
> have
> > > to
> > > >> admit that I didn't announce this RC properly in advance. In the RC0
> > > >> thread
> > > >> I said "early next week" and now its Thursday. I should have said
> > > >> something
> > > >> in that thread.
> > > >> The "trigger" for creating the release was that the number of
> blocking
> > > >> issues is 0 now.
> > > >>
> > > >> I did a quick check of the open pull requests yesterday evening and
> > > found
> > > >> one [1] to be included into the RC as well. Since the PR you
> mentioned
> > > is
> > > >> marked with [WIP] I thought its not yet ready to be merged.
> > > >>
> > > >> I would like to find a solution that works for everyone here: I
> would
> > > like
> > > >> to avoid delaying the release until tomorrow evening, and also the
> > work
> > > it
> > > >> incurs for me create a release candidate.
> > > >> How about the following: We keep this vote open, test and check the
> > > >> release
> > > >> and you merge the change to master in the meantime.
> > > >> Most likely, the release gets cancelled anyways because we find
> > > something
> > > >> and then the next RC will contain your change.
> > > >>
> > > >> [1] https://github.com/apache/flink/pull/1706
> > > >>
> > > >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > > >> balassi.marton@gmail.com>
> > > >> wrote:
> > > >>
> > > >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> > > >>>
> > > >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> > > >>> behavior
> > > >>> of DataStream.keyBy. The PR has the feature implemented and the
> java
> > > >>> tests
> > > >>> adopted, there is still a bit of outstanding fix for the scala
> tests.
> > > >>> Gábor
> > > >>> Horváth or myself will finish it by tomorrow evening.
> > > >>>
> > > >>> [1] https://github.com/apache/flink/pull/1685
> > > >>>
> > > >>> Best,
> > > >>>
> > > >>> Marton
> > > >>>
> > > >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sbaltagi@gmail.com
> >
> > > >>> wrote:
> > > >>>
> > > >>> Dear Flink community
> > > >>>>
> > > >>>> It is great news that the vote for the first release candidate
> (RC1)
> > > of
> > > >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > > >>>> As a community, we need to double our efforts and make sure that
> > Flink
> > > >>>> 1.0.0 is GA before these 2 upcoming major events:
> > > >>>>
> > > >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > > >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > > >>>>
> > > >>>> This is one aspect of the ‘market dynamics’ that we need to take
> > into
> > > >>>> account as a community.
> > > >>>>
> > > >>>> Good luck!
> > > >>>>
> > > >>>> Slim Baltagi
> > > >>>>
> > > >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> > > >>>> wrote:
> > > >>>>
> > > >>>> Dear Flink community,
> > > >>>>
> > > >>>> Please vote on releasing the following candidate as Apache Flink
> > > version
> > > >>>> 1.0.0.
> > > >>>>
> > > >>>> I've set user@flink.apache.org on CC because users are encouraged
> > to
> > > >>>> help testing Flink 1.0.0 for their specific use cases. Please
> report
> > > >>>> issues
> > > >>>> (and successful tests!) on dev@flink.apache.org.
> > > >>>>
> > > >>>>
> > > >>>> The commit to be voted on (
> > > >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > > >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > > >>>>
> > > >>>> Branch:
> > > >>>> release-1.0.0-rc1 (see
> > > >>>>
> > > >>>>
> > >
> >
>
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > > >>>> )
> > > >>>>
> > > >>>> The release artifacts to be voted on can be found at:
> > > >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > > >>>>
> > > >>>> The release artifacts are signed with the key with fingerprint
> > > D9839159:
> > > >>>> http://www.apache.org/dist/flink/KEYS
> > > >>>>
> > > >>>> The staging repository for this release can be found at:
> > > >>>>
> > > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > > >>>>
> > > >>>> -------------------------------------------------------------
> > > >>>>
> > > >>>> The vote is open until Tuesday and passes if a majority of at
> least
> > > >>>> three
> > > >>>> +1 PMC votes are cast.
> > > >>>>
> > > >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > > >>>>
> > > >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > > >>>> [ ] -1 Do not release this package because ...
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>>
> > > >
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Stephan Ewen <se...@apache.org>.
Just looked at Greg's object juggle PR - looks good for inclusion in the
next release candidate.

Have not tested the web UI Router fix, but the code looks good.
Hi,

I have two bugfix pull requests in the stack.

[FLINK-3340] [runtime] Fix object juggling in drivers
  https://github.com/apache/flink/pull/1626

[FLINK-3437] [web-dashboard] Fix UI router state for job plan
  https://github.com/apache/flink/pull/1661

Greg

On Thu, Feb 25, 2016 at 8:32 AM, Robert Metzger <rm...@apache.org> wrote:

> Damn. I agree that this is a blocker.
> I use the maven-enforcer-plugin to check for the right maven, but the
build
> profile that runs the profile is only active during "deploy", not when
> packaging the binaries.
> That's why I didn't realize that I build the binaries with the wrong maven
> version.
>
> I suggest that we keep collecting problems until Friday afternoon (CET).
> Then I'll create the next release candidate.
>
> I'd also like to address this one:
> https://issues.apache.org/jira/browse/FLINK-3509
>
>
> On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com> wrote:
>
> > Hi folks,
> >
> > I think I found a release blocker.
> > The flink-dist JAR file contains non-relocated classes of Google Guava
> and
> > Apache HttpComponents.
> >
> > Fabian
> >
> > 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> >
> > > tested the RC on Windows:
> > >
> > > - source compiles
> > > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > - start/stop scripts work in both cygwin and windows CMD
> > > - ran several examples from batch/streaming/python
> > > - scripts also work on paths containing spaces
> > >
> > >
> > > On 25.02.2016 12:41, Robert Metzger wrote:
> > >
> > >> (I'm removing user@ from the discussion)
> > >>
> > >> Thank you for bringing the pull request to my attention Marton. I
have
> > to
> > >> admit that I didn't announce this RC properly in advance. In the RC0
> > >> thread
> > >> I said "early next week" and now its Thursday. I should have said
> > >> something
> > >> in that thread.
> > >> The "trigger" for creating the release was that the number of
blocking
> > >> issues is 0 now.
> > >>
> > >> I did a quick check of the open pull requests yesterday evening and
> > found
> > >> one [1] to be included into the RC as well. Since the PR you
mentioned
> > is
> > >> marked with [WIP] I thought its not yet ready to be merged.
> > >>
> > >> I would like to find a solution that works for everyone here: I would
> > like
> > >> to avoid delaying the release until tomorrow evening, and also the
> work
> > it
> > >> incurs for me create a release candidate.
> > >> How about the following: We keep this vote open, test and check the
> > >> release
> > >> and you merge the change to master in the meantime.
> > >> Most likely, the release gets cancelled anyways because we find
> > something
> > >> and then the next RC will contain your change.
> > >>
> > >> [1] https://github.com/apache/flink/pull/1706
> > >>
> > >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > >> balassi.marton@gmail.com>
> > >> wrote:
> > >>
> > >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> > >>>
> > >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> > >>> behavior
> > >>> of DataStream.keyBy. The PR has the feature implemented and the java
> > >>> tests
> > >>> adopted, there is still a bit of outstanding fix for the scala
tests.
> > >>> Gábor
> > >>> Horváth or myself will finish it by tomorrow evening.
> > >>>
> > >>> [1] https://github.com/apache/flink/pull/1685
> > >>>
> > >>> Best,
> > >>>
> > >>> Marton
> > >>>
> > >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
> > >>> wrote:
> > >>>
> > >>> Dear Flink community
> > >>>>
> > >>>> It is great news that the vote for the first release candidate
(RC1)
> > of
> > >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > >>>> As a community, we need to double our efforts and make sure that
> Flink
> > >>>> 1.0.0 is GA before these 2 upcoming major events:
> > >>>>
> > >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > >>>>
> > >>>> This is one aspect of the ‘market dynamics’ that we need to take
> into
> > >>>> account as a community.
> > >>>>
> > >>>> Good luck!
> > >>>>
> > >>>> Slim Baltagi
> > >>>>
> > >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> > >>>> wrote:
> > >>>>
> > >>>> Dear Flink community,
> > >>>>
> > >>>> Please vote on releasing the following candidate as Apache Flink
> > version
> > >>>> 1.0.0.
> > >>>>
> > >>>> I've set user@flink.apache.org on CC because users are encouraged
> to
> > >>>> help testing Flink 1.0.0 for their specific use cases. Please
report
> > >>>> issues
> > >>>> (and successful tests!) on dev@flink.apache.org.
> > >>>>
> > >>>>
> > >>>> The commit to be voted on (
> > >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > >>>>
> > >>>> Branch:
> > >>>> release-1.0.0-rc1 (see
> > >>>>
> > >>>>
> >
>
https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > >>>> )
> > >>>>
> > >>>> The release artifacts to be voted on can be found at:
> > >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > >>>>
> > >>>> The release artifacts are signed with the key with fingerprint
> > D9839159:
> > >>>> http://www.apache.org/dist/flink/KEYS
> > >>>>
> > >>>> The staging repository for this release can be found at:
> > >>>>
> > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > >>>>
> > >>>> -------------------------------------------------------------
> > >>>>
> > >>>> The vote is open until Tuesday and passes if a majority of at least
> > >>>> three
> > >>>> +1 PMC votes are cast.
> > >>>>
> > >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > >>>>
> > >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > >>>> [ ] -1 Do not release this package because ...
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Ufuk Celebi <uc...@apache.org>.
I think that forking off is the regular way we do the releases and I
don't see any reasons to not do it this time.

– Ufuk


On Thu, Feb 25, 2016 at 7:41 PM, Robert Metzger <rm...@apache.org> wrote:
> Hi,
> I'd like to fork off a branch for the 1.0 release so that we can merge big
> changes into master. Any objections?
>
> On Thu, Feb 25, 2016 at 6:04 PM, Greg Hogan <co...@greghogan.com> wrote:
>
>> Hi,
>>
>> I have two bugfix pull requests in the stack.
>>
>> [FLINK-3340] [runtime] Fix object juggling in drivers
>>   https://github.com/apache/flink/pull/1626
>>
>> [FLINK-3437] [web-dashboard] Fix UI router state for job plan
>>   https://github.com/apache/flink/pull/1661
>>
>> Greg
>>
>> On Thu, Feb 25, 2016 at 8:32 AM, Robert Metzger <rm...@apache.org>
>> wrote:
>>
>> > Damn. I agree that this is a blocker.
>> > I use the maven-enforcer-plugin to check for the right maven, but the
>> build
>> > profile that runs the profile is only active during "deploy", not when
>> > packaging the binaries.
>> > That's why I didn't realize that I build the binaries with the wrong
>> maven
>> > version.
>> >
>> > I suggest that we keep collecting problems until Friday afternoon (CET).
>> > Then I'll create the next release candidate.
>> >
>> > I'd also like to address this one:
>> > https://issues.apache.org/jira/browse/FLINK-3509
>> >
>> >
>> > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
>> wrote:
>> >
>> > > Hi folks,
>> > >
>> > > I think I found a release blocker.
>> > > The flink-dist JAR file contains non-relocated classes of Google Guava
>> > and
>> > > Apache HttpComponents.
>> > >
>> > > Fabian
>> > >
>> > > 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
>> > >
>> > > > tested the RC on Windows:
>> > > >
>> > > > - source compiles
>> > > > - some tests categorically fail: see FLINK-3491 / FLINK-3496
>> > > > - start/stop scripts work in both cygwin and windows CMD
>> > > > - ran several examples from batch/streaming/python
>> > > > - scripts also work on paths containing spaces
>> > > >
>> > > >
>> > > > On 25.02.2016 12:41, Robert Metzger wrote:
>> > > >
>> > > >> (I'm removing user@ from the discussion)
>> > > >>
>> > > >> Thank you for bringing the pull request to my attention Marton. I
>> have
>> > > to
>> > > >> admit that I didn't announce this RC properly in advance. In the RC0
>> > > >> thread
>> > > >> I said "early next week" and now its Thursday. I should have said
>> > > >> something
>> > > >> in that thread.
>> > > >> The "trigger" for creating the release was that the number of
>> blocking
>> > > >> issues is 0 now.
>> > > >>
>> > > >> I did a quick check of the open pull requests yesterday evening and
>> > > found
>> > > >> one [1] to be included into the RC as well. Since the PR you
>> mentioned
>> > > is
>> > > >> marked with [WIP] I thought its not yet ready to be merged.
>> > > >>
>> > > >> I would like to find a solution that works for everyone here: I
>> would
>> > > like
>> > > >> to avoid delaying the release until tomorrow evening, and also the
>> > work
>> > > it
>> > > >> incurs for me create a release candidate.
>> > > >> How about the following: We keep this vote open, test and check the
>> > > >> release
>> > > >> and you merge the change to master in the meantime.
>> > > >> Most likely, the release gets cancelled anyways because we find
>> > > something
>> > > >> and then the next RC will contain your change.
>> > > >>
>> > > >> [1] https://github.com/apache/flink/pull/1706
>> > > >>
>> > > >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
>> > > >> balassi.marton@gmail.com>
>> > > >> wrote:
>> > > >>
>> > > >> Thanks for creating the candidate Robert and for the heads-up, Slim.
>> > > >>>
>> > > >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
>> > > >>> behavior
>> > > >>> of DataStream.keyBy. The PR has the feature implemented and the
>> java
>> > > >>> tests
>> > > >>> adopted, there is still a bit of outstanding fix for the scala
>> tests.
>> > > >>> Gábor
>> > > >>> Horváth or myself will finish it by tomorrow evening.
>> > > >>>
>> > > >>> [1] https://github.com/apache/flink/pull/1685
>> > > >>>
>> > > >>> Best,
>> > > >>>
>> > > >>> Marton
>> > > >>>
>> > > >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sbaltagi@gmail.com
>> >
>> > > >>> wrote:
>> > > >>>
>> > > >>> Dear Flink community
>> > > >>>>
>> > > >>>> It is great news that the vote for the first release candidate
>> (RC1)
>> > > of
>> > > >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
>> > > >>>> As a community, we need to double our efforts and make sure that
>> > Flink
>> > > >>>> 1.0.0 is GA before these 2 upcoming major events:
>> > > >>>>
>> > > >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>> > > >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>> > > >>>>
>> > > >>>> This is one aspect of the ‘market dynamics’ that we need to take
>> > into
>> > > >>>> account as a community.
>> > > >>>>
>> > > >>>> Good luck!
>> > > >>>>
>> > > >>>> Slim Baltagi
>> > > >>>>
>> > > >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
>> > > >>>> wrote:
>> > > >>>>
>> > > >>>> Dear Flink community,
>> > > >>>>
>> > > >>>> Please vote on releasing the following candidate as Apache Flink
>> > > version
>> > > >>>> 1.0.0.
>> > > >>>>
>> > > >>>> I've set user@flink.apache.org on CC because users are encouraged
>> > to
>> > > >>>> help testing Flink 1.0.0 for their specific use cases. Please
>> report
>> > > >>>> issues
>> > > >>>> (and successful tests!) on dev@flink.apache.org.
>> > > >>>>
>> > > >>>>
>> > > >>>> The commit to be voted on (
>> > > >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>> > > >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>> > > >>>>
>> > > >>>> Branch:
>> > > >>>> release-1.0.0-rc1 (see
>> > > >>>>
>> > > >>>>
>> > >
>> >
>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
>> > > >>>> )
>> > > >>>>
>> > > >>>> The release artifacts to be voted on can be found at:
>> > > >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>> > > >>>>
>> > > >>>> The release artifacts are signed with the key with fingerprint
>> > > D9839159:
>> > > >>>> http://www.apache.org/dist/flink/KEYS
>> > > >>>>
>> > > >>>> The staging repository for this release can be found at:
>> > > >>>>
>> > > https://repository.apache.org/content/repositories/orgapacheflink-1063
>> > > >>>>
>> > > >>>> -------------------------------------------------------------
>> > > >>>>
>> > > >>>> The vote is open until Tuesday and passes if a majority of at
>> least
>> > > >>>> three
>> > > >>>> +1 PMC votes are cast.
>> > > >>>>
>> > > >>>> The vote ends on Tuesday, March 1, 12:00 CET.
>> > > >>>>
>> > > >>>> [ ] +1 Release this package as Apache Flink 1.0.0
>> > > >>>> [ ] -1 Do not release this package because ...
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >>>>
>> > > >
>> > >
>> >
>>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Robert Metzger <rm...@apache.org>.
Hi,
I'd like to fork off a branch for the 1.0 release so that we can merge big
changes into master. Any objections?

On Thu, Feb 25, 2016 at 6:04 PM, Greg Hogan <co...@greghogan.com> wrote:

> Hi,
>
> I have two bugfix pull requests in the stack.
>
> [FLINK-3340] [runtime] Fix object juggling in drivers
>   https://github.com/apache/flink/pull/1626
>
> [FLINK-3437] [web-dashboard] Fix UI router state for job plan
>   https://github.com/apache/flink/pull/1661
>
> Greg
>
> On Thu, Feb 25, 2016 at 8:32 AM, Robert Metzger <rm...@apache.org>
> wrote:
>
> > Damn. I agree that this is a blocker.
> > I use the maven-enforcer-plugin to check for the right maven, but the
> build
> > profile that runs the profile is only active during "deploy", not when
> > packaging the binaries.
> > That's why I didn't realize that I build the binaries with the wrong
> maven
> > version.
> >
> > I suggest that we keep collecting problems until Friday afternoon (CET).
> > Then I'll create the next release candidate.
> >
> > I'd also like to address this one:
> > https://issues.apache.org/jira/browse/FLINK-3509
> >
> >
> > On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com>
> wrote:
> >
> > > Hi folks,
> > >
> > > I think I found a release blocker.
> > > The flink-dist JAR file contains non-relocated classes of Google Guava
> > and
> > > Apache HttpComponents.
> > >
> > > Fabian
> > >
> > > 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> > >
> > > > tested the RC on Windows:
> > > >
> > > > - source compiles
> > > > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > > - start/stop scripts work in both cygwin and windows CMD
> > > > - ran several examples from batch/streaming/python
> > > > - scripts also work on paths containing spaces
> > > >
> > > >
> > > > On 25.02.2016 12:41, Robert Metzger wrote:
> > > >
> > > >> (I'm removing user@ from the discussion)
> > > >>
> > > >> Thank you for bringing the pull request to my attention Marton. I
> have
> > > to
> > > >> admit that I didn't announce this RC properly in advance. In the RC0
> > > >> thread
> > > >> I said "early next week" and now its Thursday. I should have said
> > > >> something
> > > >> in that thread.
> > > >> The "trigger" for creating the release was that the number of
> blocking
> > > >> issues is 0 now.
> > > >>
> > > >> I did a quick check of the open pull requests yesterday evening and
> > > found
> > > >> one [1] to be included into the RC as well. Since the PR you
> mentioned
> > > is
> > > >> marked with [WIP] I thought its not yet ready to be merged.
> > > >>
> > > >> I would like to find a solution that works for everyone here: I
> would
> > > like
> > > >> to avoid delaying the release until tomorrow evening, and also the
> > work
> > > it
> > > >> incurs for me create a release candidate.
> > > >> How about the following: We keep this vote open, test and check the
> > > >> release
> > > >> and you merge the change to master in the meantime.
> > > >> Most likely, the release gets cancelled anyways because we find
> > > something
> > > >> and then the next RC will contain your change.
> > > >>
> > > >> [1] https://github.com/apache/flink/pull/1706
> > > >>
> > > >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > > >> balassi.marton@gmail.com>
> > > >> wrote:
> > > >>
> > > >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> > > >>>
> > > >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> > > >>> behavior
> > > >>> of DataStream.keyBy. The PR has the feature implemented and the
> java
> > > >>> tests
> > > >>> adopted, there is still a bit of outstanding fix for the scala
> tests.
> > > >>> Gábor
> > > >>> Horváth or myself will finish it by tomorrow evening.
> > > >>>
> > > >>> [1] https://github.com/apache/flink/pull/1685
> > > >>>
> > > >>> Best,
> > > >>>
> > > >>> Marton
> > > >>>
> > > >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sbaltagi@gmail.com
> >
> > > >>> wrote:
> > > >>>
> > > >>> Dear Flink community
> > > >>>>
> > > >>>> It is great news that the vote for the first release candidate
> (RC1)
> > > of
> > > >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > > >>>> As a community, we need to double our efforts and make sure that
> > Flink
> > > >>>> 1.0.0 is GA before these 2 upcoming major events:
> > > >>>>
> > > >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > > >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > > >>>>
> > > >>>> This is one aspect of the ‘market dynamics’ that we need to take
> > into
> > > >>>> account as a community.
> > > >>>>
> > > >>>> Good luck!
> > > >>>>
> > > >>>> Slim Baltagi
> > > >>>>
> > > >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> > > >>>> wrote:
> > > >>>>
> > > >>>> Dear Flink community,
> > > >>>>
> > > >>>> Please vote on releasing the following candidate as Apache Flink
> > > version
> > > >>>> 1.0.0.
> > > >>>>
> > > >>>> I've set user@flink.apache.org on CC because users are encouraged
> > to
> > > >>>> help testing Flink 1.0.0 for their specific use cases. Please
> report
> > > >>>> issues
> > > >>>> (and successful tests!) on dev@flink.apache.org.
> > > >>>>
> > > >>>>
> > > >>>> The commit to be voted on (
> > > >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > > >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > > >>>>
> > > >>>> Branch:
> > > >>>> release-1.0.0-rc1 (see
> > > >>>>
> > > >>>>
> > >
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > > >>>> )
> > > >>>>
> > > >>>> The release artifacts to be voted on can be found at:
> > > >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > > >>>>
> > > >>>> The release artifacts are signed with the key with fingerprint
> > > D9839159:
> > > >>>> http://www.apache.org/dist/flink/KEYS
> > > >>>>
> > > >>>> The staging repository for this release can be found at:
> > > >>>>
> > > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > > >>>>
> > > >>>> -------------------------------------------------------------
> > > >>>>
> > > >>>> The vote is open until Tuesday and passes if a majority of at
> least
> > > >>>> three
> > > >>>> +1 PMC votes are cast.
> > > >>>>
> > > >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > > >>>>
> > > >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > > >>>> [ ] -1 Do not release this package because ...
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>>
> > > >
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Greg Hogan <co...@greghogan.com>.
Hi,

I have two bugfix pull requests in the stack.

[FLINK-3340] [runtime] Fix object juggling in drivers
  https://github.com/apache/flink/pull/1626

[FLINK-3437] [web-dashboard] Fix UI router state for job plan
  https://github.com/apache/flink/pull/1661

Greg

On Thu, Feb 25, 2016 at 8:32 AM, Robert Metzger <rm...@apache.org> wrote:

> Damn. I agree that this is a blocker.
> I use the maven-enforcer-plugin to check for the right maven, but the build
> profile that runs the profile is only active during "deploy", not when
> packaging the binaries.
> That's why I didn't realize that I build the binaries with the wrong maven
> version.
>
> I suggest that we keep collecting problems until Friday afternoon (CET).
> Then I'll create the next release candidate.
>
> I'd also like to address this one:
> https://issues.apache.org/jira/browse/FLINK-3509
>
>
> On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com> wrote:
>
> > Hi folks,
> >
> > I think I found a release blocker.
> > The flink-dist JAR file contains non-relocated classes of Google Guava
> and
> > Apache HttpComponents.
> >
> > Fabian
> >
> > 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
> >
> > > tested the RC on Windows:
> > >
> > > - source compiles
> > > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > > - start/stop scripts work in both cygwin and windows CMD
> > > - ran several examples from batch/streaming/python
> > > - scripts also work on paths containing spaces
> > >
> > >
> > > On 25.02.2016 12:41, Robert Metzger wrote:
> > >
> > >> (I'm removing user@ from the discussion)
> > >>
> > >> Thank you for bringing the pull request to my attention Marton. I have
> > to
> > >> admit that I didn't announce this RC properly in advance. In the RC0
> > >> thread
> > >> I said "early next week" and now its Thursday. I should have said
> > >> something
> > >> in that thread.
> > >> The "trigger" for creating the release was that the number of blocking
> > >> issues is 0 now.
> > >>
> > >> I did a quick check of the open pull requests yesterday evening and
> > found
> > >> one [1] to be included into the RC as well. Since the PR you mentioned
> > is
> > >> marked with [WIP] I thought its not yet ready to be merged.
> > >>
> > >> I would like to find a solution that works for everyone here: I would
> > like
> > >> to avoid delaying the release until tomorrow evening, and also the
> work
> > it
> > >> incurs for me create a release candidate.
> > >> How about the following: We keep this vote open, test and check the
> > >> release
> > >> and you merge the change to master in the meantime.
> > >> Most likely, the release gets cancelled anyways because we find
> > something
> > >> and then the next RC will contain your change.
> > >>
> > >> [1] https://github.com/apache/flink/pull/1706
> > >>
> > >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> > >> balassi.marton@gmail.com>
> > >> wrote:
> > >>
> > >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> > >>>
> > >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> > >>> behavior
> > >>> of DataStream.keyBy. The PR has the feature implemented and the java
> > >>> tests
> > >>> adopted, there is still a bit of outstanding fix for the scala tests.
> > >>> Gábor
> > >>> Horváth or myself will finish it by tomorrow evening.
> > >>>
> > >>> [1] https://github.com/apache/flink/pull/1685
> > >>>
> > >>> Best,
> > >>>
> > >>> Marton
> > >>>
> > >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
> > >>> wrote:
> > >>>
> > >>> Dear Flink community
> > >>>>
> > >>>> It is great news that the vote for the first release candidate (RC1)
> > of
> > >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> > >>>> As a community, we need to double our efforts and make sure that
> Flink
> > >>>> 1.0.0 is GA before these 2 upcoming major events:
> > >>>>
> > >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> > >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> > >>>>
> > >>>> This is one aspect of the ‘market dynamics’ that we need to take
> into
> > >>>> account as a community.
> > >>>>
> > >>>> Good luck!
> > >>>>
> > >>>> Slim Baltagi
> > >>>>
> > >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> > >>>> wrote:
> > >>>>
> > >>>> Dear Flink community,
> > >>>>
> > >>>> Please vote on releasing the following candidate as Apache Flink
> > version
> > >>>> 1.0.0.
> > >>>>
> > >>>> I've set user@flink.apache.org on CC because users are encouraged
> to
> > >>>> help testing Flink 1.0.0 for their specific use cases. Please report
> > >>>> issues
> > >>>> (and successful tests!) on dev@flink.apache.org.
> > >>>>
> > >>>>
> > >>>> The commit to be voted on (
> > >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> > >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> > >>>>
> > >>>> Branch:
> > >>>> release-1.0.0-rc1 (see
> > >>>>
> > >>>>
> >
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> > >>>> )
> > >>>>
> > >>>> The release artifacts to be voted on can be found at:
> > >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> > >>>>
> > >>>> The release artifacts are signed with the key with fingerprint
> > D9839159:
> > >>>> http://www.apache.org/dist/flink/KEYS
> > >>>>
> > >>>> The staging repository for this release can be found at:
> > >>>>
> > https://repository.apache.org/content/repositories/orgapacheflink-1063
> > >>>>
> > >>>> -------------------------------------------------------------
> > >>>>
> > >>>> The vote is open until Tuesday and passes if a majority of at least
> > >>>> three
> > >>>> +1 PMC votes are cast.
> > >>>>
> > >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> > >>>>
> > >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> > >>>> [ ] -1 Do not release this package because ...
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Robert Metzger <rm...@apache.org>.
Damn. I agree that this is a blocker.
I use the maven-enforcer-plugin to check for the right maven, but the build
profile that runs the profile is only active during "deploy", not when
packaging the binaries.
That's why I didn't realize that I build the binaries with the wrong maven
version.

I suggest that we keep collecting problems until Friday afternoon (CET).
Then I'll create the next release candidate.

I'd also like to address this one:
https://issues.apache.org/jira/browse/FLINK-3509


On Thu, Feb 25, 2016 at 2:23 PM, Fabian Hueske <fh...@gmail.com> wrote:

> Hi folks,
>
> I think I found a release blocker.
> The flink-dist JAR file contains non-relocated classes of Google Guava and
> Apache HttpComponents.
>
> Fabian
>
> 2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:
>
> > tested the RC on Windows:
> >
> > - source compiles
> > - some tests categorically fail: see FLINK-3491 / FLINK-3496
> > - start/stop scripts work in both cygwin and windows CMD
> > - ran several examples from batch/streaming/python
> > - scripts also work on paths containing spaces
> >
> >
> > On 25.02.2016 12:41, Robert Metzger wrote:
> >
> >> (I'm removing user@ from the discussion)
> >>
> >> Thank you for bringing the pull request to my attention Marton. I have
> to
> >> admit that I didn't announce this RC properly in advance. In the RC0
> >> thread
> >> I said "early next week" and now its Thursday. I should have said
> >> something
> >> in that thread.
> >> The "trigger" for creating the release was that the number of blocking
> >> issues is 0 now.
> >>
> >> I did a quick check of the open pull requests yesterday evening and
> found
> >> one [1] to be included into the RC as well. Since the PR you mentioned
> is
> >> marked with [WIP] I thought its not yet ready to be merged.
> >>
> >> I would like to find a solution that works for everyone here: I would
> like
> >> to avoid delaying the release until tomorrow evening, and also the work
> it
> >> incurs for me create a release candidate.
> >> How about the following: We keep this vote open, test and check the
> >> release
> >> and you merge the change to master in the meantime.
> >> Most likely, the release gets cancelled anyways because we find
> something
> >> and then the next RC will contain your change.
> >>
> >> [1] https://github.com/apache/flink/pull/1706
> >>
> >> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
> >> balassi.marton@gmail.com>
> >> wrote:
> >>
> >> Thanks for creating the candidate Robert and for the heads-up, Slim.
> >>>
> >>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
> >>> behavior
> >>> of DataStream.keyBy. The PR has the feature implemented and the java
> >>> tests
> >>> adopted, there is still a bit of outstanding fix for the scala tests.
> >>> Gábor
> >>> Horváth or myself will finish it by tomorrow evening.
> >>>
> >>> [1] https://github.com/apache/flink/pull/1685
> >>>
> >>> Best,
> >>>
> >>> Marton
> >>>
> >>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
> >>> wrote:
> >>>
> >>> Dear Flink community
> >>>>
> >>>> It is great news that the vote for the first release candidate (RC1)
> of
> >>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
> >>>> As a community, we need to double our efforts and make sure that Flink
> >>>> 1.0.0 is GA before these 2 upcoming major events:
> >>>>
> >>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
> >>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
> >>>>
> >>>> This is one aspect of the ‘market dynamics’ that we need to take into
> >>>> account as a community.
> >>>>
> >>>> Good luck!
> >>>>
> >>>> Slim Baltagi
> >>>>
> >>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
> >>>> wrote:
> >>>>
> >>>> Dear Flink community,
> >>>>
> >>>> Please vote on releasing the following candidate as Apache Flink
> version
> >>>> 1.0.0.
> >>>>
> >>>> I've set user@flink.apache.org on CC because users are encouraged to
> >>>> help testing Flink 1.0.0 for their specific use cases. Please report
> >>>> issues
> >>>> (and successful tests!) on dev@flink.apache.org.
> >>>>
> >>>>
> >>>> The commit to be voted on (
> >>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> >>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> >>>>
> >>>> Branch:
> >>>> release-1.0.0-rc1 (see
> >>>>
> >>>>
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> >>>> )
> >>>>
> >>>> The release artifacts to be voted on can be found at:
> >>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> >>>>
> >>>> The release artifacts are signed with the key with fingerprint
> D9839159:
> >>>> http://www.apache.org/dist/flink/KEYS
> >>>>
> >>>> The staging repository for this release can be found at:
> >>>>
> https://repository.apache.org/content/repositories/orgapacheflink-1063
> >>>>
> >>>> -------------------------------------------------------------
> >>>>
> >>>> The vote is open until Tuesday and passes if a majority of at least
> >>>> three
> >>>> +1 PMC votes are cast.
> >>>>
> >>>> The vote ends on Tuesday, March 1, 12:00 CET.
> >>>>
> >>>> [ ] +1 Release this package as Apache Flink 1.0.0
> >>>> [ ] -1 Do not release this package because ...
> >>>>
> >>>>
> >>>>
> >>>>
> >
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Fabian Hueske <fh...@gmail.com>.
Hi folks,

I think I found a release blocker.
The flink-dist JAR file contains non-relocated classes of Google Guava and
Apache HttpComponents.

Fabian

2016-02-25 13:21 GMT+01:00 Chesnay Schepler <ch...@apache.org>:

> tested the RC on Windows:
>
> - source compiles
> - some tests categorically fail: see FLINK-3491 / FLINK-3496
> - start/stop scripts work in both cygwin and windows CMD
> - ran several examples from batch/streaming/python
> - scripts also work on paths containing spaces
>
>
> On 25.02.2016 12:41, Robert Metzger wrote:
>
>> (I'm removing user@ from the discussion)
>>
>> Thank you for bringing the pull request to my attention Marton. I have to
>> admit that I didn't announce this RC properly in advance. In the RC0
>> thread
>> I said "early next week" and now its Thursday. I should have said
>> something
>> in that thread.
>> The "trigger" for creating the release was that the number of blocking
>> issues is 0 now.
>>
>> I did a quick check of the open pull requests yesterday evening and found
>> one [1] to be included into the RC as well. Since the PR you mentioned is
>> marked with [WIP] I thought its not yet ready to be merged.
>>
>> I would like to find a solution that works for everyone here: I would like
>> to avoid delaying the release until tomorrow evening, and also the work it
>> incurs for me create a release candidate.
>> How about the following: We keep this vote open, test and check the
>> release
>> and you merge the change to master in the meantime.
>> Most likely, the release gets cancelled anyways because we find something
>> and then the next RC will contain your change.
>>
>> [1] https://github.com/apache/flink/pull/1706
>>
>> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <
>> balassi.marton@gmail.com>
>> wrote:
>>
>> Thanks for creating the candidate Robert and for the heads-up, Slim.
>>>
>>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing
>>> behavior
>>> of DataStream.keyBy. The PR has the feature implemented and the java
>>> tests
>>> adopted, there is still a bit of outstanding fix for the scala tests.
>>> Gábor
>>> Horváth or myself will finish it by tomorrow evening.
>>>
>>> [1] https://github.com/apache/flink/pull/1685
>>>
>>> Best,
>>>
>>> Marton
>>>
>>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com>
>>> wrote:
>>>
>>> Dear Flink community
>>>>
>>>> It is great news that the vote for the first release candidate (RC1) of
>>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
>>>> As a community, we need to double our efforts and make sure that Flink
>>>> 1.0.0 is GA before these 2 upcoming major events:
>>>>
>>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>>>>
>>>> This is one aspect of the ‘market dynamics’ that we need to take into
>>>> account as a community.
>>>>
>>>> Good luck!
>>>>
>>>> Slim Baltagi
>>>>
>>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org>
>>>> wrote:
>>>>
>>>> Dear Flink community,
>>>>
>>>> Please vote on releasing the following candidate as Apache Flink version
>>>> 1.0.0.
>>>>
>>>> I've set user@flink.apache.org on CC because users are encouraged to
>>>> help testing Flink 1.0.0 for their specific use cases. Please report
>>>> issues
>>>> (and successful tests!) on dev@flink.apache.org.
>>>>
>>>>
>>>> The commit to be voted on (
>>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>>>>
>>>> Branch:
>>>> release-1.0.0-rc1 (see
>>>>
>>>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
>>>> )
>>>>
>>>> The release artifacts to be voted on can be found at:
>>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>>>>
>>>> The release artifacts are signed with the key with fingerprint D9839159:
>>>> http://www.apache.org/dist/flink/KEYS
>>>>
>>>> The staging repository for this release can be found at:
>>>> https://repository.apache.org/content/repositories/orgapacheflink-1063
>>>>
>>>> -------------------------------------------------------------
>>>>
>>>> The vote is open until Tuesday and passes if a majority of at least
>>>> three
>>>> +1 PMC votes are cast.
>>>>
>>>> The vote ends on Tuesday, March 1, 12:00 CET.
>>>>
>>>> [ ] +1 Release this package as Apache Flink 1.0.0
>>>> [ ] -1 Do not release this package because ...
>>>>
>>>>
>>>>
>>>>
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Chesnay Schepler <ch...@apache.org>.
tested the RC on Windows:

- source compiles
- some tests categorically fail: see FLINK-3491 / FLINK-3496
- start/stop scripts work in both cygwin and windows CMD
- ran several examples from batch/streaming/python
- scripts also work on paths containing spaces

On 25.02.2016 12:41, Robert Metzger wrote:
> (I'm removing user@ from the discussion)
>
> Thank you for bringing the pull request to my attention Marton. I have to
> admit that I didn't announce this RC properly in advance. In the RC0 thread
> I said "early next week" and now its Thursday. I should have said something
> in that thread.
> The "trigger" for creating the release was that the number of blocking
> issues is 0 now.
>
> I did a quick check of the open pull requests yesterday evening and found
> one [1] to be included into the RC as well. Since the PR you mentioned is
> marked with [WIP] I thought its not yet ready to be merged.
>
> I would like to find a solution that works for everyone here: I would like
> to avoid delaying the release until tomorrow evening, and also the work it
> incurs for me create a release candidate.
> How about the following: We keep this vote open, test and check the release
> and you merge the change to master in the meantime.
> Most likely, the release gets cancelled anyways because we find something
> and then the next RC will contain your change.
>
> [1] https://github.com/apache/flink/pull/1706
>
> On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <ba...@gmail.com>
> wrote:
>
>> Thanks for creating the candidate Robert and for the heads-up, Slim.
>>
>> I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior
>> of DataStream.keyBy. The PR has the feature implemented and the java tests
>> adopted, there is still a bit of outstanding fix for the scala tests. Gábor
>> Horváth or myself will finish it by tomorrow evening.
>>
>> [1] https://github.com/apache/flink/pull/1685
>>
>> Best,
>>
>> Marton
>>
>> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com> wrote:
>>
>>> Dear Flink community
>>>
>>> It is great news that the vote for the first release candidate (RC1) of
>>> Apache Flink 1.0.0 is starting today February 25th, 2016!
>>> As a community, we need to double our efforts and make sure that Flink
>>> 1.0.0 is GA before these 2 upcoming major events:
>>>
>>>     -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>>>     -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>>>
>>> This is one aspect of the ‘market dynamics’ that we need to take into
>>> account as a community.
>>>
>>> Good luck!
>>>
>>> Slim Baltagi
>>>
>>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:
>>>
>>> Dear Flink community,
>>>
>>> Please vote on releasing the following candidate as Apache Flink version
>>> 1.0.0.
>>>
>>> I've set user@flink.apache.org on CC because users are encouraged to
>>> help testing Flink 1.0.0 for their specific use cases. Please report issues
>>> (and successful tests!) on dev@flink.apache.org.
>>>
>>>
>>> The commit to be voted on (
>>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>>>
>>> Branch:
>>> release-1.0.0-rc1 (see
>>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
>>> )
>>>
>>> The release artifacts to be voted on can be found at:
>>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>>>
>>> The release artifacts are signed with the key with fingerprint D9839159:
>>> http://www.apache.org/dist/flink/KEYS
>>>
>>> The staging repository for this release can be found at:
>>> https://repository.apache.org/content/repositories/orgapacheflink-1063
>>>
>>> -------------------------------------------------------------
>>>
>>> The vote is open until Tuesday and passes if a majority of at least three
>>> +1 PMC votes are cast.
>>>
>>> The vote ends on Tuesday, March 1, 12:00 CET.
>>>
>>> [ ] +1 Release this package as Apache Flink 1.0.0
>>> [ ] -1 Do not release this package because ...
>>>
>>>
>>>


Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Robert Metzger <rm...@apache.org>.
(I'm removing user@ from the discussion)

Thank you for bringing the pull request to my attention Marton. I have to
admit that I didn't announce this RC properly in advance. In the RC0 thread
I said "early next week" and now its Thursday. I should have said something
in that thread.
The "trigger" for creating the release was that the number of blocking
issues is 0 now.

I did a quick check of the open pull requests yesterday evening and found
one [1] to be included into the RC as well. Since the PR you mentioned is
marked with [WIP] I thought its not yet ready to be merged.

I would like to find a solution that works for everyone here: I would like
to avoid delaying the release until tomorrow evening, and also the work it
incurs for me create a release candidate.
How about the following: We keep this vote open, test and check the release
and you merge the change to master in the meantime.
Most likely, the release gets cancelled anyways because we find something
and then the next RC will contain your change.

[1] https://github.com/apache/flink/pull/1706

On Thu, Feb 25, 2016 at 12:11 PM, Márton Balassi <ba...@gmail.com>
wrote:

> Thanks for creating the candidate Robert and for the heads-up, Slim.
>
> I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior
> of DataStream.keyBy. The PR has the feature implemented and the java tests
> adopted, there is still a bit of outstanding fix for the scala tests. Gábor
> Horváth or myself will finish it by tomorrow evening.
>
> [1] https://github.com/apache/flink/pull/1685
>
> Best,
>
> Marton
>
> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com> wrote:
>
>> Dear Flink community
>>
>> It is great news that the vote for the first release candidate (RC1) of
>> Apache Flink 1.0.0 is starting today February 25th, 2016!
>> As a community, we need to double our efforts and make sure that Flink
>> 1.0.0 is GA before these 2 upcoming major events:
>>
>>    -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>>    -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>>
>> This is one aspect of the ‘market dynamics’ that we need to take into
>> account as a community.
>>
>> Good luck!
>>
>> Slim Baltagi
>>
>> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:
>>
>> Dear Flink community,
>>
>> Please vote on releasing the following candidate as Apache Flink version
>> 1.0.0.
>>
>> I've set user@flink.apache.org on CC because users are encouraged to
>> help testing Flink 1.0.0 for their specific use cases. Please report issues
>> (and successful tests!) on dev@flink.apache.org.
>>
>>
>> The commit to be voted on (
>> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>>
>> Branch:
>> release-1.0.0-rc1 (see
>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
>> )
>>
>> The release artifacts to be voted on can be found at:
>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>>
>> The release artifacts are signed with the key with fingerprint D9839159:
>> http://www.apache.org/dist/flink/KEYS
>>
>> The staging repository for this release can be found at:
>> https://repository.apache.org/content/repositories/orgapacheflink-1063
>>
>> -------------------------------------------------------------
>>
>> The vote is open until Tuesday and passes if a majority of at least three
>> +1 PMC votes are cast.
>>
>> The vote ends on Tuesday, March 1, 12:00 CET.
>>
>> [ ] +1 Release this package as Apache Flink 1.0.0
>> [ ] -1 Do not release this package because ...
>>
>>
>>
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Aljoscha Krettek <al...@apache.org>.
I’ll look at the usual testing stuff and also focus on testing savepoints on a cluster.

Btw, we don’t yet have the usual “testing checklist” document, do we?

> On 25 Feb 2016, at 12:11, Márton Balassi <ba...@gmail.com> wrote:
> 
> Thanks for creating the candidate Robert and for the heads-up, Slim.
> 
> I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior of DataStream.keyBy. The PR has the feature implemented and the java tests adopted, there is still a bit of outstanding fix for the scala tests. Gábor Horváth or myself will finish it by tomorrow evening.
> 
> [1] https://github.com/apache/flink/pull/1685
> 
> Best,
> 
> Marton
> 
> On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com> wrote:
> Dear Flink community
> 
> It is great news that the vote for the first release candidate (RC1) of Apache Flink 1.0.0 is starting today February 25th, 2016!
> As a community, we need to double our efforts and make sure that Flink 1.0.0 is GA before these 2 upcoming major events: 
> 	•  Strata + Hadoop World in San Jose on March 28-31, 2016
> 	•  Hadoop Summit Europe in Dublin on April 13-14, 2016
> This is one aspect of the ‘market dynamics’ that we need to take into account as a community. 
> 
> Good luck!
> 
> Slim Baltagi
> 
> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:
> 
>> Dear Flink community,
>> 
>> Please vote on releasing the following candidate as Apache Flink version 1.0.0.
>> 
>> I've set user@flink.apache.org on CC because users are encouraged to help testing Flink 1.0.0 for their specific use cases. Please report issues (and successful tests!) on dev@flink.apache.org.
>> 
>> 
>> The commit to be voted on (http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
>> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>> 
>> Branch:
>> release-1.0.0-rc1 (see https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1)
>> 
>> The release artifacts to be voted on can be found at:
>> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>> 
>> The release artifacts are signed with the key with fingerprint D9839159:
>> http://www.apache.org/dist/flink/KEYS
>> 
>> The staging repository for this release can be found at:
>> https://repository.apache.org/content/repositories/orgapacheflink-1063
>> 
>> -------------------------------------------------------------
>> 
>> The vote is open until Tuesday and passes if a majority of at least three +1 PMC votes are cast.
>> 
>> The vote ends on Tuesday, March 1, 12:00 CET.
>> 
>> [ ] +1 Release this package as Apache Flink 1.0.0
>> [ ] -1 Do not release this package because ...
> 
> 


Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Márton Balassi <ba...@gmail.com>.
Thanks for creating the candidate Robert and for the heads-up, Slim.

I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior
of DataStream.keyBy. The PR has the feature implemented and the java tests
adopted, there is still a bit of outstanding fix for the scala tests. Gábor
Horváth or myself will finish it by tomorrow evening.

[1] https://github.com/apache/flink/pull/1685

Best,

Marton

On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com> wrote:

> Dear Flink community
>
> It is great news that the vote for the first release candidate (RC1) of
> Apache Flink 1.0.0 is starting today February 25th, 2016!
> As a community, we need to double our efforts and make sure that Flink
> 1.0.0 is GA before these 2 upcoming major events:
>
>    -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>    -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>
> This is one aspect of the ‘market dynamics’ that we need to take into
> account as a community.
>
> Good luck!
>
> Slim Baltagi
>
> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:
>
> Dear Flink community,
>
> Please vote on releasing the following candidate as Apache Flink version
> 1.0.0.
>
> I've set user@flink.apache.org on CC because users are encouraged to help
> testing Flink 1.0.0 for their specific use cases. Please report issues (and
> successful tests!) on dev@flink.apache.org.
>
>
> The commit to be voted on (
> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>
> Branch:
> release-1.0.0-rc1 (see
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> )
>
> The release artifacts to be voted on can be found at:
> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>
> The release artifacts are signed with the key with fingerprint D9839159:
> http://www.apache.org/dist/flink/KEYS
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapacheflink-1063
>
> -------------------------------------------------------------
>
> The vote is open until Tuesday and passes if a majority of at least three
> +1 PMC votes are cast.
>
> The vote ends on Tuesday, March 1, 12:00 CET.
>
> [ ] +1 Release this package as Apache Flink 1.0.0
> [ ] -1 Do not release this package because ...
>
>
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Márton Balassi <ba...@gmail.com>.
Thanks for creating the candidate Robert and for the heads-up, Slim.

I would like to get a PR [1] in before 1.0.0 as it breaks hashing behavior
of DataStream.keyBy. The PR has the feature implemented and the java tests
adopted, there is still a bit of outstanding fix for the scala tests. Gábor
Horváth or myself will finish it by tomorrow evening.

[1] https://github.com/apache/flink/pull/1685

Best,

Marton

On Thu, Feb 25, 2016 at 12:04 PM, Slim Baltagi <sb...@gmail.com> wrote:

> Dear Flink community
>
> It is great news that the vote for the first release candidate (RC1) of
> Apache Flink 1.0.0 is starting today February 25th, 2016!
> As a community, we need to double our efforts and make sure that Flink
> 1.0.0 is GA before these 2 upcoming major events:
>
>    -  Strata + Hadoop World in San Jose on *March 28-31, 2016*
>    -  Hadoop Summit Europe in Dublin on *April 13-14, 2016*
>
> This is one aspect of the ‘market dynamics’ that we need to take into
> account as a community.
>
> Good luck!
>
> Slim Baltagi
>
> On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:
>
> Dear Flink community,
>
> Please vote on releasing the following candidate as Apache Flink version
> 1.0.0.
>
> I've set user@flink.apache.org on CC because users are encouraged to help
> testing Flink 1.0.0 for their specific use cases. Please report issues (and
> successful tests!) on dev@flink.apache.org.
>
>
> The commit to be voted on (
> http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
>
> Branch:
> release-1.0.0-rc1 (see
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1
> )
>
> The release artifacts to be voted on can be found at:
> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
>
> The release artifacts are signed with the key with fingerprint D9839159:
> http://www.apache.org/dist/flink/KEYS
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapacheflink-1063
>
> -------------------------------------------------------------
>
> The vote is open until Tuesday and passes if a majority of at least three
> +1 PMC votes are cast.
>
> The vote ends on Tuesday, March 1, 12:00 CET.
>
> [ ] +1 Release this package as Apache Flink 1.0.0
> [ ] -1 Do not release this package because ...
>
>
>

Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Slim Baltagi <sb...@gmail.com>.
Dear Flink community

It is great news that the vote for the first release candidate (RC1) of Apache Flink 1.0.0 is starting today February 25th, 2016!
As a community, we need to double our efforts and make sure that Flink 1.0.0 is GA before these 2 upcoming major events: 
 Strata + Hadoop World in San Jose on March 28-31, 2016
 Hadoop Summit Europe in Dublin on April 13-14, 2016
This is one aspect of the ‘market dynamics’ that we need to take into account as a community. 

Good luck!

Slim Baltagi

On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:

> Dear Flink community,
> 
> Please vote on releasing the following candidate as Apache Flink version 1.0.0.
> 
> I've set user@flink.apache.org on CC because users are encouraged to help testing Flink 1.0.0 for their specific use cases. Please report issues (and successful tests!) on dev@flink.apache.org.
> 
> 
> The commit to be voted on (http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> 
> Branch:
> release-1.0.0-rc1 (see https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1)
> 
> The release artifacts to be voted on can be found at:
> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> 
> The release artifacts are signed with the key with fingerprint D9839159:
> http://www.apache.org/dist/flink/KEYS
> 
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapacheflink-1063
> 
> -------------------------------------------------------------
> 
> The vote is open until Tuesday and passes if a majority of at least three +1 PMC votes are cast.
> 
> The vote ends on Tuesday, March 1, 12:00 CET.
> 
> [ ] +1 Release this package as Apache Flink 1.0.0
> [ ] -1 Do not release this package because ...


Re: [VOTE] Release Apache Flink 1.0.0 (RC1)

Posted by Slim Baltagi <sb...@gmail.com>.
Dear Flink community

It is great news that the vote for the first release candidate (RC1) of Apache Flink 1.0.0 is starting today February 25th, 2016!
As a community, we need to double our efforts and make sure that Flink 1.0.0 is GA before these 2 upcoming major events: 
 Strata + Hadoop World in San Jose on March 28-31, 2016
 Hadoop Summit Europe in Dublin on April 13-14, 2016
This is one aspect of the ‘market dynamics’ that we need to take into account as a community. 

Good luck!

Slim Baltagi

On Feb 25, 2016, at 4:34 AM, Robert Metzger <rm...@apache.org> wrote:

> Dear Flink community,
> 
> Please vote on releasing the following candidate as Apache Flink version 1.0.0.
> 
> I've set user@flink.apache.org on CC because users are encouraged to help testing Flink 1.0.0 for their specific use cases. Please report issues (and successful tests!) on dev@flink.apache.org.
> 
> 
> The commit to be voted on (http://git-wip-us.apache.org/repos/asf/flink/commit/e4d308d6)
> e4d308d64057e5f94bec8bbca8f67aab0ea78faa
> 
> Branch:
> release-1.0.0-rc1 (see https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc1)
> 
> The release artifacts to be voted on can be found at:
> http://people.apache.org/~rmetzger/flink-1.0.0-rc1/
> 
> The release artifacts are signed with the key with fingerprint D9839159:
> http://www.apache.org/dist/flink/KEYS
> 
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapacheflink-1063
> 
> -------------------------------------------------------------
> 
> The vote is open until Tuesday and passes if a majority of at least three +1 PMC votes are cast.
> 
> The vote ends on Tuesday, March 1, 12:00 CET.
> 
> [ ] +1 Release this package as Apache Flink 1.0.0
> [ ] -1 Do not release this package because ...