+1 for another RC once the shading fix is merged.

Even if RC1 would have pending issues, users that want to help testing need
a new RC, because (as Robert said), RC0's shading issues make it virtually
unusable...

On Fri, May 12, 2017 at 3:45 PM, Robert Metzger <rmetz...@apache.org> wrote:

> I'm considering to do another non voting RC1 on Monday with the incremental
> checkpointing and maven fixes in (and of course as many other things as
> possible). Once the other critical fixes are in (ideally Tuesday or
> Wednesday), I'll create the first voting one.
>
> The current RC0 is almost unusable because of the incorrect shading.
>
> On Fri, May 12, 2017 at 2:09 PM, Greg Hogan <c...@greghogan.com> wrote:
>
> > +1 for sticking to the code freeze deadline and building a new release
> > candidate but since the release is still two weeks off (5/26) I think it
> > better to delay voting to give time for additional bug fixes.
> >
> >
> > > On May 11, 2017, at 10:19 AM, Robert Metzger <rmetz...@apache.org>
> > wrote:
> > >
> > > It seems that we found quite a large number of critical issues in the
> > first
> > > RC.
> > >
> > > - FLINK-6537 Umbrella issue for fixes to incremental snapshots (Stefan
> > has
> > > a PR open to fix the critical ones)
> > > - FLINK-6531 Deserialize checkpoint hooks with user classloader (has a
> > > pending PR)
> > > - FLINK-6515 KafkaConsumer checkpointing fails because of ClassLoader
> > > issues (status unknown)
> > > - FLINK-6514 Cannot start Flink Cluster in standalone mode (Stephan is
> on
> > > it)
> > > - FLINK-6508 Include license files of packaged dependencies (Stephan is
> > on
> > > it) + FLINK-6501 Make sure NOTICE files are bundled into shaded JAR
> files
> > > - FLINK-6284 Incorrect sorting of completed checkpoints in
> > > ZooKeeperCompletedCheckpointStore (unknown)
> > >
> > > I would like to get these issues fixed by end of this week (Sunday), so
> > > that I can create the first voting RC on Monday morning.
> > > Please reject if you think we will not manage to get the stuff fixed
> > until
> > > then.
> > >
> > >
> > >
> > > On Thu, May 11, 2017 at 10:54 AM, Till Rohrmann <trohrm...@apache.org>
> > > wrote:
> > >
> > >> Unfortunately, it won't be fully functional in 1.3.
> > >>
> > >> Cheers,
> > >> Till
> > >>
> > >> On Thu, May 11, 2017 at 10:45 AM, Renjie Liu <liurenjie2...@gmail.com
> >
> > >> wrote:
> > >>
> > >>> @Rohrmann Will FLIP 6 be fully functional in 1.3 release?
> > >>>
> > >>> On Thu, May 11, 2017 at 4:12 PM Gyula Fóra <gyula.f...@gmail.com>
> > wrote:
> > >>>
> > >>>> Thanks Stefan!
> > >>>> Gyula
> > >>>>
> > >>>> Stefan Richter <s.rich...@data-artisans.com> ezt írta (időpont:
> 2017.
> > >>> máj.
> > >>>> 11., Cs, 10:04):
> > >>>>
> > >>>>>
> > >>>>> Hi,
> > >>>>>
> > >>>>> Thanks for reporting this. I found a couple of issues yesterday
> and I
> > >>> am
> > >>>>> currently working on a bundle of fixes. I will take a look at this
> > >>>> problem,
> > >>>>> and if it is already covered.
> > >>>>>
> > >>>>> Best,
> > >>>>> Stefan
> > >>>>>
> > >>>>>> Am 11.05.2017 um 09:47 schrieb Gyula Fóra <gyula.f...@gmail.com>:
> > >>>>>>
> > >>>>>> Hi,
> > >>>>>> I am not sure if this belong to this thread, but while trying to
> > >> run
> > >>> a
> > >>>>> job
> > >>>>>> with rocks incremental backend I ran into 2 issues:
> > >>>>>>
> > >>>>>> One with savepoints, I can't figure out because I can't make sense
> > >> of
> > >>>> the
> > >>>>>> error or how it happenned:
> > >>>>>> The error stack trace is here:
> > >>>>>> https://gist.github.com/gyfora/2f7bb387bbd9f455f9702908cde0b239
> > >>>>>> This happens on every savepoint attempt and seems to be related to
> > >>> the
> > >>>>>> kafka source. Interestingly other tasks succeed in writing data to
> > >>>> hdfs.
> > >>>>>>
> > >>>>>> The other one is covered by
> > >>>>> https://issues.apache.org/jira/browse/FLINK-6531 I
> > >>>>>> guess. I am not sure if the first one is related though.
> > >>>>>>
> > >>>>>> Thank you!
> > >>>>>> Gyula
> > >>>>>>
> > >>>>>> Till Rohrmann <trohrm...@apache.org> ezt írta (időpont: 2017.
> máj.
> > >>>> 11.,
> > >>>>> Cs,
> > >>>>>> 9:14):
> > >>>>>>
> > >>>>>>> Hi Renjie,
> > >>>>>>>
> > >>>>>>> 1.3 already contains some Flip-6 code. However, it is not yet
> > >> fully
> > >>>>>>> functional. What you already can do is to run local jobs on the
> > >>> Flip-6
> > >>>>> code
> > >>>>>>> base by instantiating a MiniCluster and then using the
> > >>>>>>> Flip6LocalStreamEnvironment.
> > >>>>>>>
> > >>>>>>> Cheers,
> > >>>>>>> Till
> > >>>>>>>
> > >>>>>>> On Thu, May 11, 2017 at 6:00 AM, Renjie Liu <
> > >>> liurenjie2...@gmail.com>
> > >>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> Hi, all:
> > >>>>>>>> Will the FLIP 6 be included in release 1.3?
> > >>>>>>>>
> > >>>>>>>> On Wed, May 10, 2017 at 9:48 PM Gyula Fóra <
> gyula.f...@gmail.com
> > >>>
> > >>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Thanks you! :)
> > >>>>>>>>>
> > >>>>>>>>> Chesnay Schepler <ches...@apache.org> ezt írta (időpont: 2017.
> > >>> máj.
> > >>>>>>> 10.,
> > >>>>>>>>> Sze, 15:44):
> > >>>>>>>>>
> > >>>>>>>>>> I guess it is related to this one
> > >>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-6514 ?
> > >>>>>>>>>>
> > >>>>>>>>>> On 10.05.2017 15:34, Gyula Fóra wrote:
> > >>>>>>>>>>> Hi,
> > >>>>>>>>>>>
> > >>>>>>>>>>> I tried to run an application on 1.3 but I keep getting the
> > >>>>>>> following
> > >>>>>>>>>> error:
> > >>>>>>>>>>> java.lang.NoClassDefFoundError: Could not initialize class
> > >>>>>>>>>>> org.apache.hadoop.security.UserGroupInformation
> > >>>>>>>>>>> at
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>> org.apache.flink.runtime.security.modules.HadoopModule.
> > >>>>>>>> install(HadoopModule.java:45)
> > >>>>>>>>>>> at
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>> org.apache.flink.runtime.security.SecurityUtils.
> > >>>>>>>> install(SecurityUtils.java:78)
> > >>>>>>>>>>> at org.apache.flink.client.CliFrontend.main(CliFrontend.
> > >>> java:1128)
> > >>>>>>>>>>>
> > >>>>>>>>>>> Even after adding hadoop-common to the lib manually (which I
> > >>> guess
> > >>>>>>>>> should
> > >>>>>>>>>>> not be necessary).
> > >>>>>>>>>>>
> > >>>>>>>>>>> Any idea what might cause this?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>> Gyula
> > >>>>>>>>>>>
> > >>>>>>>>>>> Chesnay Schepler <ches...@apache.org> ezt írta (időpont:
> > >> 2017.
> > >>>>>>> máj.
> > >>>>>>>>> 9.,
> > >>>>>>>>>> K,
> > >>>>>>>>>>> 13:27):
> > >>>>>>>>>>>
> > >>>>>>>>>>>> I looked into AppVeyor and managed to create a working setup
> > >>> that
> > >>>>>>>>>>>> replicates the problems i encounter locally.
> > >>>>>>>>>>>> https://ci.appveyor.com/project/zentol/flink/build/1.0.12
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> The integration should be as simple as placing an
> > >> appveyor.yml
> > >>>>>>>>> (similar
> > >>>>>>>>>>>> to the travis.yml) into the repository and asking INFRA to
> > >>>> enabled
> > >>>>>>>> it
> > >>>>>>>>>>>> for Flink.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I've opened a JIRA to track this initiative:
> > >>>>>>>>>>>> https://issues.apache.org/jira/browse/FLINK-6510
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 09.05.2017 10:17, Robert Metzger wrote:
> > >>>>>>>>>>>>> Chesnay, I would propose to look into a CI solution for
> > >>> Windows.
> > >>>>>>>> I'll
> > >>>>>>>>>> try
> > >>>>>>>>>>>>> out appveyor.com for my Flink fork and report back how
> well
> > >>> it
> > >>>>>>>> works
> > >>>>>>>>>>>> (let
> > >>>>>>>>>>>>> me know if you want to look into it).
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Tue, May 9, 2017 at 10:05 AM, Stephan Ewen <
> > >>> se...@apache.org
> > >>>>>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> @Renjie What functionality are you looking for? Most stuff
> > >>>>>>> should
> > >>>>>>>>> work
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> the normal master branch.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Tue, May 9, 2017 at 10:02 AM, Renjie Liu <
> > >>>>>>>>> liurenjie2...@gmail.com>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Stephan:
> > >>>>>>>>>>>>>>> This means that  we can use FLIP 6 for basic
> functionality
> > >>>> now?
> > >>>>>>>> Any
> > >>>>>>>>>>>>>>> document for setting up that? Will FLIP 6 be finished in
> > >>>>>>> release
> > >>>>>>>>> 1.3?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Tue, May 9, 2017 at 3:43 PM Stephan Ewen <
> > >>> se...@apache.org
> > >>>>>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> @Renjie Liu: A good part of the FLIP-6 code is in there,
> > >>> but
> > >>>>>>> not
> > >>>>>>>>>> all.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> It does run well on Yarn, Mesos, Docker, etc.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> We need to finish the FLIP-6 work mainly for full
> > >>> elasticity.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Tue, May 9, 2017 at 5:24 AM, Renjie Liu <
> > >>>>>>>>> liurenjie2...@gmail.com
> > >>>>>>>>>>>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Hi, does this include the FLIP6?
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Tue, May 9, 2017 at 2:29 AM Stephan Ewen <
> > >>>>>>> se...@apache.org>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>> Did a quick test: Simply adding the
> > >>>>>>> "org.apache.maven.plugins.
> > >>>>>>>>>>>>>>>>>> shade.resource.ApacheNoticeResourceTransformer" helps
> > >>> with
> > >>>>>>>>> NOTICE
> > >>>>>>>>>>>>>>>>>> files,
> > >>>>>>>>>>>>>>>>>> but does not add the required BSD licence copies.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Mon, May 8, 2017 at 8:25 PM, Stephan Ewen <
> > >>>>>>>> se...@apache.org>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>> I did the first pass for the legal check.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>   - Source LICENSE and NOTICE are okay
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>   - In the shaded JAR files, we are not bundling the
> > >>>>>>>> license
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>> notice files of the dependencies we include in the
> > >>> shaded
> > >>>>>>>> jars.
> > >>>>>>>>>>>>>>>>>>>      => Not a problem for Guava (Apache Licensed)
> > >>>>>>>>>>>>>>>>>>>      => I think is a problem for ASM (redistribution
> > >> in
> > >>>>>>>>> binary
> > >>>>>>>>>>>> form,
> > >>>>>>>>>>>>>>>>>>> hence needs a notice of the copy)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>   - The Table API / SQL module needs more entries for
> > >>>>>>>> Janino
> > >>>>>>>>> /
> > >>>>>>>>>>>>>>>>>>> Reflections (both BSD licensed)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> So that is definitely a blocker.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Mon, May 8, 2017 at 12:14 PM, Robert Metzger <
> > >>>>>>>>>>>> rmetz...@apache.org
> > >>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Hi Devs,
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I've created a first non-voting release candidate
> for
> > >>>>>>> Flink
> > >>>>>>>>>> 1.3.0.
> > >>>>>>>>>>>>>>>>>>>> Please use this RC to test as much as you can and
> > >>> provide
> > >>>>>>>>>> feedback
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> the Flink community. The more we find and fix now,
> > >> the
> > >>>>>>>> better
> > >>>>>>>>>>>> Flink
> > >>>>>>>>>>>>>> 1.3.0
> > >>>>>>>>>>>>>>>>>>>> wil be :)
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I've CC'ed the user@ mailing list to get more
> people
> > >>> to
> > >>>>>>>> test
> > >>>>>>>>>> it.
> > >>>>>>>>>>>> DO
> > >>>>>>>>>>>>>>>>>>>> NOT USE THIS RELEASE CANDIDATE IN PRODUCTION.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I will prepare a google document to synchronize the
> > >>>>>>> testing
> > >>>>>>>>>>>> effort a
> > >>>>>>>>>>>>>>>>>>>> bit more.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Depending on the number of issues we identify, I
> hope
> > >>>> that
> > >>>>>>>> we
> > >>>>>>>>>> can
> > >>>>>>>>>>>> do
> > >>>>>>>>>>>>>>>>>>>> the first VOTEing RC early next week.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> ------------------------------
> > >>>>>>>> ------------------------------
> > >>>>>>>>>>>>>> ---------
> > >>>>>>>>>>>>>>>>>>>> The release commit is
> > >>>>>>>>> f94c002991dcce9f1104f8e61b43efb2f8247cb4,
> > >>>>>>>>>>>>>>>>>>>> located here:
> > >>>>>>> http://git-wip-us.apache.org/repos/asf/flink/
> > >>>>>>>>>>>>>>>>>>>> commit/f94c0029
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> The artifacts are located here:
> http://people.apache
> > >> .
> > >>>>>>>>>>>>>>>>>>>> org/~rmetzger/flink-1.3.0-rc0/
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> The maven staging repository is here:
> > >>> https://repository
> > >>>> .
> > >>>>>>>>>>>>>>>>>>>> apache.org/content/repositories/orgapacheflink-1118
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> ------------------------------
> > >>>>>>>> ------------------------------
> > >>>>>>>>>>>>>> ---------
> > >>>>>>>>>>>>>>>>>>>> Happy testing!
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>> Robert
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>> Liu, Renjie
> > >>>>>>>>>>>>>>>>> Software Engineer, MVAD
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>> Liu, Renjie
> > >>>>>>>>>>>>>>> Software Engineer, MVAD
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>> --
> > >>>>>>>> Liu, Renjie
> > >>>>>>>> Software Engineer, MVAD
> > >>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>>>>
> > >>>>
> > >>> --
> > >>> Liu, Renjie
> > >>> Software Engineer, MVAD
> > >>>
> > >>
> >
> >
>

Reply via email to