You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Piotr Nowojski <pn...@apache.org> on 2020/06/17 13:29:04 UTC

[ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi all,

I would like to give an update about the RC2 status. We are now waiting for
a green azure build on one final bug fix before creating RC2. This bug fix
should be merged late afternoon/early evening Berlin time, so RC2 will be
hopefully created tomorrow morning. Until then I would ask to not
merge/backport commits to release-1.11 branch, including bug fixes. If you
have something that's truly essential and should be treated as a release
blocker, please reach out to me or Zhijiang.

Best,
Piotr Nowojski

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Robert Metzger <rm...@apache.org>.
Thanks a lot for creating another RC.

The good news first: WordCount works on my Mac :)

I didn't find this information in the email: This is the commit the RC is
based on:
https://github.com/apache/flink/commit/c4132de4a50ab9b8f653c69af1ba15af44ff29a2

Additionally, I've created [1] a docker image based on the "
flink-1.11.0-bin-scala_2.12.tgz
<https://dist.apache.org/repos/dist/dev/flink/flink-1.11.0-rc2/flink-1.11.0-bin-scala_2.12.tgz>"
binary release candidate. It's on DockerHub:
"rmetzger/flink:1.11.0-rc2-c4132de4a50ab9b8f653c69af1ba15af44ff29a2".

Happy testing :)

[1]
https://github.com/rmetzger/flink-docker-factory/runs/785600229?check_suite_focus=true

On Thu, Jun 18, 2020 at 2:41 PM Piotr Nowojski <pn...@apache.org> wrote:

> Hi all,
>
> Apache Flink-1.11.0-RC2 has been created. It has all the artifacts that we
> would typically have for a release.
>
> This RC might have still a couple of missing licensing notices (like the
> one mentioned by Jingsong Li), but as for today morning, there were no open
> release blocking bugs. No official vote will take place for it, but you can
> treat it as a solid base for release testing. It includes the following:
>
>   * The preview source release and binary convenience releases [1], which
> are signed with the key with fingerprint
> 2DA85B93244FDFA19A6244500653C0A2CEA00D0E [2],
>   * All artifacts that would normally be deployed to the Maven Central
> Repository [3]
>
> To test with these artifacts, you can create a settings.xml file with the
> content shown below [4]. This settings file can be referenced in your maven
> commands
> via --settings /path/to/settings.xml. This is useful for creating a
> quickstart project based on the staged release and also for building
> against the staged jars.
>
> Happy testing!
>
> Best,
> Piotrek
>
> [1] https://dist.apache.org/repos/dist/dev/flink/flink-1.11.0-rc2/
> [2] https://dist.apache.org/repos/dist/release/flink/KEYS
> [3]
> https://repository.apache.org/content/repositories/orgapacheflink-1374/
> [4]
> <settings>
>    <activeProfiles>
>         <activeProfile>flink-1.11.0</activeProfile>
>    </activeProfiles>
>    <profiles>
>        <profile>
>            <id>flink-1.11.0</id>
>            <repositories>
>                  <repository>
>                        <id>flink-1.11.0</id>
>                        <url>
> https://repository.apache.org/content/repositories/orgapacheflink-1374/
> </url>
>                 </repository>
>                 <repository>
>                       <id>archetype</id>
>                       <url>
> https://repository.apache.org/content/repositories/orgapacheflink-1374/
> </url>
>                 </repository>
>             </repositories>
>        </profile>
>    </profiles>
> </settings>
>
> śr., 17 cze 2020 o 15:29 Piotr Nowojski <pn...@apache.org> napisał(a):
>
> > Hi all,
> >
> > I would like to give an update about the RC2 status. We are now waiting
> > for a green azure build on one final bug fix before creating RC2. This
> bug
> > fix should be merged late afternoon/early evening Berlin time, so RC2
> will
> > be hopefully created tomorrow morning. Until then I would ask to not
> > merge/backport commits to release-1.11 branch, including bug fixes. If
> you
> > have something that's truly essential and should be treated as a release
> > blocker, please reach out to me or Zhijiang.
> >
> > Best,
> > Piotr Nowojski
> >
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Piotr Nowojski <pn...@apache.org>.
Hi all,

Apache Flink-1.11.0-RC2 has been created. It has all the artifacts that we
would typically have for a release.

This RC might have still a couple of missing licensing notices (like the
one mentioned by Jingsong Li), but as for today morning, there were no open
release blocking bugs. No official vote will take place for it, but you can
treat it as a solid base for release testing. It includes the following:

  * The preview source release and binary convenience releases [1], which
are signed with the key with fingerprint
2DA85B93244FDFA19A6244500653C0A2CEA00D0E [2],
  * All artifacts that would normally be deployed to the Maven Central
Repository [3]

To test with these artifacts, you can create a settings.xml file with the
content shown below [4]. This settings file can be referenced in your maven
commands
via --settings /path/to/settings.xml. This is useful for creating a
quickstart project based on the staged release and also for building
against the staged jars.

Happy testing!

Best,
Piotrek

[1] https://dist.apache.org/repos/dist/dev/flink/flink-1.11.0-rc2/
[2] https://dist.apache.org/repos/dist/release/flink/KEYS
[3] https://repository.apache.org/content/repositories/orgapacheflink-1374/
[4]
<settings>
   <activeProfiles>
        <activeProfile>flink-1.11.0</activeProfile>
   </activeProfiles>
   <profiles>
       <profile>
           <id>flink-1.11.0</id>
           <repositories>
                 <repository>
                       <id>flink-1.11.0</id>
                       <url>
https://repository.apache.org/content/repositories/orgapacheflink-1374/
</url>
                </repository>
                <repository>
                      <id>archetype</id>
                      <url>
https://repository.apache.org/content/repositories/orgapacheflink-1374/
</url>
                </repository>
            </repositories>
       </profile>
   </profiles>
</settings>

śr., 17 cze 2020 o 15:29 Piotr Nowojski <pn...@apache.org> napisał(a):

> Hi all,
>
> I would like to give an update about the RC2 status. We are now waiting
> for a green azure build on one final bug fix before creating RC2. This bug
> fix should be merged late afternoon/early evening Berlin time, so RC2 will
> be hopefully created tomorrow morning. Until then I would ask to not
> merge/backport commits to release-1.11 branch, including bug fixes. If you
> have something that's truly essential and should be treated as a release
> blocker, please reach out to me or Zhijiang.
>
> Best,
> Piotr Nowojski
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Thomas Weise <th...@apache.org>.
Hi Till,

Thanks for taking a look and for opening the PR.

