Hi, I would also include the following (not yet resolved) issue in the 1.2.1 scope :
https://issues.apache.org/jira/browse/FLINK-6001 NPE on TumblingEventTimeWindows with ContinuousEventTimeTrigger and allowedLateness 2017-03-14 17:34 GMT+01:00 Ufuk Celebi <u...@apache.org>: > Big +1 Gordon! > > I think (10) is very critical to have in 1.2.1. > > – Ufuk > > > On Tue, Mar 14, 2017 at 3:37 PM, Stefan Richter > <s.rich...@data-artisans.com> wrote: > > Hi, > > > > I would suggest to also include in 1.2.1: > > > > (9) https://issues.apache.org/jira/browse/FLINK-6044 < > https://issues.apache.org/jira/browse/FLINK-6044> > > Replaces unintentional calls to InputStream#read(…) with the intended > > and correct InputStream#readFully(…) > > Status: PR > > > > (10) https://issues.apache.org/jira/browse/FLINK-5985 < > https://issues.apache.org/jira/browse/FLINK-5985> > > Flink 1.2 was creating state handles for stateless tasks which caused > trouble > > at restore time for users that wanted to do some changes that only > include > > stateless operators to their topology. > > Status: PR > > > > > >> Am 14.03.2017 um 15:15 schrieb Till Rohrmann <trohrm...@apache.org>: > >> > >> Thanks for kicking off the discussion Tzu-Li. I'd like to add the > following > >> issues which have already been merged into the 1.2-release and > 1.1-release > >> branch: > >> > >> 1.2.1: > >> > >> (7) https://issues.apache.org/jira/browse/FLINK-5942 > >> Hardens the checkpoint recovery in case of corrupted ZooKeeper data. > >> Corrupted checkpoints will now be skipped. > >> Status: Merged > >> > >> (8) https://issues.apache.org/jira/browse/FLINK-5940 > >> Hardens the checkpoint recovery in case that we cannot retrieve the > >> completed checkpoint from the meta data state handle retrieved from > >> ZooKeeper. This can, for example, happen if the meta data is deleted. > >> Checkpoints with unretrievable state handles are skipped. > >> Status: Merged > >> > >> 1.1.5: > >> > >> > >> (7) https://issues.apache.org/jira/browse/FLINK-5942 > >> Hardens the checkpoint recovery in case of corrupted ZooKeeper data. > >> Corrupted checkpoints will now be skipped. > >> Status: Merged > >> > >> (8) https://issues.apache.org/jira/browse/FLINK-5940 > >> Hardens the checkpoint recovery in case that we cannot retrieve the > >> completed checkpoint from the meta data state handle retrieved from > >> ZooKeeper. This can, for example, happen if the meta data is deleted. > >> Checkpoints with unretrievable state handles are skipped. > >> Status: Merged > >> > >> Cheers, > >> Till > >> > >> On Tue, Mar 14, 2017 at 12:02 PM, Tzu-Li (Gordon) Tai < > tzuli...@apache.org> > >> wrote: > >> > >>> Hi all! > >>> > >>> I would like to start a discussion for the next bugfix release for > 1.1.x > >>> and 1.2.x. > >>> There’s been quite a few critical fixes for bugs in both the releases > >>> recently, and I think they deserve a bugfix release soon. > >>> Most of the bugs were reported by users. > >>> > >>> I’m starting the discussion for both bugfix releases because most fixes > >>> span both releases (almost identical). > >>> Of course, the actual RC votes and RC creation process doesn’t have to > be > >>> started together. > >>> > >>> Here’s an overview of what’s been collected so far, for both bugfix > >>> releases - > >>> (it’s a list of what I’m aware of so far, and may be missing stuff; > please > >>> append and bring to attention as necessary :-) ) > >>> > >>> > >>> For Flink 1.2.1: > >>> > >>> (1) https://issues.apache.org/jira/browse/FLINK-5701: > >>> Async exceptions in the FlinkKafkaProducer are not checked on > checkpoints. > >>> This compromises the producer’s at-least-once guarantee. > >>> Status: merged > >>> > >>> (2) https://issues.apache.org/jira/browse/FLINK-5949: > >>> Do not check Kerberos credentials for non-Kerberos authentications. > MapR > >>> users are affected by this, and cannot submit Flink on YARN jobs on a > >>> secured MapR cluster. > >>> Status: PR - https://github.com/apache/flink/pull/3528, one +1 already > >>> > >>> (3) https://issues.apache.org/jira/browse/FLINK-6006: > >>> Kafka Consumer can lose state if queried partition list is incomplete > on > >>> restore. > >>> Status: PR - https://github.com/apache/flink/pull/3505, one +1 already > >>> > >>> (4) https://issues.apache.org/jira/browse/FLINK-6025: > >>> KryoSerializer may use the wrong classloader when Kryo’s > JavaSerializer is > >>> used. > >>> Status: merged > >>> > >>> (5) https://issues.apache.org/jira/browse/FLINK-5771: > >>> Fix multi-char delimiters in Batch InputFormats. > >>> Status: merged > >>> > >>> (6) https://issues.apache.org/jira/browse/FLINK-5934: > >>> Set the Scheduler in the ExecutionGraph via its constructor. This > fixes a > >>> bug that causes HA recovery to fail. > >>> Status: merged > >>> > >>> > >>> > >>> For Flink 1.1.5: > >>> > >>> (1) https://issues.apache.org/jira/browse/FLINK-5701: > >>> Async exceptions in the FlinkKafkaProducer are not checked on > checkpoints. > >>> This compromises the producer’s at-least-once guarantee. > >>> Status: This is already merged for 1.2.1. I would personally like to > >>> backport the fix for this to 1.1.5 also. > >>> > >>> (2) https://issues.apache.org/jira/browse/FLINK-6006: > >>> Kafka Consumer can lose state if queried partition list is incomplete > on > >>> restore. > >>> Status: PR - https://github.com/apache/flink/pull/3507, one +1 already > >>> > >>> (3) https://issues.apache.org/jira/browse/FLINK-6025: > >>> KryoSerializer may use the wrong classloader when Kryo’s > JavaSerializer is > >>> used. > >>> Status: merged > >>> > >>> (4) https://issues.apache.org/jira/browse/FLINK-5771: > >>> Fix multi-char delimiters in Batch InputFormats. > >>> Status: merged > >>> > >>> (5) https://issues.apache.org/jira/browse/FLINK-5934: > >>> Set the Scheduler in the ExecutionGraph via its constructor. This > fixes a > >>> bug that causes HA recovery to fail. > >>> Status: merged > >>> > >>> (6) https://issues.apache.org/jira/browse/FLINK-5048: > >>> Kafka Consumer (0.9/0.10) threading model leads problematic > cancellation > >>> behavior. > >>> Status: This fix was already released in 1.2.0, but never made it into > the > >>> 1.1.x bugfixes. Do we want to backport this also for 1.1.5? > >>> > >>> > >>> What do you think? From the list so far, we pretty much already have > >>> everything in, so I think it would be nice to aim for RCs by the end of > >>> this week. > >>> Since both bugfix releases cover almost the same list of issues, I > think > >>> it shouldn’t be too hard for us to kick off both bugfix releases > around the > >>> same time. > >>> > >>> Also FYI, here’s the lists of JIRA tickets tagged with "1.2.1” / > “1.1.5” > >>> as the Fix Versions, and are still open. > >>> We should probably want to check if there’s anything on there that we > >>> should block on for the releases: > >>> > >>> For 1.2.1: > >>> https://issues.apache.org/jira/browse/FLINK-5711?jql= > >>> project%20%3D%20FLINK%20AND%20status%20in%20(Open%2C%20% > >>> 22In%20Progress%22%2C%20Reopened)%20AND%20fixVersion%20%3D%201.2.1 > >>> > >>> For 1.1.5: > >>> https://issues.apache.org/jira/browse/FLINK-6006?jql= > >>> project%20%3D%20FLINK%20AND%20status%20in%20(Open%2C%20% > >>> 22In%20Progress%22%2C%20Reopened)%20AND%20fixVersion%20%3D%201.1.5 > > >