zhijiangW commented on a change in pull request #7368: [FLINK-10742][network] 
Let Netty use Flink's buffers directly in credit-based mode
URL: https://github.com/apache/flink/pull/7368#discussion_r376218252
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ZeroCopyNettyMessageDecoder.java
 ##########
 @@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.netty;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+
+import static 
org.apache.flink.runtime.io.network.netty.NettyMessage.FRAME_HEADER_LENGTH;
+import static 
org.apache.flink.runtime.io.network.netty.NettyMessage.MAGIC_NUMBER;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Decodes messages from the fragmentary netty buffers. This decoder assumes 
the
+ * messages have the following format:
+ * +-----------------------------------+--------------------------------+
+ * | FRAME_HEADER ||  MESSAGE_HEADER   |     DATA BUFFER (Optional)     |
+ * +-----------------------------------+--------------------------------+
+ * and it decodes each part in order.
+ *
+ * This decoder tries best to eliminate copying. For the frame header and 
message header,
+ * it only cumulates data when they span multiple input buffers. For the 
buffer part, it
+ * copies directly to the input channels to avoid future copying.
+ *
+ * The format of the frame header is
+ * +------------------+------------------+--------+
+ * | FRAME LENGTH (4) | MAGIC NUMBER (4) | ID (1) |
+ * +------------------+------------------+--------+
+ */
+public class ZeroCopyNettyMessageDecoder extends ChannelInboundHandlerAdapter {
+
+    private static final int INITIAL_MESSAGE_HEADER_BUFFER_LENGTH = 128;
+
+    /** The parser to parse the message header. */
+    private final NettyMessageParser messageParser;
+
+    /** The buffer used to cumulate the frame header part. */
+    private ByteBuf frameHeaderBuffer;
+
+    /** The buffer used to receive the message header part. */
+    private ByteBuf messageHeaderBuffer;
+
+    /** Which part of the current message is being decoded. */
+    private DecodeStep decodeStep;
+
+    /** How many bytes have been decoded in current step. */
+    private int decodedBytesOfCurrentStep;
+
+    /** The intermediate state when decoding the current message. */
+    private final MessageDecodeIntermediateState intermediateState;
+
+    ZeroCopyNettyMessageDecoder(NettyMessageParser messageParser) {
+        this.messageParser = messageParser;
+        this.intermediateState = new MessageDecodeIntermediateState();
+    }
+
+    @Override
+    public void channelActive(ChannelHandlerContext ctx) throws Exception {
+        super.channelActive(ctx);
+
+        frameHeaderBuffer = 
ctx.alloc().directBuffer(NettyMessage.FRAME_HEADER_LENGTH);
+        messageHeaderBuffer = 
ctx.alloc().directBuffer(INITIAL_MESSAGE_HEADER_BUFFER_LENGTH);
+
+        decodeStep = DecodeStep.DECODING_FRAME_HEADER;
+    }
+
+    @Override
+    public void channelInactive(ChannelHandlerContext ctx) throws Exception {
+        super.channelInactive(ctx);
+
+        if (intermediateState.messageHeaderParseResult != null) {
+            Buffer buffer = 
intermediateState.messageHeaderParseResult.getTargetDataBuffer();
+
+            if (buffer != null) {
+                buffer.recycleBuffer();
+            }
+        }
+
+        clearState();
+
+        frameHeaderBuffer.release();
+        messageHeaderBuffer.release();
+    }
+
+    @Override
+    public void channelRead(ChannelHandlerContext ctx, Object msg) throws 
Exception {
+        if (!(msg instanceof ByteBuf)) {
+            ctx.fireChannelRead(msg);
+            return;
+        }
+
+        ByteBuf data = (ByteBuf) msg;
+
+        try {
+            while (data.readableBytes() > 0) {
+                if (decodeStep == DecodeStep.DECODING_FRAME_HEADER) {
+                    ByteBuf toDecode = 
cumulateBufferIfNeeded(frameHeaderBuffer, data, FRAME_HEADER_LENGTH);
+
+                    if (toDecode != null) {
+                        decodeFrameHeader(toDecode);
+
+                        decodedBytesOfCurrentStep = 0;
+                        decodeStep = DecodeStep.DECODING_MESSAGE_HEADER;
+                    }
+                }
+
+                if (decodeStep == DecodeStep.DECODING_MESSAGE_HEADER) {
+                    ByteBuf toDecoder = 
cumulateBufferIfNeeded(messageHeaderBuffer, data, 
intermediateState.messageHeaderLength);
+
+                    if (toDecoder != null) {
+                        intermediateState.messageHeaderParseResult = 
messageParser.parseMessageHeader(intermediateState.msgId, toDecoder);
+
+                        if 
(intermediateState.messageHeaderParseResult.getDataBufferAction() ==
+                                
NettyMessageParser.DataBufferAction.NO_DATA_BUFFER) {
+                            
ctx.fireChannelRead(intermediateState.messageHeaderParseResult.getParsedMessage());
+                            clearState();
+                        } else {
+                            decodedBytesOfCurrentStep = 0;
+                            decodeStep = DecodeStep.DECODING_BUFFER;
+                        }
+                    }
+                }
+
+                if (decodeStep == DecodeStep.DECODING_BUFFER) {
+                    readOrDiscardBufferResponse(data);
+
+                    if (decodedBytesOfCurrentStep == 
intermediateState.messageHeaderParseResult.getDataBufferSize()) {
+                        
ctx.fireChannelRead(intermediateState.messageHeaderParseResult.getParsedMessage());
 
 Review comment:
   If we can also cancel the partition directly while requesting the null 
buffer from `RemoteInputChannel` as concerned in 
https://github.com/apache/flink/pull/7368#pullrequestreview-354930094, then we 
do not need to fire it to the upper handler.
   
   The `NetworkClientHandler` interface already provides the cancel request 
method. If the decoder covers this component which can be also used for getting 
input channel for requesting buffer in allocator, then it is feasible to do 
that.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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


With regards,
Apache Git Services

Reply via email to