Previous options that specified a "pause" seem to be semantically different
from new options that specify a "timeout", so probably best to remove them
from code and documentation (old options are also still listed on
https://ci.apache.org/projects/flink/flink-docs-stable/ops/config.html)

Thomas



On Wed, Jun 24, 2020 at 1:36 AM Till Rohrmann <tr...@apache.org> wrote:

> Thanks for testing the RC and the feedback Thomas. The problem with the
> taskmanager options is that the old
> (taskmanager.initial-registration-pause) and new options
> (cluster.registration.initial-timeout) don't have the same type. The old
> options have not been used for a long time (since version 1.5.0) and we
> wanted to remove them. As part of the removal, we added the old keys as
> deprecated options for the new ones. I believe this was a mistake. I've
> opened a PR to remove the deprecated keys from the new ConfigOptions [1].
>
> Please be aware that
>
> "taskmanager.initial-registration-pause": "500ms",
> "taskmanager.max-registration-pause": "5s",
> "taskmanager.refused-registration-pause": "5s",
>
> Shouldn't have any effects anymore (since version 1.5.0).
>
> [1] https://github.com/apache/flink/pull/12763
>
> Cheers,
> Till
>
> On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wangzhijiang999@aliyun.com
> .invalid>
> wrote:
>
> > Hi Thomas,
> >
> > Thanks for these valuable feedbacks and suggestions, and I think they are
> > very helpful for making us better.
> >
> > I can give an direct answer for this issue:
> > > checkpoint alignment buffered metric missing - note that this job isn't
> > using the new unaligned checkpointing that should be opt-in.
> >
> > The metric of checkpoint alignment buffered would be always 0 now, no
> > matter with unaligned checkpointing or not, so we removed this metric
> > directly.
> > The motivation for such change is from reducing in-flight buffers to
> speed
> > up checkpoint somehow. The upstream side would block sending any
> following
> > buffers after sending the barrier until receiving the alignment
> > notification from downstream side. Therefore, the downstream side never
> > needs to cache
> > buffers for blocked channels during alignment. We also illustrated such
> > changes in release notes for attention by link [1].
> >
> > [1]
> >
> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
> >
> > Best,
> > Zhijiang
> > ------------------------------------------------------------------
> > From:Thomas Weise <th...@apache.org>
> > Send Time:2020年6月24日(星期三) 06:51
> > To:dev <de...@flink.apache.org>
> > Cc:zhijiang <zh...@apache.org>
> > Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
> >
> > Hi,
> >
> > Thanks for putting together the RC!
> >
> > I have some preliminary feedback from testing with commit
> > 934f91ead00fd658333f65ffa37ab60bd5ffd99b
> >
> > An internal benchmark application that reads from Kinesis and checkpoints
> > ~12GB performs comparably to 1.10.1
> >
> > There were a few issues hit upgrading our codebase that may be worthwhile
> > considering, please see details below.
> >
> > Given my observations over the past few releases, I would like to suggest
> > that the community introduces a log of incompatible changes to be
> published
> > with the release notes. Though it is possible to analyze git history when
> > hitting compile errors, there are more subtle changes that can make
> > upgrades unnecessarily time-consuming. Contributors introducing such
> > changes are probably in the best position to document.
> >
> > I'm planning to try this or the next RC with a couple more applications.
> >
> > Cheers,
> > Thomas
> >
> > * notifyCheckpointAborted needed to be implemented
> > for org.apache.flink.runtime.state.CheckpointListener - can we have the
> > default implementation in the interface so that users aren't forced to
> > change their implementations
> >
> > * following deprecated configuration values had to be modified to get
> > the job running:
> >
> >           "taskmanager.initial-registration-pause": "500ms",
> >           "taskmanager.max-registration-pause": "5s",
> >           "taskmanager.refused-registration-pause": "5s",
> >
> > The error message was:
> >
> > Could not parse value '500ms' for key
> > 'cluster.registration.initial-timeout'.\n\tat
> >
> >
> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
> >
> >
> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
> >
> >
> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
> >
> >
> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
> >
> > Though easy to fix, it's unfortunate that values are now treated
> > differently.
> >
> > * checkpoint alignment buffered metric missing - note that this job isn't
> > using the new unaligned checkpointing that should be opt-in.
> >
> > * -import org.apache.flink.table.api.java.StreamTableEnvironment;
> >   +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> >
> >  * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> > config, program.build());
> >     +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> > config, program.build(),
> >               false, false);
> >
> > * ProcessingTimeCallback removed from StreamingFileSink
> >
> >
> > On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
> > wrote:
> >
> > > Hi all,
> > >
> > > I would like to give an update about the RC2 status. We are now waiting
> > for
> > > a green azure build on one final bug fix before creating RC2. This bug
> > fix
> > > should be merged late afternoon/early evening Berlin time, so RC2 will
> be
> > > hopefully created tomorrow morning. Until then I would ask to not
> > > merge/backport commits to release-1.11 branch, including bug fixes. If
> > you
> > > have something that's truly essential and should be treated as a
> release
> > > blocker, please reach out to me or Zhijiang.
> > >
> > > Best,
> > > Piotr Nowojski
> > >
> >
> >
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Thomas Weise <th...@apache.org>.
-->

On Wed, Jun 24, 2020 at 8:24 AM Aljoscha Krettek <al...@apache.org>
wrote:

> Hi!
>
> On 24.06.20 00:51, Thomas Weise wrote:
> > * -import org.apache.flink.table.api.java.StreamTableEnvironment;
> >    +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
>
> This is very unfortunate yes, please see
>
> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R103
> for the reasoning behind it.
>

I also wish I had seen the release notes draft when I started looking at
the RC. In the future, it would be nice to have them as part of the
vote/announce thread.


> >   * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> > config, program.build());
> >      +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> > config, program.build(),
> >                false, false);
>
> The problem here is that ClientUtils is internal API (not annotated with
> @Public or @PublicEvolving). If it were public we would get automatic
> compatibility verification. Without verification all users can do is
> hope that the internal code they're using will not break. I don't like
> it but we could re-introduce the old method (which would be unused code
> in the Flink code base) but in the long run we need to make available
> @Public APIs for this if there is a user need.
>

IMO the root of the issue is that the current public API surface that comes
with compatibility guarantee is or at some point was not sufficient for
users and they end up resorting to the next best thing that they can find
in code search and tests. It might help to add documentation in cases like
this that redirects users to what we want them to use.


>
> > * ProcessingTimeCallback removed from StreamingFileSink
>
> How did you actually notice that one? It's more of an internal
> implementation detail.
>

