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

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

zhijiangW commented on issue #6417: [FLINK-9913][runtime] Improve output 
serialization only once in RecordWriter
URL: https://github.com/apache/flink/pull/6417#issuecomment-416629286
 
 
   Thanks for your reviews @NicoK 
   
   Sorry for the late updates with this PR because I am a little busy recently, 
also regarding with the benchmark results.
   
   For my own broadcast benchmark, this changes gain obvious improvement. But 
for non-broadcast cases, the throughput of 
`StreamNetworkThroughputBenchmarkExecutor` seems a bit decreased than before. 
After I adjusted to keep the same process of `pruneBuffer()` as before, the 
results seem a bit better than current, but still has a bit decrease (1% 
sometimes) than before. So I guess another reason is in the past the 
`RecordSerializer` will maintain the `BufferBuilder` internally and keep 
copying multi serialization results until full. But now for each record we have 
to get the `BufferBuilder` from the arrays in `RecordWriter` then pass it to 
the `RecordSerializer`. And this is the key difference and overhead because the 
`RecordSerializer` is stateless. So I am still trying to improve other parts to 
compensate this loss.
   
   I am trying to update this PR soon based on all the above comments!

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