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 <pnowoj...@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
>

Reply via email to