It's an edge case. We have a decorator in our codebase that happened to
implement the same interfaces:

 public class AbstractSinkDecorator<U, T extends StreamingFileSink<U>>
    extends RichSinkFunction<U> implements SinkFunction<U>, RichFunction,
    CheckpointedFunction, CheckpointListener, ProcessingTimeCallback {



>
> Aljoscha
>
>
> On 24.06.20 17:12, Stephan Ewen wrote:
> > Hi!
> >
> > About the "notifyCheckpointAborted()":
> >
> > When I wrote that comment, I was (apparently wrongly) assuming we were
> > talking about an internal interface here, because the "abort" signal
> > was originally only intended to cancel the async part of state backend
> > checkpoints.
> >
> > I just realized that this is exposed to users - and I am actually with
> > Thomas on this one. The "CheckpointListener" is a very public interface
> > that many users implement. The fact that it is tagged "@PublicEvolving"
> is
> > somehow not aligned with reality. So adding the method here will in
> reality
> > break lots and lots of user programs.
> >
> > I think also in practice it is much less relevant for user applications
> to
> > react to aborted checkpoints. Since the notifications there can not be
> > relied upon (if there is a task failure concurrently) users always have
> to
> > follow the "newer checkpoint subsumes older checkpoint" contract, so the
> > abort method is probably rarely relevant.
> >
> > This is something we should change, in my opinion.
> >
> > Best,
> > Stephan
> >
> >
> >
> > On Wed, Jun 24, 2020 at 4:17 PM Piotr Nowojski <pi...@ververica.com>
> wrote:
> >
> >> Hi Thomas,
> >>
> >> Just one quick answer from my side about:
> >>
> >>> * notifyCheckpointAborted needed to be implemented
> >>> for org.apache.flink.runtime.state.CheckpointListener - can we have the
> >>> default implementation in the interface so that users aren't forced to
> >>> change their implementations
> >>
> >> This is intentional design [1]
> >>
> >>> Implementers should generally be forced to think about what to do when
> >> checkpoint is aborted.
> >>
> >> Piotrek
> >>
> >> [1] https://github.com/apache/flink/pull/8693#issuecomment-542834147
> >>
> >> Piotr Nowojski  | Staff Engineer
> >> +48 503 187 389
> >>
> >>
> >> Follow us @VervericaData
> >> --
> >> Join Flink Forward - The Apache Flink Conference
> >> Stream Processing | Event Driven | Real Time
> >> --
> >> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> >> --
> >> Ververica GmbH
> >> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> >> Managing Directors: Yip Park Tung Jason, Jinwei (Kevin) Zhang, Karl
> Anton
> >> Wehner
> >>
> >>
> >>
> >>
> >>> On 24 Jun 2020, at 10:35, Till Rohrmann <tr...@apache.org> wrote:
> >>>
> >>> Thanks for testing the RC and the feedback Thomas. The problem with the
> >>> taskmanager options is that the old
> >>> (taskmanager.initial-registration-pause) and new options
> >>> (cluster.registration.initial-timeout) don't have the same type. The
> old
> >>> options have not been used for a long time (since version 1.5.0) and we
> >>> wanted to remove them. As part of the removal, we added the old keys as
> >>> deprecated options for the new ones. I believe this was a mistake. I've
> >>> opened a PR to remove the deprecated keys from the new ConfigOptions
> [1].
> >>>
> >>> Please be aware that
> >>>
> >>> "taskmanager.initial-registration-pause": "500ms",
> >>> "taskmanager.max-registration-pause": "5s",
> >>> "taskmanager.refused-registration-pause": "5s",
> >>>
> >>> Shouldn't have any effects anymore (since version 1.5.0).
> >>>
> >>> [1] https://github.com/apache/flink/pull/12763
> >>>
> >>> Cheers,
> >>> Till
> >>>
> >>> On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wangzhijiang999@aliyun.com
> >> .invalid>
> >>> wrote:
> >>>
> >>>> Hi Thomas,
> >>>>
> >>>> Thanks for these valuable feedbacks and suggestions, and I think they
> >> are
> >>>> very helpful for making us better.
> >>>>
> >>>> I can give an direct answer for this issue:
> >>>>> checkpoint alignment buffered metric missing - note that this job
> isn't
> >>>> using the new unaligned checkpointing that should be opt-in.
> >>>>
> >>>> The metric of checkpoint alignment buffered would be always 0 now, no
> >>>> matter with unaligned checkpointing or not, so we removed this metric
> >>>> directly.
> >>>> The motivation for such change is from reducing in-flight buffers to
> >> speed
> >>>> up checkpoint somehow. The upstream side would block sending any
> >> following
> >>>> buffers after sending the barrier until receiving the alignment
> >>>> notification from downstream side. Therefore, the downstream side
> never
> >>>> needs to cache
> >>>> buffers for blocked channels during alignment. We also illustrated
> such
> >>>> changes in release notes for attention by link [1].
> >>>>
> >>>> [1]
> >>>>
> >>
> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
> >>>>
> >>>> Best,
> >>>> Zhijiang
> >>>> ------------------------------------------------------------------
> >>>> From:Thomas Weise <th...@apache.org>
> >>>> Send Time:2020年6月24日(星期三) 06:51
> >>>> To:dev <de...@flink.apache.org>
> >>>> Cc:zhijiang <zh...@apache.org>
> >>>> Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
> >>>>
> >>>> Hi,
> >>>>
> >>>> Thanks for putting together the RC!
> >>>>
> >>>> I have some preliminary feedback from testing with commit
> >>>> 934f91ead00fd658333f65ffa37ab60bd5ffd99b
> >>>>
> >>>> An internal benchmark application that reads from Kinesis and
> >> checkpoints
> >>>> ~12GB performs comparably to 1.10.1
> >>>>
> >>>> There were a few issues hit upgrading our codebase that may be
> >> worthwhile
> >>>> considering, please see details below.
> >>>>
> >>>> Given my observations over the past few releases, I would like to
> >> suggest
> >>>> that the community introduces a log of incompatible changes to be
> >> published
> >>>> with the release notes. Though it is possible to analyze git history
> >> when
> >>>> hitting compile errors, there are more subtle changes that can make
> >>>> upgrades unnecessarily time-consuming. Contributors introducing such
> >>>> changes are probably in the best position to document.
> >>>>
> >>>> I'm planning to try this or the next RC with a couple more
> applications.
> >>>>
> >>>> Cheers,
> >>>> Thomas
> >>>>
> >>>> * notifyCheckpointAborted needed to be implemented
> >>>> for org.apache.flink.runtime.state.CheckpointListener - can we have
> the
> >>>> default implementation in the interface so that users aren't forced to
> >>>> change their implementations
> >>>>
> >>>> * following deprecated configuration values had to be modified to get
> >>>> the job running:
> >>>>
> >>>>           "taskmanager.initial-registration-pause": "500ms",
> >>>>           "taskmanager.max-registration-pause": "5s",
> >>>>           "taskmanager.refused-registration-pause": "5s",
> >>>>
> >>>> The error message was:
> >>>>
> >>>> Could not parse value '500ms' for key
> >>>> 'cluster.registration.initial-timeout'.\n\tat
> >>>>
> >>>>
> >>
> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
> >>>>
> >>>>
> >>
> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
> >>>>
> >>>>
> >>
> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
> >>>>
> >>>>
> >>
> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
> >>>>
> >>>> Though easy to fix, it's unfortunate that values are now treated
> >>>> differently.
> >>>>
> >>>> * checkpoint alignment buffered metric missing - note that this job
> >> isn't
> >>>> using the new unaligned checkpointing that should be opt-in.
> >>>>
> >>>> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
> >>>>   +import
> org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> >>>>
> >>>> * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> >>>> config, program.build());
> >>>>     +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> >>>> config, program.build(),
> >>>>               false, false);
> >>>>
> >>>> * ProcessingTimeCallback removed from StreamingFileSink
> >>>>
> >>>>
> >>>> On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
> >>>> wrote:
> >>>>
> >>>>> Hi all,
> >>>>>
> >>>>> I would like to give an update about the RC2 status. We are now
> waiting
> >>>> for
> >>>>> a green azure build on one final bug fix before creating RC2. This
> bug
> >>>> fix
> >>>>> should be merged late afternoon/early evening Berlin time, so RC2
> will
> >> be
> >>>>> hopefully created tomorrow morning. Until then I would ask to not
> >>>>> merge/backport commits to release-1.11 branch, including bug fixes.
> If
> >>>> you
> >>>>> have something that's truly essential and should be treated as a
> >> release
> >>>>> blocker, please reach out to me or Zhijiang.
> >>>>>
> >>>>> Best,
> >>>>> Piotr Nowojski
> >>>>>
> >>>>
> >>>>
> >>
> >>
> >
>
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Aljoscha Krettek <al...@apache.org>.
Hi!

On 24.06.20 00:51, Thomas Weise wrote:
> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
>    +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;

This is very unfortunate yes, please see 
https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R103 
for the reasoning behind it.

>   * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> config, program.build());
>      +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> config, program.build(),
>                false, false);

The problem here is that ClientUtils is internal API (not annotated with 
@Public or @PublicEvolving). If it were public we would get automatic 
compatibility verification. Without verification all users can do is 
hope that the internal code they're using will not break. I don't like 
it but we could re-introduce the old method (which would be unused code 
in the Flink code base) but in the long run we need to make available 
@Public APIs for this if there is a user need.

> * ProcessingTimeCallback removed from StreamingFileSink

How did you actually notice that one? It's more of an internal 
implementation detail.

Aljoscha


