Hi all,

As far as the issue that Chesnay mentioned that leads to a "Caused by:
org.apache.flink.api.common.InvalidProgramException:"  for DataSet
examples with print() collect() or count() as sink, this was a
semi-intensional side-effect of the application mode. Before, in these
cases, the output was simply ignored. Now we have the same behavior as
in the "detached" mode. I already opened a PR for the release notes
(sorry for not doing it earlier although this was a known change in
behavior, as mentioned it in the PR here
https://github.com/apache/flink/pull/11460 ) and I will merge it
today.

Cheers,
Kostas

On Thu, Jul 2, 2020 at 8:07 PM Robert Metzger <rmetz...@apache.org> wrote:
>
> +1 (binding)
>
> Checks:
> - source archive compiles
> - checked artifacts in staging repo
>   - flink-azure-fs-hadoop-1.11.0.jar seems to have a correct NOTICE file
>   - versions in pom seem correct
>   - checked some other jars
> - deployed Flink on YARN on Azure HDInsight (which uses Hadoop 3.1.1)
>   - Reported some tiny log sanity issue:
> https://issues.apache.org/jira/browse/FLINK-18474
>   - Wordcount against HDFS works
>
>
> On Thu, Jul 2, 2020 at 7:07 PM Thomas Weise <t...@apache.org> wrote:
>
> > Hi Zhijiang,
> >
> > The performance degradation manifests in backpressure which leads to
> > growing backlog in the source. I switched a few times between 1.10 and 1.11
> > and the behavior is consistent.
> >
> > The DAG is:
> >
> > KinesisConsumer -> (Flat Map, Flat Map, Flat Map)   -------- forward
> > ---------> KinesisProducer
> >
> > Parallelism: 160
> > No shuffle/rebalance.
> >
> > Checkpointing config:
> >
> > Checkpointing Mode Exactly Once
> > Interval 10s
> > Timeout 10m 0s
> > Minimum Pause Between Checkpoints 10s
> > Maximum Concurrent Checkpoints 1
> > Persist Checkpoints Externally Enabled (delete on cancellation)
> >
> > State backend: rocksdb  (filesystem leads to same symptoms)
> > Checkpoint size is tiny (500KB)
> >
> > An interesting difference to another job that I had upgraded successfully
> > is the low checkpointing interval.
> >
> > Thanks,
> > Thomas
> >
> >
> > On Wed, Jul 1, 2020 at 9:02 PM Zhijiang <wangzhijiang...@aliyun.com
> > .invalid>
> > wrote:
> >
> > > Hi Thomas,
> > >
> > > Thanks for the efficient feedback.
> > >
> > > Regarding the suggestion of adding the release notes document, I agree
> > > with your point. Maybe we should adjust the vote template accordingly in
> > > the respective wiki to guide the following release processes.
> > >
> > > Regarding the performance regression, could you provide some more details
> > > for our better measurement or reproducing on our sides?
> > > E.g. I guess the topology only includes two vertexes source and sink?
> > > What is the parallelism for every vertex?
> > > The upstream shuffles data to the downstream via rebalance partitioner or
> > > other?
> > > The checkpoint mode is exactly-once with rocksDB state backend?
> > > The backpressure happened in this case?
> > > How much percentage regression in this case?
> > >
> > > Best,
> > > Zhijiang
> > >
> > >
> > >
> > > ------------------------------------------------------------------
> > > From:Thomas Weise <t...@apache.org>
> > > Send Time:2020年7月2日(星期四) 09:54
> > > To:dev <dev@flink.apache.org>
> > > Subject:Re: [VOTE] Release 1.11.0, release candidate #4
> > >
> > > Hi Till,
> > >
> > > Yes, we don't have the setting in flink-conf.yaml.
> > >
> > > Generally, we carry forward the existing configuration and any change to
> > > default configuration values would impact the upgrade.
> > >
> > > Yes, since it is an incompatible change I would state it in the release
> > > notes.
> > >
> > > Thanks,
> > > Thomas
> > >
> > > BTW I found a performance regression while trying to upgrade another
> > > pipeline with this RC. It is a simple Kinesis to Kinesis job. Wasn't able
> > > to pin it down yet, symptoms include increased checkpoint alignment time.
> > >
> > > On Wed, Jul 1, 2020 at 12:04 AM Till Rohrmann <trohrm...@apache.org>
> > > wrote:
> > >
> > > > Hi Thomas,
> > > >
> > > > just to confirm: When starting the image in local mode, then you don't
> > > have
> > > > any of the JobManager memory configuration settings configured in the
> > > > effective flink-conf.yaml, right? Does this mean that you have
> > explicitly
> > > > removed `jobmanager.heap.size: 1024m` from the default configuration?
> > If
> > > > this is the case, then I believe it was more of an unintentional
> > artifact
> > > > that it worked before and it has been corrected now so that one needs
> > to
> > > > specify the memory of the JM process explicitly. Do you think it would
> > > help
> > > > to explicitly state this in the release notes?
> > > >
> > > > Cheers,
> > > > Till
> > > >
> > > > On Wed, Jul 1, 2020 at 7:01 AM Thomas Weise <t...@apache.org> wrote:
> > > >
> > > > > Thanks for preparing another RC!
> > > > >
> > > > > As mentioned in the previous RC thread, it would be super helpful if
> > > the
> > > > > release notes that are part of the documentation can be included [1].
> > > > It's
> > > > > a significant time-saver to have read those first.
> > > > >
> > > > > I found one more non-backward compatible change that would be worth
> > > > > addressing/mentioning:
> > > > >
> > > > > It is now necessary to configure the jobmanager heap size in
> > > > > flink-conf.yaml (with either jobmanager.heap.size
> > > > > or jobmanager.memory.heap.size). Why would I not want to do that
> > > anyways?
> > > > > Well, we set it dynamically for a cluster deployment via the
> > > > > flinkk8soperator, but the container image can also be used for
> > testing
> > > > with
> > > > > local mode (./bin/jobmanager.sh start-foreground local). That will
> > fail
> > > > if
> > > > > the heap wasn't configured and that's how I noticed it.
> > > > >
> > > > > Thanks,
> > > > > Thomas
> > > > >
> > > > > [1]
> > > > >
> > > > >
> > > >
> > >
> > https://ci.apache.org/projects/flink/flink-docs-release-1.11/release-notes/flink-1.11.html
> > > > >
> > > > > On Tue, Jun 30, 2020 at 3:18 AM Zhijiang <wangzhijiang...@aliyun.com
> > > > > .invalid>
> > > > > wrote:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > Please review and vote on the release candidate #4 for the version
> > > > > 1.11.0,
> > > > > > as follows:
> > > > > > [ ] +1, Approve the release
> > > > > > [ ] -1, Do not approve the release (please provide specific
> > comments)
> > > > > >
> > > > > > The complete staging area is available for your review, which
> > > includes:
> > > > > > * JIRA release notes [1],
> > > > > > * the official Apache source release and binary convenience
> > releases
> > > to
> > > > > be
> > > > > > deployed to dist.apache.org [2], which are signed with the key
> > with
> > > > > > fingerprint 2DA85B93244FDFA19A6244500653C0A2CEA00D0E [3],
> > > > > > * all artifacts to be deployed to the Maven Central Repository [4],
> > > > > > * source code tag "release-1.11.0-rc4" [5],
> > > > > > * website pull request listing the new release and adding
> > > announcement
> > > > > > blog post [6].
> > > > > >
> > > > > > The vote will be open for at least 72 hours. It is adopted by
> > > majority
> > > > > > approval, with at least 3 PMC affirmative votes.
> > > > > >
> > > > > > Thanks,
> > > > > > Release Manager
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > > >
> > >
> > https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&version=12346364
> > > > > > [2] https://dist.apache.org/repos/dist/dev/flink/flink-1.11.0-rc4/
> > > > > > [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> > > > > > [4]
> > > > > >
> > > >
> > https://repository.apache.org/content/repositories/orgapacheflink-1377/
> > > > > > [5]
> > https://github.com/apache/flink/releases/tag/release-1.11.0-rc4
> > > > > > [6] https://github.com/apache/flink-web/pull/352
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> >

Reply via email to