Thanks Till for the clarification. I opened https://github.com/apache/flink/pull/12816
On 03/07/2020 10:15, Till Rohrmann wrote: > @Dawid I think it would be correct to also include the classifier for the > org.apache.orc:orc-core:jar:nohive:1.4.3 dependency because it is different > from the non-classified artifact. I would not block the release on it, > though, because it is a ASL 2.0 dependency which we are not required to > list. Can you open a PR for fixing this problem? > > Concerning the Python module I believe that Jincheng could help us with the > verification process. > > Cheers, > Till > > On Fri, Jul 3, 2020 at 8:46 AM 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 >>>>>> >>>>>> >>> >>
signature.asc
Description: OpenPGP digital signature