On 24.06.20 17:12, Stephan Ewen wrote:
> Hi!
> 
> About the "notifyCheckpointAborted()":
> 
> When I wrote that comment, I was (apparently wrongly) assuming we were
> talking about an internal interface here, because the "abort" signal
> was originally only intended to cancel the async part of state backend
> checkpoints.
> 
> I just realized that this is exposed to users - and I am actually with
> Thomas on this one. The "CheckpointListener" is a very public interface
> that many users implement. The fact that it is tagged "@PublicEvolving" is
> somehow not aligned with reality. So adding the method here will in reality
> break lots and lots of user programs.
> 
> I think also in practice it is much less relevant for user applications to
> react to aborted checkpoints. Since the notifications there can not be
> relied upon (if there is a task failure concurrently) users always have to
> follow the "newer checkpoint subsumes older checkpoint" contract, so the
> abort method is probably rarely relevant.
> 
> This is something we should change, in my opinion.
> 
> Best,
> Stephan
> 
> 
> 
> On Wed, Jun 24, 2020 at 4:17 PM Piotr Nowojski <pi...@ververica.com> wrote:
> 
>> Hi Thomas,
>>
>> Just one quick answer from my side about:
>>
>>> * notifyCheckpointAborted needed to be implemented
>>> for org.apache.flink.runtime.state.CheckpointListener - can we have the
>>> default implementation in the interface so that users aren't forced to
>>> change their implementations
>>
>> This is intentional design [1]
>>
>>> Implementers should generally be forced to think about what to do when
>> checkpoint is aborted.
>>
>> Piotrek
>>
>> [1] https://github.com/apache/flink/pull/8693#issuecomment-542834147
>>
>> Piotr Nowojski  | Staff Engineer
>> +48 503 187 389
>>
>>
>> Follow us @VervericaData
>> --
>> Join Flink Forward - The Apache Flink Conference
>> Stream Processing | Event Driven | Real Time
>> --
>> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>> --
>> Ververica GmbH
>> Registered at Amtsgericht Charlottenburg: HRB 158244 B
>> Managing Directors: Yip Park Tung Jason, Jinwei (Kevin) Zhang, Karl Anton
>> Wehner
>>
>>
>>
>>
>>> On 24 Jun 2020, at 10:35, Till Rohrmann <tr...@apache.org> wrote:
>>>
>>> Thanks for testing the RC and the feedback Thomas. The problem with the
>>> taskmanager options is that the old
>>> (taskmanager.initial-registration-pause) and new options
>>> (cluster.registration.initial-timeout) don't have the same type. The old
>>> options have not been used for a long time (since version 1.5.0) and we
>>> wanted to remove them. As part of the removal, we added the old keys as
>>> deprecated options for the new ones. I believe this was a mistake. I've
>>> opened a PR to remove the deprecated keys from the new ConfigOptions [1].
>>>
>>> Please be aware that
>>>
>>> "taskmanager.initial-registration-pause": "500ms",
>>> "taskmanager.max-registration-pause": "5s",
>>> "taskmanager.refused-registration-pause": "5s",
>>>
>>> Shouldn't have any effects anymore (since version 1.5.0).
>>>
>>> [1] https://github.com/apache/flink/pull/12763
>>>
>>> Cheers,
>>> Till
>>>
>>> On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wangzhijiang999@aliyun.com
>> .invalid>
>>> wrote:
>>>
>>>> Hi Thomas,
>>>>
>>>> Thanks for these valuable feedbacks and suggestions, and I think they
>> are
>>>> very helpful for making us better.
>>>>
>>>> I can give an direct answer for this issue:
>>>>> checkpoint alignment buffered metric missing - note that this job isn't
>>>> using the new unaligned checkpointing that should be opt-in.
>>>>
>>>> The metric of checkpoint alignment buffered would be always 0 now, no
>>>> matter with unaligned checkpointing or not, so we removed this metric
>>>> directly.
>>>> The motivation for such change is from reducing in-flight buffers to
>> speed
>>>> up checkpoint somehow. The upstream side would block sending any
>> following
>>>> buffers after sending the barrier until receiving the alignment
>>>> notification from downstream side. Therefore, the downstream side never
>>>> needs to cache
>>>> buffers for blocked channels during alignment. We also illustrated such
>>>> changes in release notes for attention by link [1].
>>>>
>>>> [1]
>>>>
>> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
>>>>
>>>> Best,
>>>> Zhijiang
>>>> ------------------------------------------------------------------
>>>> From:Thomas Weise <th...@apache.org>
>>>> Send Time:2020年6月24日(星期三) 06:51
>>>> To:dev <de...@flink.apache.org>
>>>> Cc:zhijiang <zh...@apache.org>
>>>> Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
>>>>
>>>> Hi,
>>>>
>>>> Thanks for putting together the RC!
>>>>
>>>> I have some preliminary feedback from testing with commit
>>>> 934f91ead00fd658333f65ffa37ab60bd5ffd99b
>>>>
>>>> An internal benchmark application that reads from Kinesis and
>> checkpoints
>>>> ~12GB performs comparably to 1.10.1
>>>>
>>>> There were a few issues hit upgrading our codebase that may be
>> worthwhile
>>>> considering, please see details below.
>>>>
>>>> Given my observations over the past few releases, I would like to
>> suggest
>>>> that the community introduces a log of incompatible changes to be
>> published
>>>> with the release notes. Though it is possible to analyze git history
>> when
>>>> hitting compile errors, there are more subtle changes that can make
>>>> upgrades unnecessarily time-consuming. Contributors introducing such
>>>> changes are probably in the best position to document.
>>>>
>>>> I'm planning to try this or the next RC with a couple more applications.
>>>>
>>>> Cheers,
>>>> Thomas
>>>>
>>>> * notifyCheckpointAborted needed to be implemented
>>>> for org.apache.flink.runtime.state.CheckpointListener - can we have the
>>>> default implementation in the interface so that users aren't forced to
>>>> change their implementations
>>>>
>>>> * following deprecated configuration values had to be modified to get
>>>> the job running:
>>>>
>>>>           "taskmanager.initial-registration-pause": "500ms",
>>>>           "taskmanager.max-registration-pause": "5s",
>>>>           "taskmanager.refused-registration-pause": "5s",
>>>>
>>>> The error message was:
>>>>
>>>> Could not parse value '500ms' for key
>>>> 'cluster.registration.initial-timeout'.\n\tat
>>>>
>>>>
>> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
>>>>
>>>>
>> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
>>>>
>>>>
>> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
>>>>
>>>>
>> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
>>>>
>>>> Though easy to fix, it's unfortunate that values are now treated
>>>> differently.
>>>>
>>>> * checkpoint alignment buffered metric missing - note that this job
>> isn't
>>>> using the new unaligned checkpointing that should be opt-in.
>>>>
>>>> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
>>>>   +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
>>>>
>>>> * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
>>>> config, program.build());
>>>>     +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
>>>> config, program.build(),
>>>>               false, false);
>>>>
>>>> * ProcessingTimeCallback removed from StreamingFileSink
>>>>
>>>>
>>>> On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I would like to give an update about the RC2 status. We are now waiting
>>>> for
>>>>> a green azure build on one final bug fix before creating RC2. This bug
>>>> fix
>>>>> should be merged late afternoon/early evening Berlin time, so RC2 will
>> be
>>>>> hopefully created tomorrow morning. Until then I would ask to not
>>>>> merge/backport commits to release-1.11 branch, including bug fixes. If
>>>> you
>>>>> have something that's truly essential and should be treated as a
>> release
>>>>> blocker, please reach out to me or Zhijiang.
>>>>>
>>>>> Best,
>>>>> Piotr Nowojski
>>>>>
>>>>
>>>>
>>
>>
> 


Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Stephan Ewen <se...@apache.org>.
Hi!

About the "notifyCheckpointAborted()":

When I wrote that comment, I was (apparently wrongly) assuming we were
talking about an internal interface here, because the "abort" signal
was originally only intended to cancel the async part of state backend
checkpoints.

