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

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

zhijiangW 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_r213345034
 
 

 ##########
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
 ##########
 @@ -169,25 +167,26 @@ private void test(Util.MockRecords records, int 
segmentSize) throws Exception {
 
                // 
-------------------------------------------------------------------------------------------------------------
 
-               
serializer.continueWritingWithNextBufferBuilder(createBufferBuilder(segmentSize));
-
+               BufferBuilder bufferBuilder = createBufferBuilder(segmentSize);
                int numBytes = 0;
                for (SerializationTestType record : records) {
-                       RecordSerializer.SerializationResult result = 
serializer.addRecord(record);
+                       serializer.serializeRecord(record);
+                       RecordSerializer.SerializationResult result = 
serializer.copyToBufferBuilder(bufferBuilder);
                        numBytes += record.length() + serializationOverhead;
 
                        if (numBytes < segmentSize) {
                                
Assert.assertEquals(RecordSerializer.SerializationResult.FULL_RECORD, result);
                        } else if (numBytes == segmentSize) {
                                
Assert.assertEquals(RecordSerializer.SerializationResult.FULL_RECORD_MEMORY_SEGMENT_FULL,
 result);
-                               
serializer.continueWritingWithNextBufferBuilder(createBufferBuilder(segmentSize));
+                               bufferBuilder = 
createBufferBuilder(segmentSize);
                                numBytes = 0;
                        } else {
                                
Assert.assertEquals(RecordSerializer.SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL,
 result);
 
                                while (result.isFullBuffer()) {
                                        numBytes -= segmentSize;
-                                       result = 
serializer.continueWritingWithNextBufferBuilder(createBufferBuilder(segmentSize));
+                                       bufferBuilder = 
createBufferBuilder(segmentSize);
+                                       result = 
serializer.copyToBufferBuilder(bufferBuilder);
 
 Review comment:
   make sense

----------------------------------------------------------------
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