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

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_r209290889
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/RecordSerializer.java
 ##########
 @@ -66,29 +66,33 @@ public boolean isFullBuffer() {
        }
 
        /**
-        * Starts serializing and copying the given record to the target buffer
-        * (if available).
+        * Starts serializing the given record to an intermediate data buffer.
         *
         * @param record the record to serialize
-        * @return how much information was written to the target buffer and
-        *         whether this buffer is full
         */
-       SerializationResult addRecord(T record) throws IOException;
+       void serializeRecord(T record) throws IOException;
 
 Review comment:
   I'm thinking about refactoring this class and splitting it into two:
   ```
   class RecordSerializer {
    SerializedRecord serializeRecord(T record);
   };
   
   class SerializedRecord implements Autoclosable {
     CopingResult copyToBufferBuilder(BufferBuilder bufferBuilder);
   
     void close() {
        serializer.prune();
        // and code to return state (serializationBuffer) to serializer for 
reuse
     }
   }
   ```
   
   and usage:
   ```
        public void randomEmit(T record) throws IOException, 
InterruptedException {
                try (SerializedRecord serializedRecord = 
serializer.serializeRecord(record)) {
                        copyToTarget(serializedRecord, 
rng.nextInt(numChannels));
                }
        }
   ```
   
   somehow always was/is tickling my brain in current `RecordSerializer` is 
confusing to me and I have to always check it's implementation whenever I 
revisit the code. Maybe with this split it would be easier to understand? But 
I'm not sure about this. What do you think?

----------------------------------------------------------------
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: Minor
>              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.



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

Reply via email to