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_r384934330
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandler.java ########## @@ -290,43 +290,20 @@ private void decodeMsg(Object msg) throws Throwable { } private void decodeBufferOrEvent(RemoteInputChannel inputChannel, NettyMessage.BufferResponse bufferOrEvent) throws Throwable { - try { - ByteBuf nettyBuffer = bufferOrEvent.getNettyBuffer(); - final int receivedSize = nettyBuffer.readableBytes(); - if (bufferOrEvent.isBuffer()) { - // ---- Buffer ------------------------------------------------ - - // Early return for empty buffers. Otherwise Netty's readBytes() throws an - // IndexOutOfBoundsException. - if (receivedSize == 0) { - inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - return; - } - - Buffer buffer = inputChannel.requestBuffer(); - if (buffer != null) { - nettyBuffer.readBytes(buffer.asByteBuf(), receivedSize); - buffer.setCompressed(bufferOrEvent.isCompressed); - - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } else if (inputChannel.isReleased()) { - cancelRequestFor(bufferOrEvent.receiverId); - } else { - throw new IllegalStateException("No buffer available in credit-based input channel."); - } - } else { - // ---- Event ------------------------------------------------- - // TODO We can just keep the serialized data in the Netty buffer and release it later at the reader - byte[] byteArray = new byte[receivedSize]; - nettyBuffer.readBytes(byteArray); + // Early return for empty buffers. + if (bufferOrEvent.isBuffer() && bufferOrEvent.bufferSize == 0) { + inputChannel.onEmptyBuffer(bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + return; + } - MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray); - Buffer buffer = new NetworkBuffer(memSeg, FreeingBufferRecycler.INSTANCE, false, receivedSize); + Buffer dataBuffer = bufferOrEvent.getBuffer(); - inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); - } - } finally { - bufferOrEvent.releaseBuffer(); + if (dataBuffer != null) { + inputChannel.onBuffer(dataBuffer, bufferOrEvent.sequenceNumber, bufferOrEvent.backlog); + } else if (inputChannel.isReleased()) { + cancelRequestFor(bufferOrEvent.receiverId); Review comment: I guess we also need to release the `BufferOrEvent` for some cases except for `inputChannel.onBuffer`, otherwise it might be leaked. ---------------------------------------------------------------- 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