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_r384937643
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java ########## @@ -0,0 +1,377 @@ +/* + * 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.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.io.network.ConnectionID; +import org.apache.flink.runtime.io.network.ConnectionManager; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; +import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertTrue; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.BufferResponse; +import static org.apache.flink.runtime.io.network.netty.NettyMessage.ErrorResponse; +import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.powermock.api.mockito.PowerMockito.spy; +import static org.powermock.api.mockito.PowerMockito.when; + +/** + * Tests the client side message decoder. + */ +public class NettyMessageClientDecoderDelegateTest { + private static final NettyBufferPool ALLOCATOR = new NettyBufferPool(1); + + private static final InputChannelID NORMAL_INPUT_CHANNEL_ID = new InputChannelID(); + private static final InputChannelID RELEASED_INPUT_CHANNEL_ID = new InputChannelID(); + + /** + * Verifies that the client side decoder works well for unreleased input channels. + */ + @Test + public void testDownstreamMessageDecode() throws Exception { + // 6 buffers required for running 2 rounds and 3 buffers each round. + EmbeddedChannel channel = new EmbeddedChannel( + new NettyMessageClientDecoderDelegate(createPartitionRequestClientHandler(6))); + Buffer event = createDataBuffer(32); + event.tagAsEvent(); + + NettyMessage[] messages = new NettyMessage[]{ + new NettyMessage.BufferResponse(createDataBuffer(128), 0, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.BufferResponse(createDataBuffer(256), 1, NORMAL_INPUT_CHANNEL_ID, 3), + new NettyMessage.BufferResponse(event, 2, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.ErrorResponse(new RuntimeException("test"), NORMAL_INPUT_CHANNEL_ID), + new NettyMessage.BufferResponse(createDataBuffer(56), 3, NORMAL_INPUT_CHANNEL_ID, 4) + }; + + ByteBuf[] serializedBuffers = null; + ByteBuf mergedBuffer = null; + + try { + serializedBuffers = serializeMessages(messages); + int[] sizes = getBufferSizes(serializedBuffers); + mergedBuffer = mergeBuffers(serializedBuffers); + + ByteBuf[] splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0] + sizes[1] + sizes[2] / 3, + sizes[0] + sizes[1] + sizes[2] + sizes[3] / 3 * 2, + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] / 3 * 2 + }); + readInputAndVerify(channel, splitBuffers, messages); + + splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0] + sizes[1] / 3, + sizes[0] + sizes[1] + sizes[2] / 3, + sizes[0] + sizes[1] + sizes[2] + sizes[3], + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] / 3 * 2 + }); + readInputAndVerify(channel, splitBuffers, messages); + } finally { + if (serializedBuffers != null) { + releaseBuffers(serializedBuffers); + } + + if (mergedBuffer != null) { + mergedBuffer.release(); + } + } + } + + /** + * Verifies that the client side decoder works well for empty buffers. Empty buffers should not + * consume data buffers of the input channels. + */ + @Test + public void testDownstreamMessageDecodeWithEmptyBuffers() throws Exception { + // 4 buffers required for running 2 rounds and 2 buffers each round. + EmbeddedChannel channel = new EmbeddedChannel( + new NettyMessageClientDecoderDelegate(createPartitionRequestClientHandler(4))); + Buffer event = createDataBuffer(32); + event.tagAsEvent(); + + NettyMessage[] messages = new NettyMessage[]{ + new NettyMessage.BufferResponse(createDataBuffer(128), 0, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.BufferResponse(createDataBuffer(0), 1, NORMAL_INPUT_CHANNEL_ID, 3), + new NettyMessage.BufferResponse(event, 2, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.ErrorResponse(new RuntimeException("test"), NORMAL_INPUT_CHANNEL_ID), + new NettyMessage.BufferResponse(createDataBuffer(56), 3, NORMAL_INPUT_CHANNEL_ID, 4) + }; + + ByteBuf[] serializedBuffers = null; + ByteBuf mergedBuffer = null; + + try { + serializedBuffers = serializeMessages(messages); + int[] sizes = getBufferSizes(serializedBuffers); + mergedBuffer = mergeBuffers(serializedBuffers); + + ByteBuf[] splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0] + sizes[1] + sizes[2] / 3 * 2, + sizes[0] + sizes[1] + sizes[2] + sizes[3] / 3, + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] + }); + readInputAndVerify(channel, splitBuffers, messages); + + splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0] + sizes[1] / 3, + sizes[0] + sizes[1] + sizes[2] / 3, + sizes[0] + sizes[1] + sizes[2] + sizes[3], + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] / 3 * 2 + }); + readInputAndVerify(channel, splitBuffers, messages); + } finally { + if (serializedBuffers != null) { + releaseBuffers(serializedBuffers); + } + + if (mergedBuffer != null) { + mergedBuffer.release(); + } + } + } + + /** + * Verifies that NettyMessageDecoder works well with buffers sent to a released channel. + * For such a channel, no Buffer is available to receive the data buffer in the message, + * and the data buffer part should be discarded before reading the next message. + */ + @Test + public void testDownstreamMessageDecodeWithReleasedInputChannel() throws Exception { + // 6 buffers required for running 2 rounds and 3 buffers each round. + EmbeddedChannel channel = new EmbeddedChannel( + new NettyMessageClientDecoderDelegate(createPartitionRequestClientHandler(6))); + + Buffer event = createDataBuffer(32); + event.tagAsEvent(); + + NettyMessage[] messages = new NettyMessage[]{ + new NettyMessage.BufferResponse(createDataBuffer(128), 0, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.BufferResponse(createDataBuffer(256), 1, RELEASED_INPUT_CHANNEL_ID, 3), + new NettyMessage.BufferResponse(event, 2, NORMAL_INPUT_CHANNEL_ID, 4), + new NettyMessage.ErrorResponse(new RuntimeException("test"), RELEASED_INPUT_CHANNEL_ID), + new NettyMessage.BufferResponse(createDataBuffer(64), 3, NORMAL_INPUT_CHANNEL_ID, 4), + }; + + ByteBuf[] serializedBuffers = null; + ByteBuf mergedBuffer = null; + + try { + serializedBuffers = serializeMessages(messages); + int[] sizes = getBufferSizes(serializedBuffers); + mergedBuffer = mergeBuffers(serializedBuffers); + + ByteBuf[] splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0] + sizes[1] / 3, + sizes[0] + sizes[1] + sizes[2] / 3, + sizes[0] + sizes[1] + sizes[2], + sizes[0] + sizes[1] + sizes[2] + sizes[3], + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] / 3 * 2 + }); + readInputAndVerify(channel, splitBuffers, messages); + + splitBuffers = partitionBuffer(mergedBuffer, new int[]{ + sizes[0] / 3, + sizes[0], + sizes[0] + sizes[1] / 3, + sizes[0] + sizes[1] / 3 * 2, + sizes[0] + sizes[1] + sizes[2] / 3, + sizes[0] + sizes[1] + sizes[2] + sizes[3] + sizes[4] / 3, + }); + readInputAndVerify(channel, splitBuffers, messages); + } finally { + if (serializedBuffers != null) { + releaseBuffers(serializedBuffers); + } + + if (mergedBuffer != null) { + mergedBuffer.release(); + } + } + } + + //------------------------------------------------------------------------------------------------------------------ + + private void readInputAndVerify(EmbeddedChannel channel, ByteBuf[] inputBuffers, NettyMessage[] expected) throws Exception { + for (ByteBuf buffer : inputBuffers) { + channel.writeInbound(buffer); + } + + channel.runPendingTasks(); + + List<NettyMessage> decodedMessages = new ArrayList<>(); + Object input; + while ((input = channel.readInbound()) != null) { + assertTrue(input instanceof NettyMessage); + decodedMessages.add((NettyMessage) input); + } + + assertEquals(expected.length, decodedMessages.size()); + for (int i = 0; i < expected.length; ++i) { + assertEquals(expected[i].getClass(), decodedMessages.get(i).getClass()); + + if (expected[i] instanceof NettyMessage.BufferResponse) { + BufferResponse expectedBufferResponse = (BufferResponse) expected[i]; + BufferResponse decodedBufferResponse = (BufferResponse) decodedMessages.get(i); + + assertEquals(expectedBufferResponse.backlog, decodedBufferResponse.backlog); + assertEquals(expectedBufferResponse.sequenceNumber, decodedBufferResponse.sequenceNumber); + assertEquals(expectedBufferResponse.bufferSize, decodedBufferResponse.bufferSize); + assertEquals(expectedBufferResponse.receiverId, decodedBufferResponse.receiverId); + + if (expectedBufferResponse.receiverId.equals(RELEASED_INPUT_CHANNEL_ID) || expectedBufferResponse.bufferSize == 0) { + assertNull(decodedBufferResponse.getBuffer()); + } else { + assertEquals(expectedBufferResponse.getBuffer(), decodedBufferResponse.getBuffer()); + } + } else if (expected[i] instanceof NettyMessage.ErrorResponse) { + ErrorResponse expectedErrorResponse = (ErrorResponse) expected[i]; + ErrorResponse decodedErrorResponse = (ErrorResponse) decodedMessages.get(i); + + assertEquals(expectedErrorResponse.receiverId, decodedErrorResponse.receiverId); + assertEquals(expectedErrorResponse.cause.getClass(), decodedErrorResponse.cause.getClass()); + assertEquals(expectedErrorResponse.cause.getMessage(), decodedErrorResponse.cause.getMessage()); + } else { + fail("Unsupported message type"); + } + } + } + + private ByteBuf[] serializeMessages(NettyMessage[] messages) throws Exception { + ByteBuf[] serializedBuffers = new ByteBuf[messages.length]; + for (int i = 0; i < messages.length; ++i) { + serializedBuffers[i] = messages[i].write(ALLOCATOR); + } + + return serializedBuffers; + } + + private int[] getBufferSizes(ByteBuf[] buffers) { + int[] sizes = new int[buffers.length]; + for (int i = 0; i < sizes.length; ++i) { + sizes[i] = buffers[i].readableBytes(); + } + + return sizes; + } + + private ByteBuf mergeBuffers(ByteBuf[] buffers) { + ByteBuf allData = ALLOCATOR.buffer(); + for (ByteBuf buffer : buffers) { + allData.writeBytes(buffer); + } + + return allData; + } + + private ByteBuf[] partitionBuffer(ByteBuf buffer, int[] partitionPositions) { + ByteBuf[] result = new ByteBuf[partitionPositions.length + 1]; + for (int i = 0; i < result.length; ++i) { + int startPos = (i == 0 ? 0 : partitionPositions[i - 1]); + int endPos = (i < partitionPositions.length ? partitionPositions[i] : buffer.readableBytes()); + + result[i] = ALLOCATOR.buffer(); + result[i].writeBytes(buffer, startPos, endPos - startPos); + } + + return result; + } + + private void releaseBuffers(ByteBuf[] buffers) { + for (ByteBuf buffer : buffers) { + buffer.release(); + } + } + + private Buffer createDataBuffer(int size) { + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(size); + NetworkBuffer buffer = new NetworkBuffer(segment, FreeingBufferRecycler.INSTANCE); + for (int i = 0; i < size / 4; ++i) { + buffer.writeInt(i); + } + + return buffer; + } + + private CreditBasedPartitionRequestClientHandler createPartitionRequestClientHandler(int numberOfBuffersInNormalChannel) throws IOException { + NetworkBufferPool networkBufferPool = new NetworkBufferPool( + numberOfBuffersInNormalChannel, + 32 * 1024, + numberOfBuffersInNormalChannel); + + SingleInputGate singleInputGate = createSingleInputGate(2); + + CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); + + RemoteInputChannel normalInputChannel = spy(new RemoteInputChannel( + singleInputGate, + 0, + new ResultPartitionID(), + mock(ConnectionID.class), Review comment: we should avoid mocking in tests, also for other places. ---------------------------------------------------------------- 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