I just realized that this is exposed to users - and I am actually with
Thomas on this one. The "CheckpointListener" is a very public interface
that many users implement. The fact that it is tagged "@PublicEvolving" is
somehow not aligned with reality. So adding the method here will in reality
break lots and lots of user programs.

I think also in practice it is much less relevant for user applications to
react to aborted checkpoints. Since the notifications there can not be
relied upon (if there is a task failure concurrently) users always have to
follow the "newer checkpoint subsumes older checkpoint" contract, so the
abort method is probably rarely relevant.

This is something we should change, in my opinion.

Best,
Stephan



On Wed, Jun 24, 2020 at 4:17 PM Piotr Nowojski <pi...@ververica.com> wrote:

> Hi Thomas,
>
> Just one quick answer from my side about:
>
> > * notifyCheckpointAborted needed to be implemented
> > for org.apache.flink.runtime.state.CheckpointListener - can we have the
> > default implementation in the interface so that users aren't forced to
> > change their implementations
>
> This is intentional design [1]
>
> > Implementers should generally be forced to think about what to do when
> checkpoint is aborted.
>
> Piotrek
>
> [1] https://github.com/apache/flink/pull/8693#issuecomment-542834147
>
> Piotr Nowojski  | Staff Engineer
> +48 503 187 389
>
>
> Follow us @VervericaData
> --
> Join Flink Forward - The Apache Flink Conference
> Stream Processing | Event Driven | Real Time
> --
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Yip Park Tung Jason, Jinwei (Kevin) Zhang, Karl Anton
> Wehner
>
>
>
>
> > On 24 Jun 2020, at 10:35, Till Rohrmann <tr...@apache.org> wrote:
> >
> > Thanks for testing the RC and the feedback Thomas. The problem with the
> > taskmanager options is that the old
> > (taskmanager.initial-registration-pause) and new options
> > (cluster.registration.initial-timeout) don't have the same type. The old
> > options have not been used for a long time (since version 1.5.0) and we
> > wanted to remove them. As part of the removal, we added the old keys as
> > deprecated options for the new ones. I believe this was a mistake. I've
> > opened a PR to remove the deprecated keys from the new ConfigOptions [1].
> >
> > Please be aware that
> >
> > "taskmanager.initial-registration-pause": "500ms",
> > "taskmanager.max-registration-pause": "5s",
> > "taskmanager.refused-registration-pause": "5s",
> >
> > Shouldn't have any effects anymore (since version 1.5.0).
> >
> > [1] https://github.com/apache/flink/pull/12763
> >
> > Cheers,
> > Till
> >
> > On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wangzhijiang999@aliyun.com
> .invalid>
> > wrote:
> >
> >> Hi Thomas,
> >>
> >> Thanks for these valuable feedbacks and suggestions, and I think they
> are
> >> very helpful for making us better.
> >>
> >> I can give an direct answer for this issue:
> >>> checkpoint alignment buffered metric missing - note that this job isn't
> >> using the new unaligned checkpointing that should be opt-in.
> >>
> >> The metric of checkpoint alignment buffered would be always 0 now, no
> >> matter with unaligned checkpointing or not, so we removed this metric
> >> directly.
> >> The motivation for such change is from reducing in-flight buffers to
> speed
> >> up checkpoint somehow. The upstream side would block sending any
> following
> >> buffers after sending the barrier until receiving the alignment
> >> notification from downstream side. Therefore, the downstream side never
> >> needs to cache
> >> buffers for blocked channels during alignment. We also illustrated such
> >> changes in release notes for attention by link [1].
> >>
> >> [1]
> >>
> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
> >>
> >> Best,
> >> Zhijiang
> >> ------------------------------------------------------------------
> >> From:Thomas Weise <th...@apache.org>
> >> Send Time:2020年6月24日(星期三) 06:51
> >> To:dev <de...@flink.apache.org>
> >> Cc:zhijiang <zh...@apache.org>
> >> Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
> >>
> >> Hi,
> >>
> >> Thanks for putting together the RC!
> >>
> >> I have some preliminary feedback from testing with commit
> >> 934f91ead00fd658333f65ffa37ab60bd5ffd99b
> >>
> >> An internal benchmark application that reads from Kinesis and
> checkpoints
> >> ~12GB performs comparably to 1.10.1
> >>
> >> There were a few issues hit upgrading our codebase that may be
> worthwhile
> >> considering, please see details below.
> >>
> >> Given my observations over the past few releases, I would like to
> suggest
> >> that the community introduces a log of incompatible changes to be
> published
> >> with the release notes. Though it is possible to analyze git history
> when
> >> hitting compile errors, there are more subtle changes that can make
> >> upgrades unnecessarily time-consuming. Contributors introducing such
> >> changes are probably in the best position to document.
> >>
> >> I'm planning to try this or the next RC with a couple more applications.
> >>
> >> Cheers,
> >> Thomas
> >>
> >> * notifyCheckpointAborted needed to be implemented
> >> for org.apache.flink.runtime.state.CheckpointListener - can we have the
> >> default implementation in the interface so that users aren't forced to
> >> change their implementations
> >>
> >> * following deprecated configuration values had to be modified to get
> >> the job running:
> >>
> >>          "taskmanager.initial-registration-pause": "500ms",
> >>          "taskmanager.max-registration-pause": "5s",
> >>          "taskmanager.refused-registration-pause": "5s",
> >>
> >> The error message was:
> >>
> >> Could not parse value '500ms' for key
> >> 'cluster.registration.initial-timeout'.\n\tat
> >>
> >>
> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
> >>
> >>
> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
> >>
> >>
> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
> >>
> >>
> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
> >>
> >> Though easy to fix, it's unfortunate that values are now treated
> >> differently.
> >>
> >> * checkpoint alignment buffered metric missing - note that this job
> isn't
> >> using the new unaligned checkpointing that should be opt-in.
> >>
> >> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
> >>  +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
> >>
> >> * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> >> config, program.build());
> >>    +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> >> config, program.build(),
> >>              false, false);
> >>
> >> * ProcessingTimeCallback removed from StreamingFileSink
> >>
> >>
> >> On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
> >> wrote:
> >>
> >>> Hi all,
> >>>
> >>> I would like to give an update about the RC2 status. We are now waiting
> >> for
> >>> a green azure build on one final bug fix before creating RC2. This bug
> >> fix
> >>> should be merged late afternoon/early evening Berlin time, so RC2 will
> be
> >>> hopefully created tomorrow morning. Until then I would ask to not
> >>> merge/backport commits to release-1.11 branch, including bug fixes. If
> >> you
> >>> have something that's truly essential and should be treated as a
> release
> >>> blocker, please reach out to me or Zhijiang.
> >>>
> >>> Best,
> >>> Piotr Nowojski
> >>>
> >>
> >>
>
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Piotr Nowojski <pi...@ververica.com>.
Hi Thomas,

Just one quick answer from my side about:

> * notifyCheckpointAborted needed to be implemented
> for org.apache.flink.runtime.state.CheckpointListener - can we have the
> default implementation in the interface so that users aren't forced to
> change their implementations

This is intentional design [1]

> Implementers should generally be forced to think about what to do when checkpoint is aborted.

Piotrek

[1] https://github.com/apache/flink/pull/8693#issuecomment-542834147

Piotr Nowojski  | Staff Engineer 
+48 503 187 389


Follow us @VervericaData
--
Join Flink Forward - The Apache Flink Conference
Stream Processing | Event Driven | Real Time
--
Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Yip Park Tung Jason, Jinwei (Kevin) Zhang, Karl Anton Wehner




