The last fixes are finally in. Thanks to everyone who participated in the 
discussion.

I will now create the release artifacts and start the vote tomorrow (CET).

– Ufuk

On 8 November 2016 at 19:02:46, Stephan Ewen (se...@apache.org) wrote:
> I opened a pull request for the backport of [FLINK-4904]
>  
>  
> https://github.com/apache/flink/pull/2773
>  
>  
> On Tue, Nov 8, 2016 at 2:00 PM, Stephan Ewen wrote:
>  
> > The issue FLINK-4904 (Add a limit for how much data may be spilled in
> > checkpoint alignments) is doen for master and I am currently backporting
> > it. Hope to finish that this week...
> >
> > Stephan
> >
> >
> > On Wed, Nov 2, 2016 at 5:03 PM, Till Rohrmann  
> > wrote:
> >
> >> It might make sense to backport
> >>
> >> - [FLINK-4944] Replace Akka's death watch with own heartbeat on the TM
> >> side: https://github.com/apache/flink/pull/2742
> >>
> >> as well. This will allow us to activate the quarantine monitoring per
> >> default in 1.1.4 without risking to kill all TMs in case of a JM failure.
> >>
> >> Cheers,
> >> Till
> >>
> >> On Wed, Nov 2, 2016 at 11:43 AM, Ufuk Celebi wrote:
> >>
> >> > As a quick update: the "pending review" issues have all been resolved.
> >> >
> >> > The open issues are still open:
> >> >
> >> > - FLINK-4904: Add a limit for how much data may be spilled in
> >> > checkpoint alignments => fix pending
> >> > - FLINK-4910: Introduce safety net for closing file system streams
> >> >
> >> > Any updates here?
> >> >
> >> > – Ufuk
> >> >
> >> >
> >> > On Fri, Oct 28, 2016 at 5:45 PM, Stefan Richter
> >> > wrote:
> >> > > Benefit of a backport, as I see it, is increased stability. The danger
> >> > is potentially breaking some code that was casting FileSystems to
> >> subtypes
> >> > like LocalFileSytem. I don’t know how common that would be in user code.
> >> > >
> >> > >> Am 28.10.2016 um 14:27 schrieb Ufuk Celebi :
> >> > >>
> >> > >> Thanks for all your feedback.
> >> > >>
> >> > >> If there are no objections, I would like to stick to the mentioned
> >> > >> issues in this thread and create RC1 as soon as they are all
> >> > >> addressed. This will probably not be this week though, but it looks
> >> > >> good for next week.
> >> > >>
> >> > >> DONE
> >> > >> =====
> >> > >> - FLINK-4619: Answer client if savepoint restore fails
> >> > >> - FLINK-4715: Safety net for stuck task cancellation
> >> > >> - FLINK-4510: Always create CheckpointCoordinator
> >> > >> - FLINK-4894: Don't block on buffer request after broadcast event
> >> > >> - FLINK-4298: Add proper repository for Closure dependencies
> >> > >> - FLINK-4218: Do not fail checkpoints when state size cannot be
> >> > determined
> >> > >> - FLINK-3347: TaskManager (or its ActorSystem) need to restart in
> >> case
> >> > >> they notice quarantine
> >> > >> - FLINK-4875: Use correct operator name
> >> > >> - FLINK-4913: Include user jars in system class loader
> >> > >>
> >> > >> PENDING REVIEW
> >> > >> ===============
> >> > >> - FLINK-4445: Add option to ignore unmatched state when restoring
> >> from
> >> > >> savepoint => https://github.com/apache/flink/pull/2713
> >> > >> - FLINK-4932: Don't let ExecutionGraph fail when in state Restarting
> >> > >> => https://github.com/apache/flink/pull/2711
> >> > >> - FLINK-4933: ExecutionGraph.scheduleOrUpdateConsumers can fail the
> >> > >> ExecutionGraph => https://github.com/apache/flink/pull/2701
> >> > >>
> >> > >> OPEN
> >> > >> =====
> >> > >> - FLINK-4904: Add a limit for how much data may be spilled in
> >> > >> checkpoint alignments => fix pending
> >> > >> - FLINK-4910: Introduce safety net for closing file system streams =>
> >> > >> @Stephan, Stefan: What's the conclusion of your discussion whether to
> >> > >> backport this or not?
> >> > >>
> >> > >>
> >> > >> On Wed, Oct 26, 2016 at 9:57 PM, dan bress  
> >> wrote:
> >> > >>> +1 for this release,
> >> > >>> also +1 to Chesnay's suggesting for including this: [FLINK-4875]
> >> > [metrics]
> >> > >>> Use correct operator name
> >> > >>>
> >> > >>> Dan
> >> > >>>
> >> > >>> On Wed, Oct 26, 2016 at 5:06 AM Till Rohrmann > >> >
> >> > wrote:
> >> > >>>
> >> > >>>> I'll work on FLINK-3347. Additionally I would like to get in
> >> > >>>>
> >> > >>>> - https://issues.apache.org/jira/browse/FLINK-4932: Don't let
> >> > >>>> ExecutionGraph fail when in state Restarting
> >> > >>>> - https://issues.apache.org/jira/browse/FLINK-4933:
> >> > >>>> ExecutionGraph.scheduleOrUpdateConsumers
> >> > >>>> can fail the ExecutionGraph
> >> > >>>>
> >> > >>>> Cheers,
> >> > >>>> Till
> >> > >>>>
> >> > >>>> On Wed, Oct 26, 2016 at 1:02 PM, Stephan Ewen  
> >> > wrote:
> >> > >>>>
> >> > >>>>> Concerning backporting the "I/O streams safety net" - we need to
> >> make
> >> > >>>> sure
> >> > >>>>> that this does not change any behavior that users may implicitly
> >> > expect.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Wed, Oct 26, 2016 at 11:21 AM, Maximilian Michels <
> >> m...@apache.org
> >> > >
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>> +1 for a 1.1.4 release
> >> > >>>>>>
> >> > >>>>>> We could backport putting user jars into the system class loader
> >> for
> >> > >>>>>> per-job Yarn clusters: https://github.com/apache/flink/pull/2692
> >> > >>>>>> Arguably, this is somewhat a new feature but it gets rid of
> >> > duplicate
> >> > >>>>>> class loading issues users experienced in practice.
> >> > >>>>>>
> >> > >>>>>> We already have the following commits on the release-1.1 branch:
> >> > >>>>>>
> >> > >>>>>> 05a5f46 [FLINK-4862] fix Timer register in
> >> > ContinuousEventTimeTrigger
> >> > >>>>>> 5731672 [FLINK-4581] [table] Fix Table API throwing "No suitable
> >> > driver
> >> > >>>>>> found for jdbc:calcite"
> >> > >>>>>> 9c87f92 [FLINK-4586] [core] Broken AverageAccumulator
> >> > >>>>>> 210230c [FLINK-4829] snapshot accumulators on a best-effort basis
> >> > >>>>>> c1d6b24 [FLINK-4829] protect user accumulators against concurrent
> >> > >>>> updates
> >> > >>>>>> fe464b4 [FLINK-4709] [core] Fix resource leak in
> >> > >>>>> InputStreamFSInputWrapper
> >> > >>>>>> 9f72698 [FLINK-4108] [scala] Respect ResultTypeQueryable for
> >> > >>>>> InputFormats.
> >> > >>>>>> 9591d50 [FLINK-4506] [DataSet] Fix documentation of
> >> CsvOutputFormat
> >> > >>>> about
> >> > >>>>>> incorrect default of allowNullValues
> >> > >>>>>> c9433bf [FLINK-3706] Fix YARN test instability
> >> > >>>>>> 2203f74 [FLINK-4778] [docs] Fix WordCount parameters in CLI
> >> > examples.
> >> > >>>>>>
> >> > >>>>>> -Max
> >> > >>>>>>
> >> > >>>>>>
> >> > >>>>>> On Wed, Oct 26, 2016 at 7:05 AM, Jean-Baptiste Onofré <
> >> > j...@nanthrax.net
> >> > >>>>>
> >> > >>>>>> wrote:
> >> > >>>>>>> +1
> >> > >>>>>>>
> >> > >>>>>>> Looking forward this release !
> >> > >>>>>>>
> >> > >>>>>>> Regards
> >> > >>>>>>> JB
> >> > >>>>>>>
> >> > >>>>>>> ⁣
> >> > >>>>>>>
> >> > >>>>>>> On Oct 25, 2016, 14:43, at 14:43, Robert Metzger <
> >> > >>>> rmetz...@apache.org>
> >> > >>>>>> wrote:
> >> > >>>>>>>> +1 for a bugfix release soon.
> >> > >>>>>>>>
> >> > >>>>>>>> On Tue, Oct 25, 2016 at 10:53 AM, Stephan Ewen <
> >> se...@apache.org>
> >> > >>>>>>>> wrote:
> >> > >>>>>>>>
> >> > >>>>>>>>> Thanks fort starting this Ufuk.
> >> > >>>>>>>>>
> >> > >>>>>>>>> I would like to add the following issues to 1.1.4:
> >> > >>>>>>>>>
> >> > >>>>>>>>> Build errors due to Storm dependencies *(fix pending)*
> >> > >>>>>>>>> - [FLINK-4298] [storm compatibility] Add proper repository
> >> for
> >> > >>>>>>>> Closure
> >> > >>>>>>>>> dependencies.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Stability on S3 considering eventual consistency *(fix
> >> pending)*
> >> > >>>>>>>>> - [FLINK-4218] [checkpoints] Do not fail checkpoints when
> >> > state
> >> > >>>>>>>> size
> >> > >>>>>>>>> cannot be determined
> >> > >>>>>>>>>
> >> > >>>>>>>>> Avoiding Zombie TaskManagers *(still needs to be done)*
> >> > >>>>>>>>> - [FLINK-3347] [akka] TaskManager (or its ActorSystem)
> >> need to
> >> > >>>>>>>> restart
> >> > >>>>>>>>> in case they notice quarantine
> >> > >>>>>>>>>
> >> > >>>>>>>>> Adding a limit to the amount of data spilled during checkpoint
> >> > >>>>>>>> alignments
> >> > >>>>>>>>> *(fix
> >> > >>>>>>>>> is work in progress)*
> >> > >>>>>>>>> - [FLINK-4904] [checkpoints] Add a limit for how much data
> >> may
> >> > >>>> be
> >> > >>>>>>>>> spilled in checkpoint alignments
> >> > >>>>>>>>>
> >> > >>>>>>>>>
> >> > >>>>>>>>> I can push the first two fixes to the 1.1.4 branch in a bit,
> >> the
> >> > >>>>>>>> fourth one
> >> > >>>>>>>>> later today.
> >> > >>>>>>>>> The third one (akka) is still pending.
> >> > >>>>>>>>>
> >> > >>>>>>>>> Best,
> >> > >>>>>>>>> Stephan
> >> > >>>>>>>>>
> >> > >>>>>>>>>
> >> > >>>>>>>>>
> >> > >>>>>>>>> On Mon, Oct 24, 2016 at 3:32 PM, Ufuk Celebi  
> >> > >>>> wrote:
> >> > >>>>>>>>>
> >> > >>>>>>>>>> Hey all,
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> I would like to start the discussion for kicking off the next
> >> > bug
> >> > >>>>>>>> fix
> >> > >>>>>>>>>> release, Flink 1.1.4. What do you think about aiming for a
> >> RC by
> >> > >>>>>>>> end
> >> > >>>>>>>>>> of this week?
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> Users reported some instabilities/inconveniences that would
> >> be
> >> > >>>> good
> >> > >>>>>>>> to
> >> > >>>>>>>>> fix.
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> Personally, I would like to backport the following fixes:
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> (1) https://issues.apache.org/jira/browse/FLINK-4619: Answer  
> >> > >>>>> client
> >> > >>>>>>>> if
> >> > >>>>>>>>>> savepoint restore fails (Already merged for master, needs
> >> > minimal
> >> > >>>>>>>>>> adjustment for 1.1)
> >> > >>>>>>>>>> (2) https://issues.apache.org/jira/browse/FLINK-4715: Safety  
> >> > net
> >> > >>>>>>>> for
> >> > >>>>>>>>>> stuck task cancellation (Already reviewed for master, waiting
> >> > for
> >> > >>>>>>>>>> tests to finish of backport)
> >> > >>>>>>>>>> (3) https://issues.apache.org/jira/browse/FLINK-4510: Always  
> >> > >>>>> create
> >> > >>>>>>>>>> CheckpointCoordinator (Already merged for master, needs
> >> minimal
> >> > >>>>>>>>>> adjustments for 1.1)
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> Furthermore, I would like to address the following:
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> (4) https://issues.apache.org/jira/browse/FLINK-4445: Add
> >> > option
> >> > >>>>> to
> >> > >>>>>>>>>> ignore unmatched state when restoring from savepoint
> >> > >>>>>>>>>> (5) https://issues.apache.org/jira/browse/FLINK-4894: Don't
> >> > >>>> block
> >> > >>>>>>>> on
> >> > >>>>>>>>>> buffer request after broadcast event
> >> > >>>>>>>>>>
> >> > >>>>>>>>>> Strictly speaking, the (4) is not a bug fix. But given that
> >> it
> >> > >>>>>>>> would
> >> > >>>>>>>>>> only add an optional flag to savepoint restoring and should
> >> have
> >> > >>>>>>>> been
> >> > >>>>>>>>>> addressed for 1.1.0 already, I would like to get it in.
> >> > >>>>>>>>>>
> >> > >>>>>>>>>
> >> > >>>>>>
> >> > >>>>>
> >> > >>>>
> >> > >
> >> >
> >>
> >
> >
>  

Reply via email to