[ 
https://issues.apache.org/jira/browse/FLINK-9913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16605750#comment-16605750
 ] 

ASF GitHub Bot commented on FLINK-9913:
---------------------------------------

pnowojski commented on a change in pull request #6417: [FLINK-9913][runtime] 
Improve output serialization only once in RecordWriter
URL: https://github.com/apache/flink/pull/6417#discussion_r215611481
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java
 ##########
 @@ -197,40 +214,39 @@ public void setMetricGroup(TaskIOMetricGroup metrics) {
         * Marks the current {@link BufferBuilder} as finished and clears the 
state for next one.
         */
        private void tryFinishCurrentBufferBuilder(int targetChannel) {
-               if (bufferBuilders[targetChannel].isPresent()) {
-                       BufferBuilder bufferBuilder = 
bufferBuilders[targetChannel].get();
+               Optional<BufferBuilder> bufferBuilderOpt = 
bufferBuilders[targetChannel];
+               if (bufferBuilderOpt.isPresent()) {
                        bufferBuilders[targetChannel] = Optional.empty();
-                       numBytesOut.inc(bufferBuilder.finish());
+                       numBytesOut.inc(bufferBuilderOpt.get().finish());
                        numBuffersOut.inc();
                }
        }
-       
+
        /**
         * The {@link BufferBuilder} may already exist if not filled up last 
time, otherwise we need
         * request a new one for this target channel.
         */
-       @Nonnull
        private BufferBuilder getBufferBuilder(int targetChannel) throws 
IOException, InterruptedException {
-               if (bufferBuilders[targetChannel].isPresent()) {
-                       return bufferBuilders[targetChannel].get();
+               Optional<BufferBuilder> bufferBuilderOpt = 
bufferBuilders[targetChannel];
+               if (bufferBuilderOpt.isPresent()) {
+                       return bufferBuilderOpt.get();
                } else {
                        return requestNewBufferBuilder(targetChannel);
                }
        }
 
-       @Nonnull
        private BufferBuilder requestNewBufferBuilder(int targetChannel) throws 
IOException, InterruptedException {
                checkState(!bufferBuilders[targetChannel].isPresent());
-
                BufferBuilder bufferBuilder = 
targetPartition.getBufferProvider().requestBufferBuilderBlocking();
                bufferBuilders[targetChannel] = Optional.of(bufferBuilder);
                
targetPartition.addBufferConsumer(bufferBuilder.createBufferConsumer(), 
targetChannel);
                return bufferBuilder;
        }
 
        private void closeBufferBuilder(int targetChannel) {
-               if (bufferBuilders[targetChannel].isPresent()) {
-                       bufferBuilders[targetChannel].get().finish();
+               Optional<BufferBuilder> bufferBuilderOpt = 
bufferBuilders[targetChannel];
 
 Review comment:
   ditto: rename or inline

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Improve output serialization only once in RecordWriter
> ------------------------------------------------------
>
>                 Key: FLINK-9913
>                 URL: https://issues.apache.org/jira/browse/FLINK-9913
>             Project: Flink
>          Issue Type: Improvement
>          Components: Network
>    Affects Versions: 1.6.0
>            Reporter: zhijiang
>            Assignee: zhijiang
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> Currently the {{RecordWriter}} emits output into multi channels via 
> {{ChannelSelector}}  or broadcasts output to all channels directly. Each 
> channel has a separate {{RecordSerializer}} for serializing outputs, that 
> means the output will be serialized as many times as the number of selected 
> channels.
> As we know, data serialization is a high cost operation, so we can get good 
> benefits by improving the serialization only once.
> I would suggest the following changes for realizing it.
>  # Only one {{RecordSerializer}} is created in {{RecordWriter}} for all the 
> channels.
>  # The output is serialized into the intermediate data buffer only once for 
> different channels.
>  # The intermediate serialization results are copied into different 
> {{BufferBuilder}}s for different channels.
> An additional benefit by using a single serializer for all channels is that 
> we get a potentially significant reduction on heap space overhead from fewer 
> intermediate serialization buffers (only once we got over 5MiB, these buffers 
> were pruned back to 128B!).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to