> On 24 Jun 2020, at 10:35, Till Rohrmann <tr...@apache.org> wrote:
> 
> Thanks for testing the RC and the feedback Thomas. The problem with the
> taskmanager options is that the old
> (taskmanager.initial-registration-pause) and new options
> (cluster.registration.initial-timeout) don't have the same type. The old
> options have not been used for a long time (since version 1.5.0) and we
> wanted to remove them. As part of the removal, we added the old keys as
> deprecated options for the new ones. I believe this was a mistake. I've
> opened a PR to remove the deprecated keys from the new ConfigOptions [1].
> 
> Please be aware that
> 
> "taskmanager.initial-registration-pause": "500ms",
> "taskmanager.max-registration-pause": "5s",
> "taskmanager.refused-registration-pause": "5s",
> 
> Shouldn't have any effects anymore (since version 1.5.0).
> 
> [1] https://github.com/apache/flink/pull/12763
> 
> Cheers,
> Till
> 
> On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wa...@aliyun.com.invalid>
> wrote:
> 
>> Hi Thomas,
>> 
>> Thanks for these valuable feedbacks and suggestions, and I think they are
>> very helpful for making us better.
>> 
>> I can give an direct answer for this issue:
>>> checkpoint alignment buffered metric missing - note that this job isn't
>> using the new unaligned checkpointing that should be opt-in.
>> 
>> The metric of checkpoint alignment buffered would be always 0 now, no
>> matter with unaligned checkpointing or not, so we removed this metric
>> directly.
>> The motivation for such change is from reducing in-flight buffers to speed
>> up checkpoint somehow. The upstream side would block sending any following
>> buffers after sending the barrier until receiving the alignment
>> notification from downstream side. Therefore, the downstream side never
>> needs to cache
>> buffers for blocked channels during alignment. We also illustrated such
>> changes in release notes for attention by link [1].
>> 
>> [1]
>> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
>> 
>> Best,
>> Zhijiang
>> ------------------------------------------------------------------
>> From:Thomas Weise <th...@apache.org>
>> Send Time:2020年6月24日(星期三) 06:51
>> To:dev <de...@flink.apache.org>
>> Cc:zhijiang <zh...@apache.org>
>> Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
>> 
>> Hi,
>> 
>> Thanks for putting together the RC!
>> 
>> I have some preliminary feedback from testing with commit
>> 934f91ead00fd658333f65ffa37ab60bd5ffd99b
>> 
>> An internal benchmark application that reads from Kinesis and checkpoints
>> ~12GB performs comparably to 1.10.1
>> 
>> There were a few issues hit upgrading our codebase that may be worthwhile
>> considering, please see details below.
>> 
>> Given my observations over the past few releases, I would like to suggest
>> that the community introduces a log of incompatible changes to be published
>> with the release notes. Though it is possible to analyze git history when
>> hitting compile errors, there are more subtle changes that can make
>> upgrades unnecessarily time-consuming. Contributors introducing such
>> changes are probably in the best position to document.
>> 
>> I'm planning to try this or the next RC with a couple more applications.
>> 
>> Cheers,
>> Thomas
>> 
>> * notifyCheckpointAborted needed to be implemented
>> for org.apache.flink.runtime.state.CheckpointListener - can we have the
>> default implementation in the interface so that users aren't forced to
>> change their implementations
>> 
>> * following deprecated configuration values had to be modified to get
>> the job running:
>> 
>>          "taskmanager.initial-registration-pause": "500ms",
>>          "taskmanager.max-registration-pause": "5s",
>>          "taskmanager.refused-registration-pause": "5s",
>> 
>> The error message was:
>> 
>> Could not parse value '500ms' for key
>> 'cluster.registration.initial-timeout'.\n\tat
>> 
>> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
>> 
>> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
>> 
>> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
>> 
>> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
>> 
>> Though easy to fix, it's unfortunate that values are now treated
>> differently.
>> 
>> * checkpoint alignment buffered metric missing - note that this job isn't
>> using the new unaligned checkpointing that should be opt-in.
>> 
>> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
>>  +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
>> 
>> * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
>> config, program.build());
>>    +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
>> config, program.build(),
>>              false, false);
>> 
>> * ProcessingTimeCallback removed from StreamingFileSink
>> 
>> 
>> On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
>> wrote:
>> 
>>> Hi all,
>>> 
>>> I would like to give an update about the RC2 status. We are now waiting
>> for
>>> a green azure build on one final bug fix before creating RC2. This bug
>> fix
>>> should be merged late afternoon/early evening Berlin time, so RC2 will be
>>> hopefully created tomorrow morning. Until then I would ask to not
>>> merge/backport commits to release-1.11 branch, including bug fixes. If
>> you
>>> have something that's truly essential and should be treated as a release
>>> blocker, please reach out to me or Zhijiang.
>>> 
>>> Best,
>>> Piotr Nowojski
>>> 
>> 
>> 


Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Till Rohrmann <tr...@apache.org>.
Thanks for testing the RC and the feedback Thomas. The problem with the
taskmanager options is that the old
(taskmanager.initial-registration-pause) and new options
(cluster.registration.initial-timeout) don't have the same type. The old
options have not been used for a long time (since version 1.5.0) and we
wanted to remove them. As part of the removal, we added the old keys as
deprecated options for the new ones. I believe this was a mistake. I've
opened a PR to remove the deprecated keys from the new ConfigOptions [1].

Please be aware that

"taskmanager.initial-registration-pause": "500ms",
"taskmanager.max-registration-pause": "5s",
"taskmanager.refused-registration-pause": "5s",

Shouldn't have any effects anymore (since version 1.5.0).

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

Cheers,
Till

On Wed, Jun 24, 2020 at 4:17 AM Zhijiang <wa...@aliyun.com.invalid>
wrote:

> Hi Thomas,
>
> Thanks for these valuable feedbacks and suggestions, and I think they are
> very helpful for making us better.
>
> I can give an direct answer for this issue:
> > checkpoint alignment buffered metric missing - note that this job isn't
> using the new unaligned checkpointing that should be opt-in.
>
> The metric of checkpoint alignment buffered would be always 0 now, no
> matter with unaligned checkpointing or not, so we removed this metric
> directly.
> The motivation for such change is from reducing in-flight buffers to speed
> up checkpoint somehow. The upstream side would block sending any following
> buffers after sending the barrier until receiving the alignment
> notification from downstream side. Therefore, the downstream side never
> needs to cache
> buffers for blocked channels during alignment. We also illustrated such
> changes in release notes for attention by link [1].
>
> [1]
> https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174
>
> Best,
> Zhijiang
> ------------------------------------------------------------------
> From:Thomas Weise <th...@apache.org>
> Send Time:2020年6月24日(星期三) 06:51
> To:dev <de...@flink.apache.org>
> Cc:zhijiang <zh...@apache.org>
> Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2
>
> Hi,
>
> Thanks for putting together the RC!
>
> I have some preliminary feedback from testing with commit
> 934f91ead00fd658333f65ffa37ab60bd5ffd99b
>
> An internal benchmark application that reads from Kinesis and checkpoints
> ~12GB performs comparably to 1.10.1
>
> There were a few issues hit upgrading our codebase that may be worthwhile
> considering, please see details below.
>
> Given my observations over the past few releases, I would like to suggest
> that the community introduces a log of incompatible changes to be published
> with the release notes. Though it is possible to analyze git history when
> hitting compile errors, there are more subtle changes that can make
> upgrades unnecessarily time-consuming. Contributors introducing such
> changes are probably in the best position to document.
>
> I'm planning to try this or the next RC with a couple more applications.
>
> Cheers,
> Thomas
>
> * notifyCheckpointAborted needed to be implemented
> for org.apache.flink.runtime.state.CheckpointListener - can we have the
> default implementation in the interface so that users aren't forced to
> change their implementations
>
> * following deprecated configuration values had to be modified to get
> the job running:
>
>           "taskmanager.initial-registration-pause": "500ms",
>           "taskmanager.max-registration-pause": "5s",
>           "taskmanager.refused-registration-pause": "5s",
>
> The error message was:
>
> Could not parse value '500ms' for key
> 'cluster.registration.initial-timeout'.\n\tat
>
> org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
>
> org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
>
> org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
>
> org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat
>
> Though easy to fix, it's unfortunate that values are now treated
> differently.
>
> * checkpoint alignment buffered metric missing - note that this job isn't
> using the new unaligned checkpointing that should be opt-in.
>
> * -import org.apache.flink.table.api.java.StreamTableEnvironment;
>   +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
>
>  * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> config, program.build());
>     +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
> config, program.build(),
>               false, false);
>
> * ProcessingTimeCallback removed from StreamingFileSink
>
>
> On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org>
> wrote:
>
> > Hi all,
> >
> > I would like to give an update about the RC2 status. We are now waiting
> for
> > a green azure build on one final bug fix before creating RC2. This bug
> fix
> > should be merged late afternoon/early evening Berlin time, so RC2 will be
> > hopefully created tomorrow morning. Until then I would ask to not
> > merge/backport commits to release-1.11 branch, including bug fixes. If
> you
> > have something that's truly essential and should be treated as a release
> > blocker, please reach out to me or Zhijiang.
> >
> > Best,
> > Piotr Nowojski
> >
>
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Zhijiang <wa...@aliyun.com.INVALID>.
Hi Thomas,

