[ https://issues.apache.org/jira/browse/FLINK-9913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16593681#comment-16593681 ]
ASF GitHub Bot commented on FLINK-9913: --------------------------------------- NicoK 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_r212951771 ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java ########## @@ -87,62 +86,71 @@ public RecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSele this.numChannels = writer.getNumberOfSubpartitions(); - /* - * The runtime exposes a channel abstraction for the produced results - * (see {@link ChannelSelector}). Every channel has an independent - * serializer. - */ - this.serializers = new SpanningRecordSerializer[numChannels]; + this.serializer = new SpanningRecordSerializer<T>(); this.bufferBuilders = new Optional[numChannels]; for (int i = 0; i < numChannels; i++) { - serializers[i] = new SpanningRecordSerializer<T>(); bufferBuilders[i] = Optional.empty(); } } public void emit(T record) throws IOException, InterruptedException { + serializer.serializeRecord(record); + for (int targetChannel : channelSelector.selectChannels(record, numChannels)) { - sendToTarget(record, targetChannel); + copyToTarget(targetChannel); } + + // Make sure we don't hold onto the large intermediate serialization buffer for too long + serializer.prune(); } /** * This is used to broadcast Streaming Watermarks in-band with records. This ignores * the {@link ChannelSelector}. */ public void broadcastEmit(T record) throws IOException, InterruptedException { + serializer.serializeRecord(record); + for (int targetChannel = 0; targetChannel < numChannels; targetChannel++) { - sendToTarget(record, targetChannel); + copyToTarget(targetChannel); } + + serializer.prune(); } /** * This is used to send LatencyMarks to a random target channel. */ public void randomEmit(T record) throws IOException, InterruptedException { - sendToTarget(record, rng.nextInt(numChannels)); - } + serializer.serializeRecord(record); - private void sendToTarget(T record, int targetChannel) throws IOException, InterruptedException { - RecordSerializer<T> serializer = serializers[targetChannel]; + copyToTarget(rng.nextInt(numChannels)); - SerializationResult result = serializer.addRecord(record); + serializer.prune(); + } + private void copyToTarget(int targetChannel) throws IOException, InterruptedException { + // We should reset the initial position of the intermediate serialization buffer before + // copying, so the serialization results can be copied to multiple target buffers. + serializer.reset(); + + BufferBuilder bufferBuilder = getBufferBuilder(targetChannel); + SerializationResult result = serializer.copyToBufferBuilder(bufferBuilder); while (result.isFullBuffer()) { - if (tryFinishCurrentBufferBuilder(targetChannel, serializer)) { - // If this was a full record, we are done. Not breaking - // out of the loop at this point will lead to another - // buffer request before breaking out (that would not be - // a problem per se, but it can lead to stalls in the - // pipeline). - if (result.isFullRecord()) { - break; - } + tryFinishCurrentBufferBuilder(targetChannel); Review comment: actually, here, we do not only know that the buffer builder is present, we also already have its reference (in contrast to `tryFinishCurrentBufferBuilder()`) and don't need to update the `bufferBuilders` field until after the `while` loop - I'm not sure whether this is worth optimising, though (@pnowojski?) ---------------------------------------------------------------- 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)