Hi all,

The vote already lasted for more than 72 hours. Thanks everyone for helping 
test and verify the release. 
I will finalize the vote result soon in a separate email.

Best,
Zhijiang


------------------------------------------------------------------
From:Jingsong Li <jingsongl...@gmail.com>
Send Time:2020年7月6日(星期一) 12:11
To:dev <dev@flink.apache.org>
Subject:Re: [VOTE] Release 1.11.0, release candidate #4

+1 (non-binding)

- verified signature and checksum
- build from source
- checked webui and log sanity
- played with filesystem and new connectors
- played with Hive connector

Best,
Jingsonga

On Mon, Jul 6, 2020 at 9:50 AM Xintong Song <tonysong...@gmail.com> wrote:

> +1 (non-binding)
>
> - verified signature and checksum
> - build from source
> - checked log sanity
> - checked webui
> - played with memory configurations
> - played with binding addresses/ports
>
> Thank you~
>
> Xintong Song
>
>
>
> On Sun, Jul 5, 2020 at 9:41 PM Benchao Li <libenc...@apache.org> wrote:
>
> > +1 (non-binding)
> >
> > Checks:
> > - verified signature and shasum of release files [OK]
> > - build from source [OK]
> > - started standalone cluster, sql-client [mostly OK except one issue]
> >   - played with sql-client
> >   - played with new features: LIKE / Table Options
> >   - checked Web UI functionality
> >   - canceled job from UI
> >
> > While I'm playing with the new table factories, I found one issue[1]
> which
> > surprises me.
> > I don't think this should be a blocker, hence I'll still vote my +1.
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-18487
> >
> > Zhijiang <wangzhijiang...@aliyun.com.invalid> 于2020年7月5日周日 下午1:10写道:
> >
> > > Hi Thomas,
> > >
> > > Regarding [2], it has more detail infos in the Jira description (
> > > https://issues.apache.org/jira/browse/FLINK-16404).
> > >
> > > I can also give some basic explanations here to dismiss the concern.
> > > 1. In the past, the following buffers after the barrier will be cached
> on
> > > downstream side before alignment.
> > > 2. In 1.11, the upstream would not send the buffers after the barrier.
> > > When the downstream finishes the alignment, it will notify the
> downstream
> > > of continuing sending following buffers, since it can process them
> after
> > > alignment.
> > > 3. The only difference is that the temporary blocked buffers are cached
> > > either on downstream side or on upstream side before alignment.
> > > 4. The side effect would be the additional notification cost for every
> > > barrier alignment. If the downstream and upstream are deployed in
> > separate
> > > TaskManager, the cost is network transport delay (the effect can be
> > ignored
> > > based on our testing with 1s checkpoint interval). For sharing slot in
> > your
> > > case, the cost is only one method call in processor, can be ignored
> also.
> > >
> > > You mentioned "In this case, the downstream task has a high average
> > > checkpoint duration(~30s, sync part)." This duration is not reflecting
> > the
> > > changes above, and it is only indicating the duration for calling
> > > `Operation.snapshotState`.
> > > If this duration is beyond your expectation, you can check or debug
> > > whether the source/sink operations might take more time to finish
> > > `snapshotState` in practice. E.g. you can
> > > make the implementation of this method as empty to further verify the
> > > effect.
> > >
> > > Best,
> > > Zhijiang
> > >
> > >
> > > ------------------------------------------------------------------
> > > From:Thomas Weise <t...@apache.org>
> > > Send Time:2020年7月5日(星期日) 12:22
> > > To:dev <dev@flink.apache.org>; Zhijiang <wangzhijiang...@aliyun.com>
> > > Cc:Yingjie Cao <kevin.ying...@gmail.com>
> > > Subject:Re: [VOTE] Release 1.11.0, release candidate #4
> > >
> > > Hi Zhijiang,
> > >
> > > Could you please point me to more details regarding: "[2]: Delay send
> the
> > > following buffers after checkpoint barrier on upstream side until
> barrier
> > > alignment on downstream side."
> > >
> > > In this case, the downstream task has a high average checkpoint
> duration
> > > (~30s, sync part). If there was a change to hold buffers depending on
> > > downstream performance, could this possibly apply to this case (even
> when
> > > there is no shuffle that would require alignment)?
> > >
> > > Thanks,
> > > Thomas
> > >
> > >
> > > On Sat, Jul 4, 2020 at 7:39 AM Zhijiang <wangzhijiang...@aliyun.com
> > > .invalid>
> > > wrote:
> > >
> > > > Hi Thomas,
> > > >
> > > > Thanks for the further update information.
> > > >
> > > > I guess we can dismiss the network stack changes, since in your case
> > the
> > > > downstream and upstream would probably be deployed in the same slot
> > > > bypassing the network data shuffle.
> > > > Also I guess release-1.11 will not bring general performance
> regression
> > > in
> > > > runtime engine, as we also did the performance testing for all
> general
> > > > cases by [1] in real cluster before and the testing results should
> fit
> > > the
> > > > expectation. But we indeed did not test the specific source and sink
> > > > connectors yet as I known.
> > > >
> > > > Regarding your performance regression with 40%, I wonder it is
> probably
> > > > related to specific source/sink changes (e.g. kinesis) or environment
> > > > issues with corner case.
> > > > If possible, it would be helpful to further locate whether the
> > regression
> > > > is caused by kinesis, by replacing the kinesis source & sink and
> > keeping
> > > > the others same.
> > > >
> > > > As you said, it would be efficient to contact with you directly next
> > week
> > > > to further discuss this issue. And we are willing/eager to provide
> any
> > > help
> > > > to resolve this issue soon.
> > > >
> > > > Besides that, I guess this issue should not be the blocker for the
> > > > release, since it is probably a corner case based on the current
> > > analysis.
> > > > If we really conclude anything need to be resolved after the final
> > > > release, then we can also make the next minor release-1.11.1 come
> soon.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/FLINK-18433
> > > >
> > > > Best,
> > > > Zhijiang
> > > >
> > > >
> > > > ------------------------------------------------------------------
> > > > From:Thomas Weise <t...@apache.org>
> > > > Send Time:2020年7月4日(星期六) 12:26
> > > > To:dev <dev@flink.apache.org>; Zhijiang <wangzhijiang...@aliyun.com>
> > > > Cc:Yingjie Cao <kevin.ying...@gmail.com>
> > > > Subject:Re: [VOTE] Release 1.11.0, release candidate #4
> > > >
> > > > Hi Zhijiang,
> > > >
> > > > It will probably be best if we connect next week and discuss the
> issue
> > > > directly since this could be quite difficult to reproduce.
> > > >
> > > > Before the testing result on our side comes out for your respective
> job
> > > > case, I have some other questions to confirm for further analysis:
> > > >     -  How much percentage regression you found after switching to
> > 1.11?
> > > >
> > > > ~40% throughput decline
> > > >
> > > >     -  Are there any network bottleneck in your cluster? E.g. the
> > network
> > > > bandwidth is full caused by other jobs? If so, it might have more
> > effects
> > > > by above [2]
> > > >
> > > > The test runs on a k8s cluster that is also used for other production
> > > jobs.
> > > > There is no reason be believe network is the bottleneck.
> > > >
> > > >     -  Did you adjust the default network buffer setting? E.g.
> > > > "taskmanager.network.memory.floating-buffers-per-gate" or
> > > > "taskmanager.network.memory.buffers-per-channel"
> > > >
> > > > The job is using the defaults, i.e we don't configure the settings.
> If
> > > you
> > > > want me to try specific settings in the hope that it will help to
> > isolate
> > > > the issue please let me know.
> > > >
> > > >     -  I guess the topology has three vertexes "KinesisConsumer ->
> > > Chained
> > > > FlatMap -> KinesisProducer", and the partition mode for
> > "KinesisConsumer
> > > ->
> > > > FlatMap" and "FlatMap->KinesisProducer" are both "forward"? If so,
> the
> > > edge
> > > > connection is one-to-one, not all-to-all, then the above [1][2]
> should
> > no
> > > > effects in theory with default network buffer setting.
> > > >
> > > > There are only 2 vertices and the edge is "forward".
> > > >
> > > >     - By slot sharing, I guess these three vertex parallelism task
> > would
> > > > probably be deployed into the same slot, then the data shuffle is by
> > > memory
> > > > queue, not network stack. If so, the above [2] should no effect.
> > > >
> > > > Yes, vertices share slots.
> > > >
> > > >     - I also saw some Jira changes for kinesis in this release, could
> > you
> > > > confirm that these changes would not effect the performance?
> > > >
> > > > I will need to take a look. 1.10 already had a regression introduced
> by
> > > the
> > > > Kinesis producer update.
> > > >
> > > >
> > > > Thanks,
> > > > Thomas
> > > >
> > > >
> > > > On Thu, Jul 2, 2020 at 11:46 PM Zhijiang <wangzhijiang...@aliyun.com
> > > > .invalid>
> > > > wrote:
> > > >
> > > > > Hi Thomas,
> > > > >
> > > > > Thanks for your reply with rich information!
> > > > >
> > > > > We are trying to reproduce your case in our cluster to further
> verify
> > > it,
> > > > > and  @Yingjie Cao is working on it now.
> > > > >  As we have not kinesis consumer and producer internally, so we
> will
> > > > > construct the common source and sink instead in the case of
> > > backpressure.
> > > > >
> > > > > Firstly, we can dismiss the rockdb factor in this release, since
> you
> > > also
> > > > > mentioned that "filesystem leads to same symptoms".
> > > > >
> > > > > Secondly, if my understanding is right, you emphasis that the
> > > regression
> > > > > only exists for the jobs with low checkpoint interval (10s).
> > > > > Based on that, I have two suspicions with the network related
> changes
> > > in
> > > > > this release:
> > > > >     - [1]: Limited the maximum backlog value (default 10) in
> > > subpartition
> > > > > queue.
> > > > >     - [2]: Delay send the following buffers after checkpoint
> barrier
> > on
> > > > > upstream side until barrier alignment on downstream side.
> > > > >
> > > > > These changes are motivated for reducing the in-flight buffers to
> > > speedup
> > > > > checkpoint especially in the case of backpressure.
> > > > > In theory they should have very minor performance effect and
> actually
> > > we
> > > > > also tested in cluster to verify within expectation before merging
> > > them,
> > > > >  but maybe there are other corner cases we have not thought of
> > before.
> > > > >
> > > > > Before the testing result on our side comes out for your respective
> > job
> > > > > case, I have some other questions to confirm for further analysis:
> > > > >     -  How much percentage regression you found after switching to
> > > 1.11?
> > > > >     -  Are there any network bottleneck in your cluster? E.g. the
> > > network
> > > > > bandwidth is full caused by other jobs? If so, it might have more
> > > effects
> > > > > by above [2]
> > > > >     -  Did you adjust the default network buffer setting? E.g.
> > > > > "taskmanager.network.memory.floating-buffers-per-gate" or
> > > > > "taskmanager.network.memory.buffers-per-channel"
> > > > >     -  I guess the topology has three vertexes "KinesisConsumer ->
> > > > Chained
> > > > > FlatMap -> KinesisProducer", and the partition mode for
> > > "KinesisConsumer
> > > > ->
> > > > > FlatMap" and "FlatMap->KinesisProducer" are both "forward"? If so,
> > the
> > > > edge
> > > > > connection is one-to-one, not all-to-all, then the above [1][2]
> > should
> > > no
> > > > > effects in theory with default network buffer setting.
> > > > >     - By slot sharing, I guess these three vertex parallelism task
> > > would
> > > > > probably be deployed into the same slot, then the data shuffle is
> by
> > > > memory
> > > > > queue, not network stack. If so, the above [2] should no effect.
> > > > >     - I also saw some Jira changes for kinesis in this release,
> could
> > > you
> > > > > confirm that these changes would not effect the performance?
> > > > >
> > > > > Best,
> > > > > Zhijiang
> > > > >
> > > > >
> > > > > ------------------------------------------------------------------
> > > > > From:Thomas Weise <t...@apache.org>
> > > > > Send Time:2020年7月3日(星期五) 01:07
> > > > > To:dev <dev@flink.apache.org>; Zhijiang <
> wangzhijiang...@aliyun.com>
> > > > > Subject:Re: [VOTE] Release 1.11.0, release candidate #4
> > > > >
> > > > > 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
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > > >
> > >
> > >
> >
> > --
> >
> > Best,
> > Benchao Li
> >
>


-- 
Best, Jingsong Lee

Reply via email to