Thanks for these valuable feedbacks and suggestions, and I think they are very helpful for making us better.

I can give an direct answer for this issue:
> checkpoint alignment buffered metric missing - note that this job isn't using the new unaligned checkpointing that should be opt-in.

The metric of checkpoint alignment buffered would be always 0 now, no matter with unaligned checkpointing or not, so we removed this metric directly.
The motivation for such change is from reducing in-flight buffers to speed up checkpoint somehow. The upstream side would block sending any following
buffers after sending the barrier until receiving the alignment notification from downstream side. Therefore, the downstream side never needs to cache
buffers for blocked channels during alignment. We also illustrated such changes in release notes for attention by link [1].

[1] https://github.com/apache/flink/pull/12699/files#diff-eaa874e007e88f283e96de2d61cc4140R174

Best,
Zhijiang
------------------------------------------------------------------
From:Thomas Weise <th...@apache.org>
Send Time:2020年6月24日(星期三) 06:51
To:dev <de...@flink.apache.org>
Cc:zhijiang <zh...@apache.org>
Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi,

Thanks for putting together the RC!

I have some preliminary feedback from testing with commit
934f91ead00fd658333f65ffa37ab60bd5ffd99b

An internal benchmark application that reads from Kinesis and checkpoints
~12GB performs comparably to 1.10.1

There were a few issues hit upgrading our codebase that may be worthwhile
considering, please see details below.

Given my observations over the past few releases, I would like to suggest
that the community introduces a log of incompatible changes to be published
with the release notes. Though it is possible to analyze git history when
hitting compile errors, there are more subtle changes that can make
upgrades unnecessarily time-consuming. Contributors introducing such
changes are probably in the best position to document.

I'm planning to try this or the next RC with a couple more applications.

Cheers,
Thomas

* notifyCheckpointAborted needed to be implemented
for org.apache.flink.runtime.state.CheckpointListener - can we have the
default implementation in the interface so that users aren't forced to
change their implementations

* following deprecated configuration values had to be modified to get
the job running:

          "taskmanager.initial-registration-pause": "500ms",
          "taskmanager.max-registration-pause": "5s",
          "taskmanager.refused-registration-pause": "5s",

The error message was:

Could not parse value '500ms' for key
'cluster.registration.initial-timeout'.\n\tat
org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat

Though easy to fix, it's unfortunate that values are now treated
differently.

* checkpoint alignment buffered metric missing - note that this job isn't
using the new unaligned checkpointing that should be opt-in.

* -import org.apache.flink.table.api.java.StreamTableEnvironment;
  +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;

 * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
config, program.build());
    +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
config, program.build(),
              false, false);

* ProcessingTimeCallback removed from StreamingFileSink


On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org> wrote:

> Hi all,
>
> I would like to give an update about the RC2 status. We are now waiting for
> a green azure build on one final bug fix before creating RC2. This bug fix
> should be merged late afternoon/early evening Berlin time, so RC2 will be
> hopefully created tomorrow morning. Until then I would ask to not
> merge/backport commits to release-1.11 branch, including bug fixes. If you
> have something that's truly essential and should be treated as a release
> blocker, please reach out to me or Zhijiang.
>
> Best,
> Piotr Nowojski
>


Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Thomas Weise <th...@apache.org>.
Hi,

Thanks for putting together the RC!

I have some preliminary feedback from testing with commit
934f91ead00fd658333f65ffa37ab60bd5ffd99b

An internal benchmark application that reads from Kinesis and checkpoints
~12GB performs comparably to 1.10.1

There were a few issues hit upgrading our codebase that may be worthwhile
considering, please see details below.

Given my observations over the past few releases, I would like to suggest
that the community introduces a log of incompatible changes to be published
with the release notes. Though it is possible to analyze git history when
hitting compile errors, there are more subtle changes that can make
upgrades unnecessarily time-consuming. Contributors introducing such
changes are probably in the best position to document.

I'm planning to try this or the next RC with a couple more applications.

Cheers,
Thomas

* notifyCheckpointAborted needed to be implemented
for org.apache.flink.runtime.state.CheckpointListener - can we have the
default implementation in the interface so that users aren't forced to
change their implementations

* following deprecated configuration values had to be modified to get
the job running:

          "taskmanager.initial-registration-pause": "500ms",
          "taskmanager.max-registration-pause": "5s",
          "taskmanager.refused-registration-pause": "5s",

The error message was:

Could not parse value '500ms' for key
'cluster.registration.initial-timeout'.\n\tat
org.apache.flink.configuration.Configuration.getOptional(Configuration.java:753)\n\tat
org.apache.flink.configuration.Configuration.getLong(Configuration.java:298)\n\tat
org.apache.flink.runtime.registration.RetryingRegistrationConfiguration.fromConfiguration(RetryingRegistrationConfiguration.java:72)\n\tat
org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration.fromConfiguration(TaskManagerServicesConfiguration.java:262)\n\tat

Though easy to fix, it's unfortunate that values are now treated
differently.

* checkpoint alignment buffered metric missing - note that this job isn't
using the new unaligned checkpointing that should be opt-in.

* -import org.apache.flink.table.api.java.StreamTableEnvironment;
  +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;

 * -ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
config, program.build());
    +ClientUtils.executeProgram(DefaultExecutorServiceLoader.INSTANCE,
config, program.build(),
              false, false);

* ProcessingTimeCallback removed from StreamingFileSink


On Wed, Jun 17, 2020 at 6:29 AM Piotr Nowojski <pn...@apache.org> wrote:

> Hi all,
>
> I would like to give an update about the RC2 status. We are now waiting for
> a green azure build on one final bug fix before creating RC2. This bug fix
> should be merged late afternoon/early evening Berlin time, so RC2 will be
> hopefully created tomorrow morning. Until then I would ask to not
> merge/backport commits to release-1.11 branch, including bug fixes. If you
> have something that's truly essential and should be treated as a release
> blocker, please reach out to me or Zhijiang.
>
> Best,
> Piotr Nowojski
>

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Jingsong Li <ji...@gmail.com>.
Hi Piotr and Zhijiang,
I merged three commits in 1 hour ago:
one blocker: 0dafcf8792220dbe5b77544261f726a566b054f5
two critical issues:
1830c1c47b8a985ec328a7332e92d21433c0a4df
80fa0f5c5b8600f4b386487f267bde80b882bd07

