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 <trohrm...@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 <wangzhijiang...@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 <t...@apache.org>
Send Time:2020年6月24日(星期三) 06:51
To:dev <dev@flink.apache.org>
Cc:zhijiang <zhiji...@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 <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