[
https://issues.apache.org/jira/browse/FLINK-30624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17681756#comment-17681756
]
Lijie Wang edited comment on FLINK-30624 at 1/29/23 3:51 PM:
-------------------------------------------------------------
After an offline discussion with [~zhuzh], we believe that the regressions are
caused by some statebackend-related code
blocks([HeapPriorityQueue|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapPriorityQueue.java],
[HeapPriorityQueueElement|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapPriorityQueueElement.java])
that can no longer be inlined by jvm after FLINK-30544. Before FLINK-30544,
there was only one implementation of {{HeapPriorityQueueElement}} (the
[TimerHeapInternalTimer|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java])
while running the benchmark, which allowed the jvm to inline its methods
(getInternalIndex,setInternalIndex). However, we introduced
[InputChannelStatus|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValve.java#:~:text=protected%20static%20class%20InputChannelStatus%20implements%20HeapPriorityQueueElement%20%7B]
(implements {{{}HeapPriorityQueueElement{}}}) in FLINK-30544, resulting in two
implementations of {{{}HeapPriorityQueueElement{}}}, the related methods can no
longer be inlined, introduced the extra virtual method call overhead, resulting
in the performance regression.
I have ran a [test
branch|https://github.com/wanglijie95/flink/tree/after-30544-fix] to verify our
thought. I created new classes
{{{}HeapPriorityQueue2/HeapPriorityQueueElement2{}}}, which have exactly the
same logic as {{{}HeapPriorityQueue/HeapPriorityQueueElement{}}}, and used them
in {{StatusWatermarkValve}} as replacements. Then I ran the benchmarks and
found that the regressions disappeared:
* [209 - test
branch|http://codespeed.dak8s.net:8080/job/flink-benchmark-request/209/]
** stateBackends.MEMORY: 4138.693 (+-66.226)
** stateBackends.FS: 4172.001 (+-50.766)
** stateBackends.FS_ASYNC: 4200.842 (+-51.285)
I think we can solve this ticket by the above approach (introduce new classes
with the same logic), I will open a PR soon.
was (Author: wanglijie95):
After an offline discussion with [~zhuzh], we believe that the regressions are
caused by some statebackend-related code
blocks([HeapPriorityQueue|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapPriorityQueue.java],
[HeapPriorityQueueElement|https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapPriorityQueueElement.java])
that can no longer be inlined by jvm after FLINK-30544. Before FLINK-30544,
there was only one implementation of {{HeapPriorityQueueElement}} (the
[TimerHeapInternalTimer|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java])
while running the benchmark, which allowed the jvm to inline its methods
(getInternalIndex,setInternalIndex). However, we introduced
[InputChannelStatus|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/watermarkstatus/StatusWatermarkValve.java#:~:text=protected%20static%20class%20InputChannelStatus%20implements%20HeapPriorityQueueElement%20%7B]
(implements {{{}HeapPriorityQueueElement{}}}) in FLINK-30544, resulting in two
implementations of {{{}HeapPriorityQueueElement{}}}, the related methods can no
longer be inlined, introduced the extra virtual method call overhead, resulting
in the performance regression.
I have ran the code
[here|https://github.com/wanglijie95/flink/tree/after-30544-fix] to verify our
thought. I created new classes
{{{}HeapPriorityQueue2/HeapPriorityQueueElement2{}}}, which have exactly the
same logic as {{{}HeapPriorityQueue/HeapPriorityQueueElement{}}}, and used them
in {{StatusWatermarkValve}} as replacements. Then I ran the benchmarks and
found that the regressions disappeared:
* [209|http://codespeed.dak8s.net:8080/job/flink-benchmark-request/209/]
** stateBackends.MEMORY: 4138.693 (+-66.226)
** stateBackends.FS: 4172.001 (+-50.766)
** stateBackends.FS_ASYNC: 4200.842 (+-51.285)
I think we can solve this ticket by the above approach (introduce new classes
with the same logic), I will open a PR soon.
> Performance regression in stateBackends.FS / FS_ASYNC / MEMORY on 05.01.2023
> ----------------------------------------------------------------------------
>
> Key: FLINK-30624
> URL: https://issues.apache.org/jira/browse/FLINK-30624
> Project: Flink
> Issue Type: Bug
> Components: Runtime / State Backends
> Reporter: Martijn Visser
> Assignee: Lijie Wang
> Priority: Blocker
> Fix For: 1.17.0
>
>
> stateBackends.FS median=4147.01197 recent_median=3957.8419495
> stateBackends.FS_ASYNC median=4148.8160595 recent_median=3973.418166
> stateBackends.MEMORY median=4114.406091 recent_median=3935.8805775
> http://codespeed.dak8s.net:8000/timeline/#/?exe=1&ben=stateBackends.FS&extr=on&quarts=on&equid=off&env=2&revs=200
> http://codespeed.dak8s.net:8000/timeline/#/?exe=1&ben=stateBackends.FS_ASYNC&extr=on&quarts=on&equid=off&env=2&revs=200
> http://codespeed.dak8s.net:8000/timeline/#/?exe=1&ben=stateBackends.MEMORY&extr=on&quarts=on&equid=off&env=2&revs=200
--
This message was sent by Atlassian Jira
(v8.20.10#820010)