I have synced with Zhijiang, they were merged after RC2 branch cutting. So
RC2 does not include them.

Hope not bother your RC testing. If there are some problems, I will revert
them at any time.

Best,
Jingsong

On Wed, Jun 17, 2020 at 9:29 PM Piotr Nowojski <pn...@apache.org> wrote:

> Hi all,
>
> I would like to give an update about the RC2 status. We are now waiting for
> a green azure build on one final bug fix before creating RC2. This bug fix
> should be merged late afternoon/early evening Berlin time, so RC2 will be
> hopefully created tomorrow morning. Until then I would ask to not
> merge/backport commits to release-1.11 branch, including bug fixes. If you
> have something that's truly essential and should be treated as a release
> blocker, please reach out to me or Zhijiang.
>
> Best,
> Piotr Nowojski
>


-- 
Best, Jingsong Lee

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Zhijiang <wa...@aliyun.com.INVALID>.
Hi Febian,

I do not think that issue would block the current testing purpose, since the codes of RC2 will not cover that compile issue. 
You can checkout the RC2 tag [1] for compiling if needed. And we might prepare for the next formal votable RC3 soon.


[1] https://dist.apache.org/repos/dist/dev/flink/flink-1.11.0-rc2/

Best,
Zhijiang


------------------------------------------------------------------
From:Fabian Paul <fa...@data-artisans.com>
Send Time:2020年6月23日(星期二) 15:41
To:Zhijiang <wa...@aliyun.com>; dev <de...@flink.apache.org>
Cc:zhijiang <zh...@apache.org>; pnowojski <pn...@apache.org>
Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi,

Thanks again for uploading the missing artifacts. Unfortunately this rc does not fully compile due to [1].

Would it be possible for testing purposed to quickly include this fix into the rc or do you think it is necessary to open a complete new one?


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

Best,
Fabian


Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Arvid Heise <ar...@ververica.com>.
I tested RC2 successfully on EMR with 160 cores across 5 nodes using the
performance benchmark [1] with s3 file backend.

I used different backpressure settings to compare aligned and unaligned
checkpoints on 1.11-rc2 and aligned checkpoints of 1.10.1. I saw no errors
and no regressions (rather we improved the general performance a bit but
it's within the margin of error) [2].

I also manually tested failure recovery of unaligned checkpoints by killing
taskmanagers under heavy backpressure.

[1] https://github.com/dataArtisans/performance
[2]
https://docs.google.com/spreadsheets/d/18GO15zO-WI2EzK0fTkWucMmAGicmi0mGjxHlHgzitHQ/edit?usp=sharing

On Tue, Jun 23, 2020 at 9:41 AM Fabian Paul <fa...@data-artisans.com>
wrote:

> Hi,
>
> Thanks again for uploading the missing artifacts. Unfortunately this rc
> does not fully compile due to [1].
>
> Would it be possible for testing purposed to quickly include this fix into
> the rc or do you think it is necessary to open a complete new one?
>
>
> [1] https://issues.apache.org/jira/browse/FLINK-18411 <
> https://issues.apache.org/jira/browse/FLINK-18411>
>
> Best,
> Fabian



-- 

Arvid Heise | Senior Java Developer

<https://www.ververica.com/>

Follow us @VervericaData

--

Join Flink Forward <https://flink-forward.org/> - The Apache Flink
Conference

Stream Processing | Event Driven | Real Time

--

Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany

--
Ververica GmbH
Registered at Amtsgericht Charlottenburg: HRB 158244 B
Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
(Toni) Cheng

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Fabian Paul <fa...@data-artisans.com>.
Hi,

Thanks again for uploading the missing artifacts. Unfortunately this rc does not fully compile due to [1].

Would it be possible for testing purposed to quickly include this fix into the rc or do you think it is necessary to open a complete new one?


[1] https://issues.apache.org/jira/browse/FLINK-18411 <https://issues.apache.org/jira/browse/FLINK-18411>

Best,
Fabian

Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Zhijiang <wa...@aliyun.com.INVALID>.
Hi all, 

The previous link [1] for all artifacts in Maven Central Repository missed many artifacts, so we deploy them again by the new link [2].
Sorry for the inconvenience and happy testing again!

[1] https://repository.apache.org/content/repositories/orgapacheflink-1374
[2] https://repository.apache.org/content/repositories/orgapacheflink-1375/

Best,
Zhijiang


------------------------------------------------------------------
From:Zhijiang <wa...@aliyun.com.INVALID>
Send Time:2020年6月22日(星期一) 18:34
To:Fabian Paul <fa...@data-artisans.com>; dev <de...@flink.apache.org>
Cc:zhijiang <zh...@apache.org>; pnowojski <pn...@apache.org>
Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi Fabian,

Thanks for this finding and let us know. I will double check it and update the missing jars afterwards if needed.

Best,
Zhijiang


------------------------------------------------------------------
From:Fabian Paul <fa...@data-artisans.com>
Send Time:2020年6月22日(星期一) 16:55
To:dev <de...@flink.apache.org>
Cc:zhijiang <zh...@apache.org>; pnowojski <pn...@apache.org>
Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi,

Thanks for the great efforts in preparing the second rc. I was just going through the published artifacts and it seems that some are missing in the latest release.

In comparison you can look at 

https://repository.apache.org/content/repositories/orgapacheflink-1370/org/apache/flink/ with the full list of artifacts for the first rc and 
https://repository.apache.org/content/repositories/orgapacheflink-1374/org/apache/flink/ with only a subset for the second one.

Did you only upload the artifacts which have not been changed?

Best,
Fabian



Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Zhijiang <wa...@aliyun.com.INVALID>.
Hi Fabian,

Thanks for this finding and let us know. I will double check it and update the missing jars afterwards if needed.

Best,
Zhijiang


------------------------------------------------------------------
From:Fabian Paul <fa...@data-artisans.com>
Send Time:2020年6月22日(星期一) 16:55
To:dev <de...@flink.apache.org>
Cc:zhijiang <zh...@apache.org>; pnowojski <pn...@apache.org>
Subject:Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Hi,

Thanks for the great efforts in preparing the second rc. I was just going through the published artifacts and it seems that some are missing in the latest release.

In comparison you can look at 

https://repository.apache.org/content/repositories/orgapacheflink-1370/org/apache/flink/ with the full list of artifacts for the first rc and 
https://repository.apache.org/content/repositories/orgapacheflink-1374/org/apache/flink/ with only a subset for the second one.

Did you only upload the artifacts which have not been changed?

Best,
Fabian


Re: [ANNOUNCE] Apache Flink 1.11.0, release candidate #2

Posted by Fabian Paul <fa...@data-artisans.com>.
Hi,

Thanks for the great efforts in preparing the second rc. I was just going through the published artifacts and it seems that some are missing in the latest release.

In comparison you can look at 

https://repository.apache.org/content/repositories/orgapacheflink-1370/org/apache/flink/ <https://repository.apache.org/content/repositories/orgapacheflink-1370/org/apache/flink/> with the full list of artifacts for the first rc and 
https://repository.apache.org/content/repositories/orgapacheflink-1374/org/apache/flink/ <https://repository.apache.org/content/repositories/orgapacheflink-1374/org/apache/flink/> with only a subset for the second one.

Did you only upload the artifacts which have not been changed?

Best,
Fabian