From 98065aa82ae2ce42c92a8e6602b2383a55fa3da7 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sun, 13 Aug 2023 16:52:41 +0800 Subject: [PATCH 1/7] [FLINK-32850][flink-runtime][JUnit5 Migration] Module: The io package of flink-runtime --- .../disk/BatchShuffleReadBufferPoolTest.java | 119 +- .../runtime/io/disk/ChannelViewsTest.java | 79 +- .../io/disk/FileChannelManagerImplTest.java | 71 +- .../io/disk/FileChannelStreamsITCase.java | 55 +- .../io/disk/FileChannelStreamsTest.java | 19 +- .../SeekableFileChannelInputViewTest.java | 23 +- .../runtime/io/disk/SpillingBufferTest.java | 89 +- .../AsynchronousBufferFileWriterTest.java | 110 +- .../AsynchronousFileIOChannelTest.java | 21 +- ...BufferFileWriterFileSegmentReaderTest.java | 38 +- .../iomanager/BufferFileWriterReaderTest.java | 48 +- .../io/disk/iomanager/IOManagerAsyncTest.java | 36 +- .../io/disk/iomanager/IOManagerITCase.java | 60 +- .../io/disk/iomanager/IOManagerTest.java | 33 +- .../io/network/DataExchangeModeTest.java | 18 +- .../network/DefaultChannelSelectorTest.java | 10 +- .../io/network/TaskEventDispatcherTest.java | 79 +- .../io/network/api/CheckpointBarrierTest.java | 8 +- .../api/reader/AbstractReaderTest.java | 31 +- .../CheckpointSerializationTest.java | 21 +- .../serialization/EventSerializerTest.java | 44 +- .../api/serialization/PagedViewsTest.java | 66 +- .../SpanningRecordSerializationTest.java | 55 +- .../serialization/SpanningWrapperTest.java | 21 +- .../api/writer/RecordWriterDelegateTest.java | 55 +- .../api/writer/SubtaskStateMapperTest.java | 169 +- .../network/buffer/AbstractByteBufTest.java | 3123 +++++++++-------- .../buffer/BufferBuilderAndConsumerTest.java | 160 +- .../buffer/BufferBuilderTestUtils.java | 6 +- .../network/buffer/BufferCompressionTest.java | 148 +- ...erConsumerWithPartialRecordLengthTest.java | 58 +- .../network/buffer/BufferPoolFactoryTest.java | 289 +- .../buffer/LocalBufferPoolDestroyTest.java | 40 +- .../network/buffer/LocalBufferPoolTest.java | 4 +- .../network/buffer/NetworkBufferPoolTest.java | 356 +- .../io/network/buffer/NetworkBufferTest.java | 228 +- .../buffer/ReadOnlySlicedBufferTest.java | 214 +- .../io/network/netty/ByteBufUtilsTest.java | 38 +- .../netty/CancelPartitionRequestTest.java | 10 +- ...tBasedSequenceNumberingViewReaderTest.java | 30 +- .../io/network/netty/NettyBufferPoolTest.java | 49 +- .../netty/NettyClientServerSslTest.java | 110 +- .../netty/NettyConnectionManagerTest.java | 38 +- .../netty/NettyLeakDetectionExtension.java | 2 +- ...NettyMessageClientDecoderDelegateTest.java | 39 +- ...ttyMessageClientSideSerializationTest.java | 3 - ...ttyMessageServerSideSerializationTest.java | 65 +- .../NettyPartitionRequestClientTest.java | 112 +- .../io/network/netty/NettyTestUtil.java | 6 +- .../PartitionRequestClientFactoryTest.java | 8 +- .../netty/PartitionRequestQueueTest.java | 153 +- .../PartitionRequestServerHandlerTest.java | 44 +- .../ServerTransportErrorHandlingTest.java | 8 +- .../AbstractPartitionTrackerTest.java | 36 +- .../network/partition/AvailabilityUtil.java | 9 +- ...dBlockingSubpartitionAvailabilityTest.java | 40 +- .../BoundedBlockingSubpartitionTest.java | 66 +- ...ndedBlockingSubpartitionWriteReadTest.java | 54 +- .../partition/BoundedDataTestBase.java | 82 +- .../partition/BufferReaderWriterUtilTest.java | 3 - .../io/network/partition/DataBufferTest.java | 152 +- .../partition/FileBufferReaderITCase.java | 32 +- .../partition/FileChannelBoundedDataTest.java | 58 +- .../partition/InputGateFairnessTest.java | 47 +- .../JobMasterPartitionTrackerImplTest.java | 133 +- .../PartialConsumePipelinedResultTest.java | 17 +- .../PipelinedApproximateSubpartitionTest.java | 58 +- ...proximateSubpartitionWithReadViewTest.java | 25 +- ...sultPartitionReleaseOnConsumptionTest.java | 28 +- .../partition/PipelinedSubpartitionTest.java | 188 +- ...PipelinedSubpartitionWithReadViewTest.java | 408 ++- .../partition/PrioritizedDequeTest.java | 34 +- .../ProducerFailedExceptionTest.java | 18 +- ...sourceManagerPartitionTrackerImplTest.java | 90 +- .../partition/ResultPartitionFactoryTest.java | 3 - .../partition/ResultPartitionManagerTest.java | 12 +- .../partition/SubpartitionTestBase.java | 65 +- .../TaskExecutorPartitionTrackerImplTest.java | 42 +- .../consumer/ChannelStatePersisterTest.java | 43 +- .../consumer/GateBuffersSpecTest.java | 3 - .../consumer/InputBuffersMetricsTest.java | 82 +- .../consumer/InputChannelIDTest.java | 12 +- .../partition/consumer/InputChannelTest.java | 76 +- .../partition/consumer/InputGateTestBase.java | 42 +- .../consumer/LocalInputChannelTest.java | 126 +- .../consumer/RecoveredInputChannelTest.java | 40 +- .../consumer/RemoteInputChannelTest.java | 693 ++-- .../partition/consumer/TestInputChannel.java | 2 +- .../consumer/UnionInputGateTest.java | 81 +- .../hybrid/HsFileDataIndexImplTest.java | 3 - .../hybrid/HsFileDataManagerTest.java | 3 - .../hybrid/HsMemoryDataSpillerTest.java | 3 - .../HsSubpartitionFileReaderImplTest.java | 3 - .../io/network/util/DeserializationUtils.java | 6 +- .../io/network/util/TestConsumerCallback.java | 4 +- 95 files changed, 4805 insertions(+), 4836 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java index 116fb5473bc4b..da40914e83163 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java @@ -19,70 +19,69 @@ import org.apache.flink.core.memory.MemorySegment; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link BatchShuffleReadBufferPool}. */ -public class BatchShuffleReadBufferPoolTest { +@Timeout(60) +class BatchShuffleReadBufferPoolTest { - @Rule public Timeout timeout = new Timeout(60, TimeUnit.SECONDS); - - @Test(expected = IllegalArgumentException.class) - public void testIllegalTotalBytes() { - createBufferPool(0, 1024); + @Test + void testIllegalTotalBytes() { + assertThatThrownBy(() -> createBufferPool(0, 1024)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testIllegalBufferSize() { - createBufferPool(32 * 1024 * 1024, 0); + @Test + void testIllegalBufferSize() { + assertThatThrownBy(() -> createBufferPool(32 * 1024 * 1024, 0)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testLargeTotalBytes() { + void testLargeTotalBytes() { BatchShuffleReadBufferPool bufferPool = createBufferPool(Long.MAX_VALUE, 1024); - assertEquals(Integer.MAX_VALUE, bufferPool.getNumTotalBuffers()); + assertThat(bufferPool.getNumTotalBuffers()).isEqualTo(Integer.MAX_VALUE); bufferPool.destroy(); } - @Test(expected = IllegalArgumentException.class) - public void testTotalBytesSmallerThanBufferSize() { - createBufferPool(4096, 32 * 1024); + @Test + void testTotalBytesSmallerThanBufferSize() { + assertThatThrownBy(() -> createBufferPool(4096, 32 * 1024)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testBufferCalculation() { + void testBufferCalculation() { long totalBytes = 32 * 1024 * 1024; for (int bufferSize = 4 * 1024; bufferSize <= totalBytes; bufferSize += 1024) { BatchShuffleReadBufferPool bufferPool = createBufferPool(totalBytes, bufferSize); - assertEquals(totalBytes, bufferPool.getTotalBytes()); - assertEquals(totalBytes / bufferSize, bufferPool.getNumTotalBuffers()); - assertTrue(bufferPool.getNumBuffersPerRequest() <= bufferPool.getNumTotalBuffers()); - assertTrue(bufferPool.getNumBuffersPerRequest() > 0); + assertThat(bufferPool.getTotalBytes()).isEqualTo(totalBytes); + assertThat(bufferPool.getNumTotalBuffers()).isEqualTo(totalBytes / bufferSize); + assertThat(bufferPool.getNumBuffersPerRequest()) + .isLessThanOrEqualTo(bufferPool.getNumTotalBuffers()); + assertThat(bufferPool.getNumBuffersPerRequest()).isGreaterThan(0); } } @Test - public void testRequestBuffers() throws Exception { + void testRequestBuffers() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = new ArrayList<>(); try { buffers.addAll(bufferPool.requestBuffers()); - assertEquals(bufferPool.getNumBuffersPerRequest(), buffers.size()); + assertThat(buffers).hasSize(bufferPool.getNumBuffersPerRequest()); } finally { bufferPool.recycle(buffers); bufferPool.destroy(); @@ -90,44 +89,44 @@ public void testRequestBuffers() throws Exception { } @Test - public void testRecycle() throws Exception { + void testRecycle() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.recycle(buffers); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); } @Test - public void testBufferOperationTimestampUpdated() throws Exception { + void testBufferOperationTimestampUpdated() throws Exception { BatchShuffleReadBufferPool bufferPool = new BatchShuffleReadBufferPool(1024, 1024); long oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); List buffers = bufferPool.requestBuffers(); - assertEquals(1, buffers.size()); + assertThat(buffers).hasSize(1); // The timestamp is updated when requesting buffers successfully - assertTrue(bufferPool.getLastBufferOperationTimestamp() > oldTimestamp); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isGreaterThan(oldTimestamp); oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); bufferPool.recycle(buffers); // The timestamp is updated when recycling buffers - assertTrue(bufferPool.getLastBufferOperationTimestamp() > oldTimestamp); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isGreaterThan(oldTimestamp); buffers = bufferPool.requestBuffers(); oldTimestamp = bufferPool.getLastBufferOperationTimestamp(); Thread.sleep(100); - assertEquals(0, bufferPool.requestBuffers().size()); + assertThat(bufferPool.requestBuffers()).isEmpty(); // The timestamp is not updated when requesting buffers is failed - assertEquals(oldTimestamp, bufferPool.getLastBufferOperationTimestamp()); + assertThat(bufferPool.getLastBufferOperationTimestamp()).isEqualTo(oldTimestamp); bufferPool.recycle(buffers); bufferPool.destroy(); } @Test - public void testBufferFulfilledByRecycledBuffers() throws Exception { + void testBufferFulfilledByRecycledBuffers() throws Exception { int numRequestThreads = 2; AtomicReference exception = new AtomicReference<>(); BatchShuffleReadBufferPool bufferPool = createBufferPool(); @@ -139,7 +138,7 @@ public void testBufferFulfilledByRecycledBuffers() throws Exception { owners[i] = new Object(); buffers.put(owners[i], bufferPool.requestBuffers()); } - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); Thread[] requestThreads = new Thread[numRequestThreads]; for (int i = 0; i < numRequestThreads; ++i) { @@ -172,20 +171,20 @@ public void testBufferFulfilledByRecycledBuffers() throws Exception { requestThread.join(); } - assertNull(exception.get()); - assertEquals(0, bufferPool.getAvailableBuffers()); - assertEquals(8, buffers.size()); + assertThat(exception.get()).isNull(); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(buffers).hasSize(8); } finally { for (Object owner : buffers.keySet()) { bufferPool.recycle(buffers.remove(owner)); } - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); bufferPool.destroy(); } } @Test - public void testMultipleThreadRequestAndRecycle() throws Exception { + void testMultipleThreadRequestAndRecycle() throws Exception { int numRequestThreads = 10; AtomicReference exception = new AtomicReference<>(); BatchShuffleReadBufferPool bufferPool = createBufferPool(); @@ -220,52 +219,52 @@ public void testMultipleThreadRequestAndRecycle() throws Exception { requestThread.join(); } - assertNull(exception.get()); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(exception.get()).isNull(); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); } finally { bufferPool.destroy(); } } @Test - public void testDestroy() throws Exception { + void testDestroy() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.recycle(buffers); - assertFalse(bufferPool.isDestroyed()); - assertEquals(bufferPool.getNumTotalBuffers(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.isDestroyed()).isFalse(); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(bufferPool.getNumTotalBuffers()); buffers = bufferPool.requestBuffers(); - assertEquals( - bufferPool.getNumTotalBuffers() - buffers.size(), bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()) + .isEqualTo(bufferPool.getNumTotalBuffers() - buffers.size()); bufferPool.destroy(); - assertTrue(bufferPool.isDestroyed()); - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.isDestroyed()).isTrue(); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); } - @Test(expected = IllegalStateException.class) - public void testRequestBuffersAfterDestroyed() throws Exception { + @Test + void testRequestBuffersAfterDestroyed() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); bufferPool.requestBuffers(); bufferPool.destroy(); - bufferPool.requestBuffers(); + assertThatThrownBy(bufferPool::requestBuffers).isInstanceOf(IllegalStateException.class); } @Test - public void testRecycleAfterDestroyed() throws Exception { + void testRecycleAfterDestroyed() throws Exception { BatchShuffleReadBufferPool bufferPool = createBufferPool(); List buffers = bufferPool.requestBuffers(); bufferPool.destroy(); bufferPool.recycle(buffers); - assertEquals(0, bufferPool.getAvailableBuffers()); + assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); } @Test - public void testDestroyWhileBlockingRequest() throws Exception { + void testDestroyWhileBlockingRequest() throws Exception { AtomicReference exception = new AtomicReference<>(); BatchShuffleReadBufferPool bufferPool = createBufferPool(); @@ -286,7 +285,7 @@ public void testDestroyWhileBlockingRequest() throws Exception { bufferPool.destroy(); requestThread.join(); - assertTrue(exception.get() instanceof IllegalStateException); + assertThat(exception.get()).isInstanceOf(IllegalStateException.class); } private BatchShuffleReadBufferPool createBufferPool(long totalBytes, int bufferSize) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java index 40257f22463b3..d6efc7a016a67 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java @@ -36,16 +36,18 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + /** */ -public class ChannelViewsTest { +class ChannelViewsTest { private static final long SEED = 649180756312423613L; private static final int KEY_MAX = Integer.MAX_VALUE; @@ -72,8 +74,8 @@ public class ChannelViewsTest { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { this.memoryManager = MemoryManagerBuilder.newBuilder() .setMemorySize(MEMORY_SIZE) @@ -82,14 +84,15 @@ public void beforeTest() { this.ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { this.ioManager.close(); if (memoryManager != null) { - Assert.assertTrue( - "Memory leak: not all segments have been returned to the memory manager.", - this.memoryManager.verifyEmpty()); + assertThat(this.memoryManager.verifyEmpty()) + .withFailMessage( + "Memory leak: not all segments have been returned to the memory manager.") + .isTrue(); this.memoryManager.shutdown(); this.memoryManager = null; } @@ -98,7 +101,7 @@ public void afterTest() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadSmallRecords() throws Exception { + void testWriteReadSmallRecords() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -141,9 +144,9 @@ public void testWriteReadSmallRecords() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } this.memoryManager.release(inView.close()); @@ -151,7 +154,7 @@ public void testWriteReadSmallRecords() throws Exception { } @Test - public void testWriteAndReadLongRecords() throws Exception { + void testWriteAndReadLongRecords() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LONG_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -192,9 +195,9 @@ public void testWriteAndReadLongRecords() throws Exception { final String v1 = rec.f1; final int k2 = readRec.f0; final String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } this.memoryManager.release(inView.close()); @@ -202,7 +205,7 @@ public void testWriteAndReadLongRecords() throws Exception { } @Test - public void testReadTooMany() throws Exception { + void testReadTooMany() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -244,16 +247,16 @@ public void testReadTooMany() throws Exception { final String v1 = rec.f1; final int k2 = readRec.f0; final String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } - Assert.fail("Expected an EOFException which did not occur."); + fail("Expected an EOFException which did not occur."); } catch (EOFException eofex) { // expected } catch (Throwable t) { // unexpected - Assert.fail("Unexpected Exception: " + t.getMessage()); + fail("Unexpected Exception: " + t.getMessage()); } this.memoryManager.release(inView.close()); @@ -261,7 +264,7 @@ public void testReadTooMany() throws Exception { } @Test - public void testReadWithoutKnownBlockCount() throws Exception { + void testReadWithoutKnownBlockCount() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -304,9 +307,9 @@ public void testReadWithoutKnownBlockCount() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } this.memoryManager.release(inView.close()); @@ -314,7 +317,7 @@ public void testReadWithoutKnownBlockCount() throws Exception { } @Test - public void testWriteReadOneBufferOnly() throws Exception { + void testWriteReadOneBufferOnly() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -357,9 +360,9 @@ public void testWriteReadOneBufferOnly() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } this.memoryManager.release(inView.close()); @@ -367,7 +370,7 @@ public void testWriteReadOneBufferOnly() throws Exception { } @Test - public void testWriteReadNotAll() throws Exception { + void testWriteReadNotAll() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_SHORT_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -411,9 +414,9 @@ public void testWriteReadNotAll() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the read record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the read record do not match.") + .isTrue(); } this.memoryManager.release(inView.close()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java index dc96ea654e2dc..6a6d8474c308b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java @@ -21,26 +21,25 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.testutils.TestJvmProcess; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.nio.file.Path; import java.time.Duration; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; /** Tests the logic of {@link FileChannelManagerImpl}. */ -public class FileChannelManagerImplTest extends TestLogger { +class FileChannelManagerImplTest { private static final Logger LOG = LoggerFactory.getLogger(FileChannelManagerImplTest.class); private static final String DIR_NAME_PREFIX = "manager-test"; @@ -54,12 +53,14 @@ public class FileChannelManagerImplTest extends TestLogger { private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10); - @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir private Path temporaryFolder; @Test - public void testFairness() throws Exception { - String directory1 = temporaryFolder.newFolder().getAbsoluteFile().getAbsolutePath(); - String directory2 = temporaryFolder.newFolder().getAbsoluteFile().getAbsolutePath(); + void testFairness() throws Exception { + String directory1 = + TempDirUtils.newFolder(temporaryFolder).getAbsoluteFile().getAbsolutePath(); + String directory2 = + TempDirUtils.newFolder(temporaryFolder).getAbsoluteFile().getAbsolutePath(); FileChannelManager fileChannelManager = new FileChannelManagerImpl(new String[] {directory1, directory2}, "test"); @@ -89,28 +90,29 @@ public void testFairness() throws Exception { threads[i].join(); } - assertEquals(counter1.get(), counter2.get()); + assertThat(counter2.get()).isEqualTo(counter1.get()); } @Test - public void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception { + void testDirectoriesCleanupOnKillWithoutCallerHook() throws Exception { testDirectoriesCleanupOnKill(false); } @Test - public void testDirectoriesCleanupOnKillWithCallerHook() throws Exception { + void testDirectoriesCleanupOnKillWithCallerHook() throws Exception { testDirectoriesCleanupOnKill(true); } private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exception { - assumeTrue( - OperatingSystem.isLinux() - || OperatingSystem.isFreeBSD() - || OperatingSystem.isSolaris() - || OperatingSystem.isMac()); - - File fileChannelDir = temporaryFolder.newFolder(); - File signalDir = temporaryFolder.newFolder(); + assumeThat( + OperatingSystem.isLinux() + || OperatingSystem.isFreeBSD() + || OperatingSystem.isSolaris() + || OperatingSystem.isMac()) + .isTrue(); + + File fileChannelDir = TempDirUtils.newFolder(temporaryFolder); + File signalDir = TempDirUtils.newFolder(temporaryFolder); File signalFile = new File(signalDir.getAbsolutePath(), SIGNAL_FILE_FOR_KILLING); FileChannelManagerTestProcess fileChannelManagerTestProcess = @@ -130,24 +132,27 @@ private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exceptio Runtime.getRuntime() .exec("kill " + fileChannelManagerTestProcess.getProcessId()); kill.waitFor(); - assertEquals("Failed to send SIG_TERM to process", 0, kill.exitValue()); + assertThat(kill.exitValue()) + .withFailMessage("Failed to send SIG_TERM to process") + .isEqualTo(0); Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) { Thread.sleep(100); } - assertFalse( - "The file channel manager test process does not terminate in time, its output is: \n" - + fileChannelManagerTestProcess.getProcessOutput(), - fileChannelManagerTestProcess.isAlive()); + assertThat(fileChannelManagerTestProcess.isAlive()) + .withFailMessage( + "The file channel manager test process does not terminate in time, its output is: \n%s", + fileChannelManagerTestProcess.getProcessOutput()) + .isFalse(); // Checks if the directories are cleared. - assertFalse( - "The file channel manager test process does not remove the tmp shuffle directories after termination, " - + "its output is \n" - + fileChannelManagerTestProcess.getProcessOutput(), - fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)); + assertThat(fileOrDirExists(fileChannelDir, DIR_NAME_PREFIX)) + .withFailMessage( + "The file channel manager test process does not remove the tmp shuffle directories after termination, its output is \n%s", + fileChannelManagerTestProcess.getProcessOutput()) + .isFalse(); } finally { fileChannelManagerTestProcess.destroy(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java index d6c24918f6bea..0ed8fb18aeed7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java @@ -31,18 +31,18 @@ import org.apache.flink.runtime.operators.testutils.PairGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.PairGenerator.Pair; import org.apache.flink.runtime.operators.testutils.PairGenerator.ValueMode; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class FileChannelStreamsITCase extends TestLogger { +class FileChannelStreamsITCase { private static final long SEED = 649180756312423613L; @@ -66,8 +66,8 @@ public class FileChannelStreamsITCase extends TestLogger { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memManager = MemoryManagerBuilder.newBuilder() .setMemorySize(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE) @@ -76,16 +76,18 @@ public void beforeTest() { ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); - assertTrue("The memory has not been properly released", memManager.verifyEmpty()); + assertThat(memManager.verifyEmpty()) + .withFailMessage("The memory has not been properly released") + .isTrue(); } // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadSmallRecords() { + void testWriteReadSmallRecords() { try { List memory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); @@ -129,7 +131,9 @@ public void testWriteReadSmallRecords() { for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(pair); readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } inView.close(); @@ -141,7 +145,7 @@ public void testWriteReadSmallRecords() { } @Test - public void testWriteAndReadLongRecords() { + void testWriteAndReadLongRecords() { try { final List memory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); @@ -185,7 +189,9 @@ public void testWriteAndReadLongRecords() { for (int i = 0; i < NUM_PAIRS_LONG; i++) { generator.next(pair); readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } inView.close(); @@ -197,7 +203,7 @@ public void testWriteAndReadLongRecords() { } @Test - public void testReadTooMany() { + void testReadTooMany() { try { final List memory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); @@ -242,8 +248,9 @@ public void testReadTooMany() { for (int i = 0; i < NUM_PAIRS_SHORT + 1; i++) { generator.next(pair); readPair.read(inView); - assertEquals( - "The re-generated and the read record do not match.", pair, readPair); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } fail("Expected an EOFException which did not occur."); } catch (EOFException eofex) { @@ -259,7 +266,7 @@ public void testReadTooMany() { } @Test - public void testWriteReadOneBufferOnly() { + void testWriteReadOneBufferOnly() { try { final List memory = memManager.allocatePages(new DummyInvokable(), 1); @@ -301,7 +308,9 @@ public void testWriteReadOneBufferOnly() { for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(pair); readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } inView.close(); @@ -313,7 +322,7 @@ public void testWriteReadOneBufferOnly() { } @Test - public void testWriteReadNotAll() { + void testWriteReadNotAll() { try { final List memory = memManager.allocatePages(new DummyInvokable(), NUM_MEMORY_SEGMENTS); @@ -357,7 +366,9 @@ public void testWriteReadNotAll() { for (int i = 0; i < NUM_PAIRS_SHORT / 2; i++) { generator.next(pair); readPair.read(inView); - assertEquals("The re-generated and the read record do not match.", pair, readPair); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } inView.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java index 1a0838345e51d..7e897da163946 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java @@ -29,19 +29,20 @@ import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.StringValue; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.io.FileWriter; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class FileChannelStreamsTest { +class FileChannelStreamsTest { @Test - public void testCloseAndDeleteOutputView() { + void testCloseAndDeleteOutputView() { try (IOManager ioManager = new IOManagerAsync()) { MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); @@ -56,14 +57,14 @@ public void testCloseAndDeleteOutputView() { // close for the first time, make sure all memory returns out.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); // close again, should not cause an exception out.close(); // delete, make sure file is removed out.closeAndDelete(); - assertFalse(new File(channel.getPath()).exists()); + assertThat(new File(channel.getPath()).exists()).isFalse(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -71,7 +72,7 @@ public void testCloseAndDeleteOutputView() { } @Test - public void testCloseAndDeleteInputView() { + void testCloseAndDeleteInputView() { try (IOManager ioManager = new IOManagerAsync()) { MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); @@ -92,14 +93,14 @@ public void testCloseAndDeleteInputView() { // close for the first time, make sure all memory returns in.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); // close again, should not cause an exception in.close(); // delete, make sure file is removed in.closeAndDelete(); - assertFalse(new File(channel.getPath()).exists()); + assertThat(new File(channel.getPath()).exists()).isFalse(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java index 36b5912c5e46a..d1650057dabd2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java @@ -27,18 +27,19 @@ import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class SeekableFileChannelInputViewTest { +class SeekableFileChannelInputViewTest { @Test - public void testSeek() { + void testSeek() { final int PAGE_SIZE = 16 * 1024; final int NUM_RECORDS = 120000; // integers across 7.x pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes) @@ -64,7 +65,7 @@ public void testSeek() { } // close for the first time, make sure all memory returns out.close(); - assertTrue(memMan.verifyEmpty()); + assertThat(memMan.verifyEmpty()).isTrue(); memMan.allocatePages(new DummyInvokable(), memory, 4); SeekableFileChannelInputView in = @@ -73,7 +74,7 @@ public void testSeek() { // read first, complete for (int i = 0; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); @@ -85,7 +86,7 @@ public void testSeek() { int i = 2 * PAGE_SIZE + PAGE_SIZE / 4; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); @@ -97,7 +98,7 @@ public void testSeek() { i = 120000 - 4; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); @@ -109,7 +110,7 @@ public void testSeek() { i = 0; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); @@ -121,7 +122,7 @@ public void testSeek() { i = PAGE_SIZE; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); @@ -133,7 +134,7 @@ public void testSeek() { i = 3 * PAGE_SIZE; in.seek(i); for (; i < NUM_RECORDS; i += 4) { - assertEquals(i, in.readInt()); + assertThat(in.readInt()).isEqualTo(i); } try { in.readInt(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java index 9db101888982e..e096e87e33476 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java @@ -33,15 +33,17 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.util.ArrayList; -public class SpillingBufferTest { +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +class SpillingBufferTest { private static final long SEED = 649180756312423613L; @@ -65,20 +67,21 @@ public class SpillingBufferTest { // -------------------------------------------------------------------------------------------- - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); if (memoryManager != null) { - Assert.assertTrue( - "Memory leak: not all segments have been returned to the memory manager.", - memoryManager.verifyEmpty()); + assertThat(memoryManager.verifyEmpty()) + .withFailMessage( + "Memory leak: not all segments have been returned to the memory manager.") + .isTrue(); memoryManager.shutdown(); memoryManager = null; } @@ -87,7 +90,7 @@ public void afterTest() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadInMemory() throws Exception { + void testWriteReadInMemory() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -126,9 +129,9 @@ public void testWriteReadInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } // re-notifyNonEmpty the data @@ -146,9 +149,9 @@ public void testWriteReadInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -156,7 +159,7 @@ public void testWriteReadInMemory() throws Exception { } @Test - public void testWriteReadTooMuchInMemory() throws Exception { + void testWriteReadTooMuchInMemory() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -196,11 +199,12 @@ public void testWriteReadTooMuchInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage( + "The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } - Assert.fail("Read too much, expected EOFException."); + fail("Read too much, expected EOFException."); } catch (EOFException eofex) { // expected } @@ -220,9 +224,9 @@ public void testWriteReadTooMuchInMemory() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -232,7 +236,7 @@ public void testWriteReadTooMuchInMemory() throws Exception { // -------------------------------------------------------------------------------------------- @Test - public void testWriteReadExternal() throws Exception { + void testWriteReadExternal() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -271,9 +275,9 @@ public void testWriteReadExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } // re-notifyNonEmpty the data @@ -291,9 +295,9 @@ public void testWriteReadExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); @@ -301,7 +305,7 @@ public void testWriteReadExternal() throws Exception { } @Test - public void testWriteReadTooMuchExternal() throws Exception { + void testWriteReadTooMuchExternal() throws Exception { final TestData.TupleGenerator generator = new TestData.TupleGenerator( SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH); @@ -341,11 +345,12 @@ public void testWriteReadTooMuchExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage( + "The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } - Assert.fail("Read too much, expected EOFException."); + fail("Read too much, expected EOFException."); } catch (EOFException eofex) { // expected } @@ -365,9 +370,9 @@ public void testWriteReadTooMuchExternal() throws Exception { int k2 = readRec.f0; String v2 = readRec.f1; - Assert.assertTrue( - "The re-generated and the notifyNonEmpty record do not match.", - k1 == k2 && v1.equals(v2)); + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } this.memoryManager.release(outView.close()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java index 17d191f406801..c50d198d39c38 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -24,12 +24,9 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.util.TestNotificationListener; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.Callable; @@ -39,14 +36,13 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Mockito.mock; /** Tests for {@link AsynchronousBufferFileWriter}. */ -public class AsynchronousBufferFileWriterTest { - @Rule public ExpectedException exception = ExpectedException.none(); +class AsynchronousBufferFileWriterTest { private static final IOManager ioManager = new IOManagerAsync(); @@ -54,82 +50,78 @@ public class AsynchronousBufferFileWriterTest { private AsynchronousBufferFileWriter writer; - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUp() throws IOException { + @BeforeEach + void setUp() throws IOException { writer = new AsynchronousBufferFileWriter( ioManager.createChannel(), new RequestQueue()); } @Test - public void testAddAndHandleRequest() throws Exception { + void testAddAndHandleRequest() throws Exception { addRequest(); - assertEquals( - "Didn't increment number of outstanding requests.", - 1, - writer.getNumberOfOutstandingRequests()); + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Didn't increment number of outstanding requests.") + .isEqualTo(1); handleRequest(); - assertEquals( - "Didn't decrement number of outstanding requests.", - 0, - writer.getNumberOfOutstandingRequests()); + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Didn't decrement number of outstanding requests.") + .isEqualTo(0); } @Test - public void testAddWithFailingWriter() throws Exception { + void testAddWithFailingWriter() throws Exception { AsynchronousBufferFileWriter writer = new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue<>()); writer.close(); - exception.expect(IOException.class); - Buffer buffer = new NetworkBuffer( MemorySegmentFactory.allocateUnpooledSegment(4096), FreeingBufferRecycler.INSTANCE); - try { - writer.writeBlock(buffer); - } finally { - if (!buffer.isRecycled()) { - buffer.recycleBuffer(); - Assert.fail("buffer not recycled"); - } - assertEquals( - "Shouln't increment number of outstanding requests.", - 0, - writer.getNumberOfOutstandingRequests()); + + assertThatThrownBy(() -> writer.writeBlock(buffer)).isInstanceOf(IOException.class); + + if (!buffer.isRecycled()) { + buffer.recycleBuffer(); + fail("buffer not recycled"); } + assertThat(writer.getNumberOfOutstandingRequests()) + .withFailMessage("Shouldn't increment number of outstanding requests.") + .isEqualTo(0); } @Test - public void testSubscribe() throws Exception { + void testSubscribe() throws Exception { final TestNotificationListener listener = new TestNotificationListener(); // Unsuccessful subscription, because no outstanding requests - assertFalse( - "Allowed to subscribe w/o any outstanding requests.", - writer.registerAllRequestsProcessedListener(listener)); + assertThat(writer.registerAllRequestsProcessedListener(listener)) + .withFailMessage("Allowed to subscribe w/o any outstanding requests.") + .isFalse(); // Successful subscription addRequest(); - assertTrue( - "Didn't allow to subscribe.", - writer.registerAllRequestsProcessedListener(listener)); + assertThat(writer.registerAllRequestsProcessedListener(listener)) + .withFailMessage("Didn't allow to subscribe.") + .isTrue(); // Test notification handleRequest(); - assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage("Listener was not notified.") + .isEqualTo(1); } @Test - public void testSubscribeAndClose() throws IOException, InterruptedException { + void testSubscribeAndClose() throws IOException, InterruptedException { final TestNotificationListener listener = new TestNotificationListener(); final AtomicReference error = new AtomicReference(); @@ -163,11 +155,13 @@ public void run() { sync.await(); - assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage("Listener was not notified.") + .isEqualTo(1); } @Test - public void testConcurrentSubscribeAndHandleRequest() throws Exception { + void testConcurrentSubscribeAndHandleRequest() throws Exception { final ExecutorService executor = Executors.newFixedThreadPool(2); final TestNotificationListener listener = new TestNotificationListener(); @@ -203,19 +197,19 @@ public Void call() throws Exception { try { if (subscribeFuture.get()) { - assertEquals( - "Race: Successfully subscribed, but was never notified.", - 1, - listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage( + "Race: Successfully subscribed, but was never notified.") + .isEqualTo(1); } else { - assertEquals( - "Race: Never subscribed successfully, but was notified.", - 0, - listener.getNumberOfNotifications()); + assertThat(listener.getNumberOfNotifications()) + .withFailMessage( + "Race: Never subscribed successfully, but was notified.") + .isEqualTo(0); } } catch (Throwable t) { System.out.println(i); - Assert.fail(t.getMessage()); + fail(t.getMessage()); } } } finally { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java index af85fc4073cec..751acb6a80d14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.util.TestNotificationListener; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,16 +41,15 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class AsynchronousFileIOChannelTest { +class AsynchronousFileIOChannelTest { private static final Logger LOG = LoggerFactory.getLogger(AsynchronousFileIOChannelTest.class); @Test - public void testAllRequestsProcessedListenerNotification() throws Exception { + void testAllRequestsProcessedListenerNotification() throws Exception { // -- Config ---------------------------------------------------------- final int numberOfRuns = 10; final int numberOfRequests = 100; @@ -185,7 +184,7 @@ public Void call() throws Exception { } @Test - public void testClosedButAddRequestAndRegisterListenerRace() throws Exception { + void testClosedButAddRequestAndRegisterListenerRace() throws Exception { // -- Config ---------------------------------------------------------- final int numberOfRuns = 1024; @@ -270,7 +269,7 @@ public Void call() throws Exception { } @Test - public void testClosingWaits() { + void testClosingWaits() { try (final IOManagerAsync ioMan = new IOManagerAsync()) { final int NUM_BLOCKS = 100; @@ -308,8 +307,8 @@ public void requestFailed(MemorySegment buffer, IOException e) { writer.close(); - assertEquals(NUM_BLOCKS, callbackCounter.get()); - assertFalse(exceptionOccurred.get()); + assertThat(callbackCounter.get()).isEqualTo(NUM_BLOCKS); + assertThat(exceptionOccurred.get()).isFalse(); } finally { writer.closeAndDelete(); } @@ -320,7 +319,7 @@ public void requestFailed(MemorySegment buffer, IOException e) { } @Test - public void testExceptionForwardsToClose() throws Exception { + void testExceptionForwardsToClose() throws Exception { try (IOManagerAsync ioMan = new IOManagerAsync()) { testExceptionForwardsToClose(ioMan, 100, 1); testExceptionForwardsToClose(ioMan, 100, 50); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java index 064700c9aa596..cbf1a2284ade3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java @@ -26,10 +26,10 @@ import org.apache.flink.runtime.util.event.NotificationListener; import org.apache.flink.util.IOUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; @@ -39,12 +39,10 @@ import static org.apache.flink.runtime.io.disk.iomanager.BufferFileWriterReaderTest.fillBufferWithAscendingNumbers; import static org.apache.flink.runtime.io.disk.iomanager.BufferFileWriterReaderTest.verifyBufferFilledWithAscendingNumbers; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class BufferFileWriterFileSegmentReaderTest { +class BufferFileWriterFileSegmentReaderTest { private static final int BUFFER_SIZE = 32 * 1024; @@ -60,13 +58,13 @@ public class BufferFileWriterFileSegmentReaderTest { private LinkedBlockingQueue returnedFileSegments = new LinkedBlockingQueue<>(); - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUpWriterAndReader() { + @BeforeEach + void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); try { @@ -82,8 +80,8 @@ public void setUpWriterAndReader() { } } - @After - public void tearDownWriterAndReader() { + @AfterEach + void tearDownWriterAndReader() { if (writer != null) { if (!writer.isClosed()) { IOUtils.closeQuietly(() -> writer.close()); @@ -102,7 +100,7 @@ public void tearDownWriterAndReader() { } @Test - public void testWriteRead() throws IOException, InterruptedException { + void testWriteRead() throws IOException, InterruptedException { int numBuffers = 1024; int currentNumber = 0; @@ -124,7 +122,7 @@ public void testWriteRead() throws IOException, InterruptedException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.read(); } @@ -142,10 +140,12 @@ public void onNotification() { sync.await(); } - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedFileSegments.size()); + assertThat(returnedFileSegments) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); currentNumber = 0; FileSegment fileSegment; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java index e97109b4c755c..89b9658ef670d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java @@ -25,22 +25,20 @@ import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Random; import java.util.concurrent.LinkedBlockingQueue; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class BufferFileWriterReaderTest { +class BufferFileWriterReaderTest { private static final int BUFFER_SIZE = 32 * 1024; @@ -56,13 +54,13 @@ public class BufferFileWriterReaderTest { private LinkedBlockingQueue returnedBuffers = new LinkedBlockingQueue<>(); - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { ioManager.close(); } - @Before - public void setUpWriterAndReader() { + @BeforeEach + void setUpWriterAndReader() { final FileIOChannel.ID channel = ioManager.createChannel(); try { @@ -83,8 +81,8 @@ public void setUpWriterAndReader() { } } - @After - public void tearDownWriterAndReader() { + @AfterEach + void tearDownWriterAndReader() { if (writer != null) { writer.deleteChannel(); } @@ -97,7 +95,7 @@ public void tearDownWriterAndReader() { } @Test - public void testWriteRead() throws IOException { + void testWriteRead() throws IOException { int numBuffers = 1024; int currentNumber = 0; @@ -119,16 +117,18 @@ public void testWriteRead() throws IOException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.readInto(createBuffer()); } reader.close(); - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + assertThat(returnedBuffers) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); currentNumber = 0; Buffer buffer; @@ -139,7 +139,7 @@ public void testWriteRead() throws IOException { } @Test - public void testWriteSkipRead() throws IOException { + void testWriteSkipRead() throws IOException { int numBuffers = 1024; int currentNumber = 0; @@ -165,16 +165,18 @@ public void testWriteSkipRead() throws IOException { // Read buffers back in... for (int i = 0; i < numBuffers; i++) { - assertFalse(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isFalse(); reader.readInto(createBuffer()); } reader.close(); - assertTrue(reader.hasReachedEndOfFile()); + assertThat(reader.hasReachedEndOfFile()).isTrue(); // Verify that the content is the same - assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + assertThat(returnedBuffers) + .withFailMessage("Read less buffers than written.") + .hasSize(numBuffers); // Start number after skipped buffers... currentNumber = (BUFFER_SIZE / 4) * toSkip; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java index c274a607c2779..e1c9de9b46207 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java @@ -21,19 +21,19 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class IOManagerAsyncTest { +class IOManagerAsyncTest { private IOManagerAsync ioManager; @@ -41,13 +41,13 @@ public class IOManagerAsyncTest { // Setup & Shutdown // ------------------------------------------------------------------------ - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { this.ioManager.close(); } @@ -56,7 +56,7 @@ public void afterTest() throws Exception { // ------------------------------------------------------------------------ @Test - public void channelReadWriteOneSegment() { + void channelReadWriteOneSegment() { final int NUM_IOS = 1111; try { @@ -98,7 +98,7 @@ public void channelReadWriteOneSegment() { } @Test - public void channelReadWriteMultipleSegments() { + void channelReadWriteMultipleSegments() { final int NUM_IOS = 1111; final int NUM_SEGS = 16; @@ -161,7 +161,7 @@ public void channelReadWriteMultipleSegments() { } @Test - public void testExceptionPropagationReader() { + void testExceptionPropagationReader() { try { // use atomic boolean as a boolean reference final AtomicBoolean handlerCalled = new AtomicBoolean(); @@ -199,7 +199,7 @@ public void read() throws IOException { } } - assertTrue(exceptionForwarded.get()); + assertThat(exceptionForwarded.get()).isTrue(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -207,7 +207,7 @@ public void read() throws IOException { } @Test - public void testExceptionPropagationWriter() { + void testExceptionPropagationWriter() { try { // use atomic boolean as a boolean reference final AtomicBoolean handlerCalled = new AtomicBoolean(); @@ -246,7 +246,7 @@ public void write() throws IOException { } } - assertTrue(exceptionForwarded.get()); + assertThat(exceptionForwarded.get()).isTrue(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -254,7 +254,7 @@ public void write() throws IOException { } @Test - public void testExceptionInCallbackRead() { + void testExceptionInCallbackRead() { try { final AtomicBoolean handlerCalled = new AtomicBoolean(); @@ -304,7 +304,7 @@ public void read() {} } @Test - public void testExceptionInCallbackWrite() { + void testExceptionInCallbackWrite() { try { final AtomicBoolean handlerCalled = new AtomicBoolean(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index 17ea323fa7ea2..991d879d43aa9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -26,12 +26,10 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.io.File; @@ -39,8 +37,11 @@ import java.util.List; import java.util.Random; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + /** Integration test case for the I/O manager. */ -public class IOManagerITCase extends TestLogger { +class IOManagerITCase { private static final long SEED = 649180756312423613L; @@ -56,19 +57,19 @@ public class IOManagerITCase extends TestLogger { private MemoryManager memoryManager; - @Before - public void beforeTest() { + @BeforeEach + void beforeTest() { memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } - @After - public void afterTest() throws Exception { + @AfterEach + void afterTest() throws Exception { ioManager.close(); - Assert.assertTrue( - "Not all memory was returned to the memory manager in the test.", - memoryManager.verifyEmpty()); + assertThat(memoryManager.verifyEmpty()) + .withFailMessage("Not all memory was returned to the memory manager in the test.") + .isTrue(); memoryManager.shutdown(); memoryManager = null; } @@ -82,7 +83,7 @@ public void afterTest() throws Exception { */ @Test @SuppressWarnings("unchecked") - public void parallelChannelsTest() throws Exception { + void parallelChannelsTest() throws Exception { final Random rnd = new Random(SEED); final AbstractInvokable memOwner = new DummyInvokable(); @@ -145,23 +146,22 @@ public void parallelChannelsTest() throws Exception { try { intValue = Integer.parseInt(val.value); } catch (NumberFormatException nfex) { - Assert.fail( - "Invalid value read from reader. Valid decimal number expected."); + fail("Invalid value read from reader. Valid decimal number expected."); } - Assert.assertEquals( - "Written and read values do not match during sequential read.", - nextVal, - intValue); + assertThat(intValue) + .withFailMessage( + "Written and read values do not match during sequential read.") + .isEqualTo(nextVal); nextVal++; } } catch (EOFException eofex) { // expected } - Assert.assertEquals( - "NUmber of written numbers differs from number of read numbers.", - writingCounters[i], - nextVal); + assertThat(nextVal) + .withFailMessage( + "NUmber of written numbers differs from number of read numbers.") + .isEqualTo(writingCounters[i]); this.memoryManager.release(in.close()); } @@ -189,15 +189,13 @@ public void parallelChannelsTest() throws Exception { try { intValue = Integer.parseInt(val.value); } catch (NumberFormatException nfex) { - Assert.fail( - "Invalid value read from reader. Valid decimal number expected."); + fail("Invalid value read from reader. Valid decimal number expected."); return; } - Assert.assertEquals( - "Written and read values do not match.", - readingCounters[channel]++, - intValue); + assertThat(intValue) + .withFailMessage("Written and read values do not match.") + .isEqualTo(readingCounters[channel]++); break; } catch (EOFException eofex) { @@ -222,7 +220,7 @@ public void parallelChannelsTest() throws Exception { // check that files are deleted for (int i = 0; i < NUM_CHANNELS; i++) { File f = new File(ids[i].getPath()); - Assert.assertFalse("Channel file has not been deleted.", f.exists()); + assertThat(f.exists()).withFailMessage("Channel file has not been deleted.").isFalse(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java index ce8a50cb30305..c971eacea5181 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java @@ -22,27 +22,20 @@ import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.IOException; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; -public class IOManagerTest { - - @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); +class IOManagerTest { @Test - public void channelEnumerator() throws Exception { - File tempPath = temporaryFolder.newFolder(); - + void channelEnumerator(@TempDir File tempPath) throws Exception { String[] tempDirs = new String[] { new File(tempPath, "a").getAbsolutePath(), @@ -61,12 +54,16 @@ public void channelEnumerator() throws Exception { File path = id.getPathFile(); - assertTrue("Channel IDs must name an absolute path.", path.isAbsolute()); - assertFalse("Channel IDs must name a file, not a directory.", path.isDirectory()); + assertThat(path.isAbsolute()) + .withFailMessage("Channel IDs must name an absolute path.") + .isTrue(); + assertThat(path.isDirectory()) + .withFailMessage("Channel IDs must name a file, not a directory.") + .isFalse(); - assertTrue( - "Path is not in the temp directory.", - tempPath.equals(path.getParentFile().getParentFile().getParentFile())); + assertThat(path.getParentFile().getParentFile().getParentFile()) + .withFailMessage("Path is not in the temp directory.") + .isEqualTo(tempPath); for (int k = 0; k < tempDirs.length; k++) { if (path.getParentFile().getParent().equals(tempDirs[k])) { @@ -76,7 +73,7 @@ public void channelEnumerator() throws Exception { } for (int k = 0; k < tempDirs.length; k++) { - assertEquals(3, counters[k]); + assertThat(counters[k]).isEqualTo(3); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java index 368d09e083648..da5506c78c4f4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DataExchangeModeTest.java @@ -20,31 +20,31 @@ import org.apache.flink.api.common.ExecutionMode; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertNotNull; +import static org.assertj.core.api.Assertions.assertThat; /** This test verifies that the data exchange modes are defined for every execution mode. */ -public class DataExchangeModeTest { +class DataExchangeModeTest { @Test - public void testForward() { + void testForward() { for (ExecutionMode mode : ExecutionMode.values()) { - assertNotNull(DataExchangeMode.getForForwardExchange(mode)); + assertThat(DataExchangeMode.getForForwardExchange(mode)).isNotNull(); } } @Test - public void testShuffleAndBroadcast() { + void testShuffleAndBroadcast() { for (ExecutionMode mode : ExecutionMode.values()) { - assertNotNull(DataExchangeMode.getForShuffleOrBroadcast(mode)); + assertThat(DataExchangeMode.getForShuffleOrBroadcast(mode)).isNotNull(); } } @Test - public void testPipelineBreaking() { + void testPipelineBreaking() { for (ExecutionMode mode : ExecutionMode.values()) { - assertNotNull(DataExchangeMode.getPipelineBreakingExchange(mode)); + assertThat(DataExchangeMode.getPipelineBreakingExchange(mode)).isNotNull(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DefaultChannelSelectorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DefaultChannelSelectorTest.java index 0fbb8f5f6e6c9..c47e138507891 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DefaultChannelSelectorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/DefaultChannelSelectorTest.java @@ -22,16 +22,16 @@ import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector; import org.apache.flink.types.StringValue; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** This class checks the functionality of the {@link RoundRobinChannelSelector} class. */ -public class DefaultChannelSelectorTest { +class DefaultChannelSelectorTest { /** This test checks the channel selection. */ @Test - public void channelSelect() { + void channelSelect() { final StringValue dummyRecord = new StringValue("abc"); final RoundRobinChannelSelector selector = new RoundRobinChannelSelector<>(); selector.setup(2); @@ -44,6 +44,6 @@ private void assertSelectedChannel( ChannelSelector selector, StringValue record, int expectedChannel) { int actualResult = selector.selectChannel(record); - assertEquals(expectedChannel, actualResult); + assertThat(actualResult).isEqualTo(expectedChannel); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TaskEventDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TaskEventDispatcherTest.java index f7700da0d4d55..300ed5f0bc2f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TaskEventDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/TaskEventDispatcherTest.java @@ -23,42 +23,39 @@ import org.apache.flink.runtime.iterative.event.AllWorkersDoneEvent; import org.apache.flink.runtime.iterative.event.TerminationEvent; import org.apache.flink.runtime.util.event.EventListener; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; -import static junit.framework.TestCase.assertTrue; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkState; -import static org.junit.Assert.assertFalse; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Basic tests for {@link TaskEventDispatcher}. */ -public class TaskEventDispatcherTest extends TestLogger { - - @Rule public ExpectedException expectedException = ExpectedException.none(); +class TaskEventDispatcherTest { @Test - public void registerPartitionTwice() throws Exception { + void registerPartitionTwice() { ResultPartitionID partitionId = new ResultPartitionID(); TaskEventDispatcher ted = new TaskEventDispatcher(); ted.registerPartition(partitionId); - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("already registered at task event dispatcher"); - - ted.registerPartition(partitionId); + assertThatThrownBy(() -> ted.registerPartition(partitionId)) + .hasMessageContaining("already registered at task event dispatcher") + .isInstanceOf(IllegalStateException.class); } @Test - public void subscribeToEventNotRegistered() throws Exception { + void subscribeToEventNotRegistered() { TaskEventDispatcher ted = new TaskEventDispatcher(); - - expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("not registered at task event dispatcher"); - - ted.subscribeToEvent(new ResultPartitionID(), new ZeroShotEventListener(), TaskEvent.class); + assertThatThrownBy( + () -> + ted.subscribeToEvent( + new ResultPartitionID(), + new ZeroShotEventListener(), + TaskEvent.class)) + .hasMessageContaining("not registered at task event dispatcher") + .isInstanceOf(IllegalStateException.class); } /** @@ -66,20 +63,20 @@ public void subscribeToEventNotRegistered() throws Exception { * TaskEventDispatcher#subscribeToEvent(ResultPartitionID, EventListener, Class)} methods. */ @Test - public void publishSubscribe() throws Exception { + void publishSubscribe() { ResultPartitionID partitionId1 = new ResultPartitionID(); ResultPartitionID partitionId2 = new ResultPartitionID(); TaskEventDispatcher ted = new TaskEventDispatcher(); AllWorkersDoneEvent event1 = new AllWorkersDoneEvent(); TerminationEvent event2 = new TerminationEvent(); - assertFalse(ted.publish(partitionId1, event1)); + assertThat(ted.publish(partitionId1, event1)).isFalse(); ted.registerPartition(partitionId1); ted.registerPartition(partitionId2); // no event listener subscribed yet, but the event is forwarded to a TaskEventHandler - assertTrue(ted.publish(partitionId1, event1)); + assertThat(ted.publish(partitionId1, event1)).isTrue(); OneShotEventListener eventListener1a = new OneShotEventListener(event1); ZeroShotEventListener eventListener1b = new ZeroShotEventListener(); @@ -90,23 +87,29 @@ public void publishSubscribe() throws Exception { ted.subscribeToEvent(partitionId1, eventListener2, TaskEvent.class); ted.subscribeToEvent(partitionId1, eventListener3, TerminationEvent.class); - assertTrue(ted.publish(partitionId1, event1)); - assertTrue("listener should have fired for AllWorkersDoneEvent", eventListener1a.fired); - assertFalse("listener should not have fired for AllWorkersDoneEvent", eventListener3.fired); + assertThat(ted.publish(partitionId1, event1)).isTrue(); + assertThat(eventListener1a.fired) + .withFailMessage("listener should have fired for AllWorkersDoneEvent") + .isTrue(); + assertThat(eventListener3.fired) + .withFailMessage("listener should not have fired for AllWorkersDoneEvent") + .isFalse(); // publish another event, verify that only the right subscriber is called - assertTrue(ted.publish(partitionId1, event2)); - assertTrue("listener should have fired for TerminationEvent", eventListener3.fired); + assertThat(ted.publish(partitionId1, event2)).isTrue(); + assertThat(eventListener3.fired) + .withFailMessage("listener should have fired for TerminationEvent") + .isTrue(); } @Test - public void unregisterPartition() throws Exception { + void unregisterPartition() { ResultPartitionID partitionId1 = new ResultPartitionID(); ResultPartitionID partitionId2 = new ResultPartitionID(); TaskEventDispatcher ted = new TaskEventDispatcher(); AllWorkersDoneEvent event = new AllWorkersDoneEvent(); - assertFalse(ted.publish(partitionId1, event)); + assertThat(ted.publish(partitionId1, event)).isFalse(); ted.registerPartition(partitionId1); ted.registerPartition(partitionId2); @@ -121,16 +124,20 @@ public void unregisterPartition() throws Exception { ted.unregisterPartition(partitionId2); // publish something for partitionId1 triggering all according listeners - assertTrue(ted.publish(partitionId1, event)); - assertTrue("listener should have fired for AllWorkersDoneEvent", eventListener1a.fired); - assertTrue("listener should have fired for AllWorkersDoneEvent", eventListener2.fired); + assertThat(ted.publish(partitionId1, event)).isTrue(); + assertThat(eventListener1a.fired) + .withFailMessage("listener should have fired for AllWorkersDoneEvent") + .isTrue(); + assertThat(eventListener2.fired) + .withFailMessage("listener should have fired for AllWorkersDoneEvent") + .isTrue(); // now publish something for partitionId2 which should not trigger any listeners - assertFalse(ted.publish(partitionId2, event)); + assertThat(ted.publish(partitionId2, event)).isFalse(); } @Test - public void clearAll() throws Exception { + void clearAll() throws Exception { ResultPartitionID partitionId = new ResultPartitionID(); TaskEventDispatcher ted = new TaskEventDispatcher(); ted.registerPartition(partitionId); @@ -141,7 +148,7 @@ public void clearAll() throws Exception { ted.clearAll(); - assertFalse(ted.publish(partitionId, new AllWorkersDoneEvent())); + assertThat(ted.publish(partitionId, new AllWorkersDoneEvent())).isFalse(); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java index 70de3450654cc..d704b2833ee3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java @@ -22,19 +22,19 @@ import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.fail; /** Tests for the {@link CheckpointBarrier} type. */ -public class CheckpointBarrierTest { +class CheckpointBarrierTest { /** * Test serialization of the checkpoint barrier. The checkpoint barrier does not support its own * serialization, in order to be immutable. */ @Test - public void testSerialization() throws Exception { + void testSerialization() throws Exception { long id = Integer.MAX_VALUE + 123123L; long timestamp = Integer.MAX_VALUE + 1228L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java index 969cae489971f..842f82de4625e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java @@ -26,25 +26,24 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate; import org.apache.flink.runtime.util.event.EventListener; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Matchers; import java.io.IOException; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** Tests for the event handling behaviour. */ -public class AbstractReaderTest { +class AbstractReaderTest { @Test @SuppressWarnings("unchecked") - public void testTaskEvent() throws Exception { + void testTaskEvent() throws Exception { final AbstractReader reader = new MockReader(createInputGate(1)); final EventListener listener1 = mock(EventListener.class); @@ -64,10 +63,10 @@ public void testTaskEvent() throws Exception { } @Test - public void testEndOfPartitionEvent() throws Exception { + void testEndOfPartitionEvent() throws Exception { final AbstractReader reader = new MockReader(createInputGate(1)); - assertTrue(reader.handleEvent(EndOfPartitionEvent.INSTANCE)); + assertThat(reader.handleEvent(EndOfPartitionEvent.INSTANCE)).isTrue(); } /** @@ -75,12 +74,12 @@ public void testEndOfPartitionEvent() throws Exception { * non-iterative reader. */ @Test - public void testExceptionsNonIterativeReader() throws Exception { + void testExceptionsNonIterativeReader() throws Exception { final AbstractReader reader = new MockReader(createInputGate(4)); // Non-iterative reader cannot reach end of superstep - assertFalse(reader.hasReachedEndOfSuperstep()); + assertThat(reader.hasReachedEndOfSuperstep()).isFalse(); try { reader.startNextSuperstep(); @@ -102,7 +101,7 @@ public void testExceptionsNonIterativeReader() throws Exception { } @Test - public void testEndOfSuperstepEventLogic() throws IOException { + void testEndOfSuperstepEventLogic() throws IOException { final int numberOfInputChannels = 4; final AbstractReader reader = new MockReader(createInputGate(numberOfInputChannels)); @@ -124,12 +123,12 @@ public void testEndOfSuperstepEventLogic() throws IOException { // One end of superstep event for each input channel. The superstep finishes with the last // received event. for (int i = 0; i < numberOfInputChannels - 1; i++) { - assertFalse(reader.handleEvent(eos)); - assertFalse(reader.hasReachedEndOfSuperstep()); + assertThat(reader.handleEvent(eos)).isFalse(); + assertThat(reader.hasReachedEndOfSuperstep()).isFalse(); } - assertTrue(reader.handleEvent(eos)); - assertTrue(reader.hasReachedEndOfSuperstep()); + assertThat(reader.handleEvent(eos)).isTrue(); + assertThat(reader.hasReachedEndOfSuperstep()).isTrue(); try { // Verify exception, when receiving too many end of superstep events. @@ -143,7 +142,7 @@ public void testEndOfSuperstepEventLogic() throws IOException { // Start next superstep. reader.startNextSuperstep(); - assertFalse(reader.hasReachedEndOfSuperstep()); + assertThat(reader.hasReachedEndOfSuperstep()).isFalse(); } private InputGate createInputGate(int numberOfInputChannels) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/CheckpointSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/CheckpointSerializationTest.java index 9dd3ca8222f0c..8a265e61933b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/CheckpointSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/CheckpointSerializationTest.java @@ -25,25 +25,24 @@ import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests the {@link EventSerializer} functionality for serializing {@link CheckpointBarrier * checkpoint barriers}. */ -public class CheckpointSerializationTest { +class CheckpointSerializationTest { private static final byte[] STORAGE_LOCATION_REF = new byte[] {15, 52, 52, 11, 0, 0, 0, 0, -1, -23, -19, 35}; @Test - public void testSuspendingCheckpointBarrierSerialization() throws Exception { + void testSuspendingCheckpointBarrierSerialization() throws Exception { CheckpointOptions suspendSavepointToSerialize = new CheckpointOptions( SavepointType.suspend(SavepointFormatType.CANONICAL), @@ -52,7 +51,7 @@ public void testSuspendingCheckpointBarrierSerialization() throws Exception { } @Test - public void testSavepointBarrierSerialization() throws Exception { + void testSavepointBarrierSerialization() throws Exception { CheckpointOptions savepointToSerialize = new CheckpointOptions( SavepointType.savepoint(SavepointFormatType.CANONICAL), @@ -61,7 +60,7 @@ public void testSavepointBarrierSerialization() throws Exception { } @Test - public void testCheckpointBarrierSerialization() throws Exception { + void testCheckpointBarrierSerialization() throws Exception { CheckpointOptions checkpointToSerialize = new CheckpointOptions( CheckpointType.CHECKPOINT, @@ -70,7 +69,7 @@ public void testCheckpointBarrierSerialization() throws Exception { } @Test - public void testFullCheckpointBarrierSerialization() throws Exception { + void testFullCheckpointBarrierSerialization() throws Exception { CheckpointOptions checkpointToSerialize = new CheckpointOptions( CheckpointType.FULL_CHECKPOINT, @@ -79,7 +78,7 @@ public void testFullCheckpointBarrierSerialization() throws Exception { } @Test - public void testCheckpointWithDefaultLocationSerialization() throws Exception { + void testCheckpointWithDefaultLocationSerialization() throws Exception { CheckpointOptions checkpointToSerialize = CheckpointOptions.forCheckpointWithDefaultLocation(); testCheckpointBarrierSerialization(checkpointToSerialize); @@ -94,7 +93,7 @@ private void testCheckpointBarrierSerialization(CheckpointOptions options) throw final CheckpointBarrier barrierAfterDeserialization = serializeAndDeserializeCheckpointBarrier(barrierBeforeSerialization); - assertEquals(barrierBeforeSerialization, barrierAfterDeserialization); + assertThat(barrierAfterDeserialization).isEqualTo(barrierBeforeSerialization); } private CheckpointBarrier serializeAndDeserializeCheckpointBarrier( @@ -103,7 +102,7 @@ private CheckpointBarrier serializeAndDeserializeCheckpointBarrier( final ByteBuffer serialized = EventSerializer.toSerializedEvent(barrierUnderTest); final CheckpointBarrier deserialized = (CheckpointBarrier) EventSerializer.fromSerializedEvent(serialized, cl); - assertFalse(serialized.hasRemaining()); + assertThat(serialized.hasRemaining()).isFalse(); return deserialized; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java index 00d5c09743700..18d20f1b1b119 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java @@ -37,18 +37,15 @@ import org.apache.flink.runtime.io.network.util.TestTaskEvent; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link EventSerializer}. */ -public class EventSerializerTest { +class EventSerializerTest { private final AbstractEvent[] events = { EndOfPartitionEvent.INSTANCE, @@ -119,50 +116,51 @@ public class EventSerializerTest { }; @Test - public void testSerializeDeserializeEvent() throws Exception { + void testSerializeDeserializeEvent() throws Exception { for (AbstractEvent evt : events) { ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(evt); - assertTrue(serializedEvent.hasRemaining()); + assertThat(serializedEvent.hasRemaining()).isTrue(); AbstractEvent deserialized = EventSerializer.fromSerializedEvent( serializedEvent, getClass().getClassLoader()); - assertNotNull(deserialized); - assertEquals(evt, deserialized); + assertThat(deserialized).isNotNull(); + assertThat(deserialized).isEqualTo(evt); } } @Test - public void testToBufferConsumer() throws IOException { + void testToBufferConsumer() throws IOException { for (AbstractEvent evt : events) { BufferConsumer bufferConsumer = EventSerializer.toBufferConsumer(evt, false); - assertFalse(bufferConsumer.isBuffer()); - assertTrue(bufferConsumer.isFinished()); - assertTrue(bufferConsumer.isDataAvailable()); - assertFalse(bufferConsumer.isRecycled()); + assertThat(bufferConsumer.isBuffer()).isFalse(); + assertThat(bufferConsumer.isFinished()).isTrue(); + assertThat(bufferConsumer.isDataAvailable()).isTrue(); + assertThat(bufferConsumer.isRecycled()).isFalse(); if (evt instanceof CheckpointBarrier) { - assertTrue(bufferConsumer.build().getDataType().isBlockingUpstream()); + assertThat(bufferConsumer.build().getDataType().isBlockingUpstream()).isTrue(); } else { - assertEquals(Buffer.DataType.EVENT_BUFFER, bufferConsumer.build().getDataType()); + assertThat(bufferConsumer.build().getDataType()) + .isEqualTo(Buffer.DataType.EVENT_BUFFER); } } } @Test - public void testToBuffer() throws IOException { + void testToBuffer() throws IOException { for (AbstractEvent evt : events) { Buffer buffer = EventSerializer.toBuffer(evt, false); - assertFalse(buffer.isBuffer()); - assertTrue(buffer.readableBytes() > 0); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isBuffer()).isFalse(); + assertThat(buffer.readableBytes()).isGreaterThan(0); + assertThat(buffer.isRecycled()).isFalse(); if (evt instanceof CheckpointBarrier) { - assertTrue(buffer.getDataType().isBlockingUpstream()); + assertThat(buffer.getDataType().isBlockingUpstream()).isTrue(); } else { - assertEquals(Buffer.DataType.EVENT_BUFFER, buffer.getDataType()); + assertThat(buffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java index 6fc42feb05ba0..d44a0bfc61ff9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java @@ -26,7 +26,7 @@ import org.apache.flink.testutils.serialization.types.SerializationTestTypeFactory; import org.apache.flink.testutils.serialization.types.Util; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.EOFException; import java.io.IOException; @@ -34,16 +34,14 @@ import java.util.List; import java.util.Random; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** Tests for the {@link AbstractPagedInputView} and {@link AbstractPagedOutputView}. */ -public class PagedViewsTest { +class PagedViewsTest { @Test - public void testSequenceOfIntegersWithAlignedBuffers() { + void testSequenceOfIntegersWithAlignedBuffers() { try { final int numInts = 1000000; @@ -57,7 +55,7 @@ public void testSequenceOfIntegersWithAlignedBuffers() { } @Test - public void testSequenceOfIntegersWithUnalignedBuffers() { + void testSequenceOfIntegersWithUnalignedBuffers() { try { final int numInts = 1000000; @@ -71,7 +69,7 @@ public void testSequenceOfIntegersWithUnalignedBuffers() { } @Test - public void testRandomTypes() { + void testRandomTypes() { try { final int numTypes = 100000; @@ -85,7 +83,7 @@ public void testRandomTypes() { } @Test - public void testReadFully() { + void testReadFully() { int bufferSize = 100; byte[] expected = new byte[bufferSize]; new Random().nextBytes(expected); @@ -111,12 +109,12 @@ public void testReadFully() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(inputView.getCurrentPositionInSegment(), bufferSize); - assertArrayEquals(expected, buffer); + assertThat(inputView.getCurrentPositionInSegment()).isEqualTo(bufferSize); + assertThat(buffer).isEqualTo(expected); } @Test - public void testReadFullyAcrossSegments() { + void testReadFullyAcrossSegments() { int bufferSize = 100; int segmentSize = 30; byte[] expected = new byte[bufferSize]; @@ -143,12 +141,12 @@ public void testReadFullyAcrossSegments() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(inputView.getCurrentPositionInSegment(), bufferSize % segmentSize); - assertArrayEquals(expected, buffer); + assertThat(inputView.getCurrentPositionInSegment()).isEqualTo(bufferSize % segmentSize); + assertThat(buffer).isEqualTo(expected); } @Test - public void testReadAcrossSegments() { + void testReadAcrossSegments() { int bufferSize = 100; int bytes2Write = 75; int segmentSize = 30; @@ -177,16 +175,16 @@ public void testReadAcrossSegments() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(bytes2Write, bytesRead); - assertEquals(inputView.getCurrentPositionInSegment(), bytes2Write % segmentSize); + assertThat(bytesRead).isEqualTo(bytes2Write); + assertThat(inputView.getCurrentPositionInSegment()).isEqualTo(bytes2Write % segmentSize); byte[] tempBuffer = new byte[bytesRead]; System.arraycopy(buffer, 0, tempBuffer, 0, bytesRead); - assertArrayEquals(expected, tempBuffer); + assertThat(tempBuffer).isEqualTo(expected); } @Test - public void testEmptyingInputView() { + void testEmptyingInputView() { int bufferSize = 100; int bytes2Write = 75; int segmentSize = 30; @@ -215,11 +213,11 @@ public void testEmptyingInputView() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(bytes2Write, bytesRead); + assertThat(bytesRead).isEqualTo(bytes2Write); byte[] tempBuffer = new byte[bytesRead]; System.arraycopy(buffer, 0, tempBuffer, 0, bytesRead); - assertArrayEquals(expected, tempBuffer); + assertThat(tempBuffer).isEqualTo(expected); try { bytesRead = inputView.read(buffer); @@ -228,12 +226,12 @@ public void testEmptyingInputView() { fail("Unexpected exception: Input view should be empty and thus return -1."); } - assertEquals(-1, bytesRead); - assertEquals(inputView.getCurrentPositionInSegment(), bytes2Write % segmentSize); + assertThat(bytesRead).isEqualTo(-1); + assertThat(inputView.getCurrentPositionInSegment()).isEqualTo(bytes2Write % segmentSize); } @Test - public void testReadFullyWithNotEnoughData() { + void testReadFullyWithNotEnoughData() { int bufferSize = 100; int bytes2Write = 99; int segmentSize = 30; @@ -265,7 +263,7 @@ public void testReadFullyWithNotEnoughData() { fail("Unexpected exception: Could not read TestInputView."); } - assertTrue("EOFException should have occurred.", eofException); + assertThat(eofException).withFailMessage("EOFException should have occurred.").isTrue(); int bytesRead = 0; @@ -276,11 +274,11 @@ public void testReadFullyWithNotEnoughData() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(-1, bytesRead); + assertThat(bytesRead).isEqualTo(-1); } @Test - public void testReadFullyWithOffset() { + void testReadFullyWithOffset() { int bufferSize = 100; int segmentSize = 30; byte[] expected = new byte[bufferSize]; @@ -307,14 +305,14 @@ public void testReadFullyWithOffset() { fail("Unexpected exception: Could not read TestInputView."); } - assertEquals(inputView.getCurrentPositionInSegment(), bufferSize % segmentSize); + assertThat(inputView.getCurrentPositionInSegment()).isEqualTo(bufferSize % segmentSize); byte[] tempBuffer = new byte[bufferSize]; System.arraycopy(buffer, bufferSize, tempBuffer, 0, bufferSize); - assertArrayEquals(expected, tempBuffer); + assertThat(tempBuffer).isEqualTo(expected); } @Test - public void testReadFullyEmptyView() { + void testReadFullyEmptyView() { int segmentSize = 30; TestOutputView outputView = new TestOutputView(segmentSize); outputView.close(); @@ -333,7 +331,7 @@ public void testReadFullyEmptyView() { fail("Unexpected exception: Could not read TestInputView."); } - assertTrue("EOFException expected.", eofException); + assertThat(eofException).withFailMessage("EOFException expected.").isTrue(); } private static void testSequenceOfTypes( @@ -356,7 +354,7 @@ private static void testSequenceOfTypes( for (SerializationTestType reference : elements) { SerializationTestType result = reference.getClass().newInstance(); result.read(inView); - assertEquals(reference, result); + assertThat(result).isEqualTo(reference); } } @@ -367,7 +365,7 @@ private static final class SegmentWithPosition { private final MemorySegment segment; private final int position; - public SegmentWithPosition(MemorySegment segment, int position) { + SegmentWithPosition(MemorySegment segment, int position) { this.segment = segment; this.position = position; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java index 6924a881bf5a2..43d86bfdb4576 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java @@ -33,14 +33,12 @@ import org.apache.flink.testutils.serialization.types.SerializationTestTypeFactory; import org.apache.flink.testutils.serialization.types.Util; import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.TestLogger; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.ByteArrayOutputStream; +import java.io.File; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -53,15 +51,16 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferBuilder; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link SpillingAdaptiveSpanningRecordDeserializer}. */ -public class SpanningRecordSerializationTest extends TestLogger { +class SpanningRecordSerializationTest { private static final Random RANDOM = new Random(42); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir private File tempFolder; @Test - public void testIntRecordsSpanningMultipleSegments() throws Exception { + void testIntRecordsSpanningMultipleSegments() throws Exception { final int segmentSize = 1; final int numValues = 10; @@ -70,7 +69,7 @@ public void testIntRecordsSpanningMultipleSegments() throws Exception { } @Test - public void testIntRecordsWithAlignedBuffers() throws Exception { + void testIntRecordsWithAlignedBuffers() throws Exception { final int segmentSize = 64; final int numValues = 64; @@ -79,7 +78,7 @@ public void testIntRecordsWithAlignedBuffers() throws Exception { } @Test - public void testIntRecordsWithUnalignedBuffers() throws Exception { + void testIntRecordsWithUnalignedBuffers() throws Exception { final int segmentSize = 31; final int numValues = 248; @@ -88,7 +87,7 @@ public void testIntRecordsWithUnalignedBuffers() throws Exception { } @Test - public void testRandomRecords() throws Exception { + void testRandomRecords() throws Exception { final int segmentSize = 127; final int numValues = 10000; @@ -96,7 +95,7 @@ public void testRandomRecords() throws Exception { } @Test - public void testHandleMixedLargeRecords() throws Exception { + void testHandleMixedLargeRecords() throws Exception { final int numValues = 99; final int segmentSize = 32 * 1024; @@ -121,7 +120,7 @@ private void testSerializationRoundTrip( Iterable records, int segmentSize) throws Exception { RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[] {tempFolder.getRoot().getAbsolutePath()}); + new String[] {tempFolder.getAbsolutePath()}); testSerializationRoundTrip(records, segmentSize, deserializer); } @@ -173,7 +172,7 @@ private static void testSerializationRoundTrip( deserializer.setNextBuffer(serializationResult.buildBuffer()); } } - Assert.assertFalse(serializedRecord.hasRemaining()); + assertThat(serializedRecord.hasRemaining()).isFalse(); } // deserialize left over records @@ -185,20 +184,20 @@ private static void testSerializationRoundTrip( SerializationTestType actual = expected.getClass().newInstance(); RecordDeserializer.DeserializationResult result = deserializer.getNextRecord(actual); - Assert.assertTrue(result.isFullRecord()); - Assert.assertEquals(expected, actual); + assertThat(result.isFullRecord()).isTrue(); + assertThat(actual).isEqualTo(expected); numRecords--; } // assert that all records have been serialized and deserialized - Assert.assertEquals(0, numRecords); + assertThat(numRecords).isEqualTo(0); } @Test - public void testSmallRecordUnconsumedBuffer() throws Exception { + void testSmallRecordUnconsumedBuffer() throws Exception { RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[] {tempFolder.getRoot().getAbsolutePath()}); + new String[] {tempFolder.getAbsolutePath()}); testUnconsumedBuffer( deserializer, Util.randomRecord(SerializationTestTypeFactory.INT), 1024); @@ -209,29 +208,29 @@ public void testSmallRecordUnconsumedBuffer() throws Exception { * by byte. */ @Test - public void testSpanningRecordUnconsumedBuffer() throws Exception { + void testSpanningRecordUnconsumedBuffer() throws Exception { RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[] {tempFolder.getRoot().getAbsolutePath()}); + new String[] {tempFolder.getAbsolutePath()}); testUnconsumedBuffer(deserializer, Util.randomRecord(SerializationTestTypeFactory.INT), 1); } @Test - public void testLargeSpanningRecordUnconsumedBuffer() throws Exception { + void testLargeSpanningRecordUnconsumedBuffer() throws Exception { RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[] {tempFolder.getRoot().getAbsolutePath()}); + new String[] {tempFolder.getAbsolutePath()}); testUnconsumedBuffer( deserializer, Util.randomRecord(SerializationTestTypeFactory.BYTE_ARRAY), 1); } @Test - public void testLargeSpanningRecordUnconsumedBufferWithLeftOverBytes() throws Exception { + void testLargeSpanningRecordUnconsumedBufferWithLeftOverBytes() throws Exception { RecordDeserializer deserializer = new SpillingAdaptiveSpanningRecordDeserializer<>( - new String[] {tempFolder.getRoot().getAbsolutePath()}); + new String[] {tempFolder.getAbsolutePath()}); testUnconsumedBuffer( deserializer, @@ -248,7 +247,7 @@ public void testLargeSpanningRecordUnconsumedBufferWithLeftOverBytes() throws Ex new byte[] {42, 43, 44}); } - public void testUnconsumedBuffer( + void testUnconsumedBuffer( RecordDeserializer deserializer, SerializationTestType record, int segmentSize, @@ -308,12 +307,12 @@ private static Buffer appendLeftOverBytes(Buffer buffer, byte[] leftOverBytes) { private static void assertUnconsumedBuffer( ByteArrayOutputStream expected, CloseableIterator actual) throws Exception { if (!actual.hasNext()) { - Assert.assertEquals(expected.size(), 0); + assertThat(expected.size()).isEqualTo(0); } ByteBuffer expectedByteBuffer = ByteBuffer.wrap(expected.toByteArray()); ByteBuffer actualByteBuffer = actual.next().getNioBufferReadable(); - Assert.assertEquals(expectedByteBuffer, actualByteBuffer); + assertThat(actualByteBuffer).isEqualTo(expectedByteBuffer); actual.close(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningWrapperTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningWrapperTest.java index 46719d1c3bdf2..7436ed1adab49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningWrapperTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningWrapperTest.java @@ -19,30 +19,31 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.CloseableIterator; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.Random; import static org.apache.flink.core.memory.MemorySegmentFactory.wrap; import static org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.LENGTH_BYTES; -import static org.junit.Assert.assertArrayEquals; +import static org.assertj.core.api.Assertions.assertThat; /** {@link SpanningWrapper} test. */ -public class SpanningWrapperTest { +class SpanningWrapperTest { private static final Random random = new Random(); - @Rule public TemporaryFolder folder = new TemporaryFolder(); + @TempDir private Path folder; @Test - public void testLargeUnconsumedSegment() throws Exception { + void testLargeUnconsumedSegment() throws Exception { int recordLen = 100; int firstChunk = (int) (recordLen * .9); int spillingThreshold = (int) (firstChunk * .9); @@ -50,14 +51,14 @@ public void testLargeUnconsumedSegment() throws Exception { byte[] record1 = recordBytes(recordLen); byte[] record2 = recordBytes(recordLen * 2); - File canNotEecutableFile = folder.newFolder(); + File canNotEecutableFile = TempDirUtils.newFolder(folder); canNotEecutableFile.setExecutable(false); // Always pick 'canNotEecutableFile' first as the Spilling Channel TmpDir. Thus trigger an // IOException. SpanningWrapper spanningWrapper = new SpanningWrapper( new String[] { - folder.newFolder().getAbsolutePath(), + TempDirUtils.newFolder(folder).getAbsolutePath(), canNotEecutableFile.getAbsolutePath() + File.separator + "pathdonotexit" }, spillingThreshold, @@ -79,7 +80,7 @@ public void testLargeUnconsumedSegment() throws Exception { canNotEecutableFile.setExecutable(true); - assertArrayEquals(concat(record1, record2), toByteArray(unconsumedSegment)); + assertThat(concat(record1, record2)).isEqualTo(toByteArray(unconsumedSegment)); } private byte[] recordBytes(int recordLen) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java index 4d6d1260d1d09..e24355217fdd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java @@ -28,23 +28,20 @@ import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.types.IntValue; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link SingleRecordWriter} and {@link MultipleRecordWriters}. */ -public class RecordWriterDelegateTest extends TestLogger { +class RecordWriterDelegateTest { private static final int recordSize = 8; @@ -54,31 +51,31 @@ public class RecordWriterDelegateTest extends TestLogger { private NetworkBufferPool globalPool; - @Before - public void setup() { - assertEquals("Illegal memory segment size,", 0, memorySegmentSize % recordSize); + @BeforeEach + void setup() { + assertThat(memorySegmentSize % recordSize).as("Illegal memory segment size").isEqualTo(0); globalPool = new NetworkBufferPool(numberOfBuffers, memorySegmentSize); } - @After - public void teardown() { + @AfterEach + void teardown() { globalPool.destroyAllBufferPools(); globalPool.destroy(); } @Test @SuppressWarnings("unchecked") - public void testSingleRecordWriterAvailability() throws Exception { + void testSingleRecordWriterAvailability() throws Exception { final RecordWriter recordWriter = createRecordWriter(globalPool); final RecordWriterDelegate writerDelegate = new SingleRecordWriter(recordWriter); - assertEquals(recordWriter, writerDelegate.getRecordWriter(0)); + assertThat(writerDelegate.getRecordWriter(0)).isEqualTo(recordWriter); verifyAvailability(writerDelegate); } @Test @SuppressWarnings("unchecked") - public void testMultipleRecordWritersAvailability() throws Exception { + void testMultipleRecordWritersAvailability() throws Exception { // setup final int numRecordWriters = 2; final List recordWriters = new ArrayList<>(numRecordWriters); @@ -89,7 +86,7 @@ public void testMultipleRecordWritersAvailability() throws Exception { RecordWriterDelegate writerDelegate = new MultipleRecordWriters(recordWriters); for (int i = 0; i < numRecordWriters; i++) { - assertEquals(recordWriters.get(i), writerDelegate.getRecordWriter(i)); + assertThat(writerDelegate.getRecordWriter(i)).isEqualTo(recordWriters.get(i)); } verifyAvailability(writerDelegate); @@ -97,7 +94,7 @@ public void testMultipleRecordWritersAvailability() throws Exception { @Test @SuppressWarnings("unchecked") - public void testSingleRecordWriterBroadcastEvent() throws Exception { + void testSingleRecordWriterBroadcastEvent() throws Exception { // setup final ResultPartition partition = RecordWriterTest.createResultPartition(memorySegmentSize, 2); @@ -109,7 +106,7 @@ public void testSingleRecordWriterBroadcastEvent() throws Exception { @Test @SuppressWarnings("unchecked") - public void testMultipleRecordWritersBroadcastEvent() throws Exception { + void testMultipleRecordWritersBroadcastEvent() throws Exception { // setup final int numRecordWriters = 2; final List recordWriters = new ArrayList<>(numRecordWriters); @@ -137,17 +134,17 @@ private RecordWriter createRecordWriter(NetworkBufferPool globalPool) throws Exc private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exception { // writer is available at the beginning - assertTrue(writerDelegate.isAvailable()); - assertTrue(writerDelegate.getAvailableFuture().isDone()); + assertThat(writerDelegate.isAvailable()).isTrue(); + assertThat(writerDelegate.getAvailableFuture().isDone()).isTrue(); // request one buffer from the local pool to make it unavailable RecordWriter recordWriter = writerDelegate.getRecordWriter(0); for (int i = 0; i < memorySegmentSize / recordSize; ++i) { recordWriter.emit(new IntValue(i)); } - assertFalse(writerDelegate.isAvailable()); + assertThat(writerDelegate.isAvailable()).isFalse(); CompletableFuture future = writerDelegate.getAvailableFuture(); - assertFalse(future.isDone()); + assertThat(future.isDone()).isFalse(); // recycle the buffer to make the local pool available again ResultSubpartitionView readView = @@ -157,9 +154,9 @@ private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exce Buffer buffer = readView.getNextBuffer().buffer(); buffer.recycleBuffer(); - assertTrue(future.isDone()); - assertTrue(writerDelegate.isAvailable()); - assertTrue(writerDelegate.getAvailableFuture().isDone()); + assertThat(future.isDone()).isTrue(); + assertThat(writerDelegate.isAvailable()).isTrue(); + assertThat(writerDelegate.getAvailableFuture().isDone()).isTrue(); } private void verifyBroadcastEvent( @@ -172,13 +169,13 @@ private void verifyBroadcastEvent( // verify the added messages in all the queues for (ResultPartition partition : partitions) { for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) { - assertEquals(1, partition.getNumberOfQueuedBuffers(i)); + assertThat(partition.getNumberOfQueuedBuffers(i)).isEqualTo(1); ResultSubpartitionView view = partition.createSubpartitionView(i, new NoOpBufferAvailablityListener()); BufferOrEvent boe = RecordWriterTest.parseBuffer(view.getNextBuffer().buffer(), i); - assertTrue(boe.isEvent()); - assertEquals(message, boe.getEvent()); + assertThat(boe.isEvent()).isTrue(); + assertThat(boe.getEvent()).isEqualTo(message); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapperTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapperTest.java index a1ae90a4164b7..5fb3ae8d3aae8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapperTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapperTest.java @@ -17,146 +17,125 @@ package org.apache.flink.runtime.io.network.api.writer; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ErrorCollector; +import org.junit.jupiter.api.Test; import static org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptorUtil.mappings; import static org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptorUtil.to; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** Tests to(@link SubtaskStateMapper). */ -public class SubtaskStateMapperTest { - @Rule public ErrorCollector collector = new ErrorCollector(); +class SubtaskStateMapperTest { @Test - public void testFirstTaskMappingOnScaleDown() { - assertEquals( - mappings(to(0, 1, 2), to()), - SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 2)); + void testFirstTaskMappingOnScaleDown() { + assertThat(SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 2)) + .isEqualTo(mappings(to(0, 1, 2), to())); } @Test - public void testFirstTaskMappingOnNoScale() { + void testFirstTaskMappingOnNoScale() { // this may be a bit surprising, but the optimization should be done on call-site - assertEquals( - mappings(to(0, 1, 2), to(), to()), - SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 3)); + assertThat(SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 3)) + .isEqualTo(mappings(to(0, 1, 2), to(), to())); } @Test - public void testFirstTaskMappingOnScaleUp() { - assertEquals( - mappings(to(0, 1, 2), to(), to(), to()), - SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 4)); + void testFirstTaskMappingOnScaleUp() { + assertThat(SubtaskStateMapper.FIRST.getNewToOldSubtasksMapping(3, 4)) + .isEqualTo(mappings(to(0, 1, 2), to(), to(), to())); } @Test - public void testFullTaskMappingOnScaleDown() { - assertEquals( - mappings(to(0, 1, 2), to(0, 1, 2)), - SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 2)); + void testFullTaskMappingOnScaleDown() { + assertThat(SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 2)) + .isEqualTo(mappings(to(0, 1, 2), to(0, 1, 2))); } @Test - public void testFullTaskMappingOnNoScale() { + void testFullTaskMappingOnNoScale() { // this may be a bit surprising, but the optimization should be done on call-site - assertEquals( - mappings(to(0, 1, 2), to(0, 1, 2), to(0, 1, 2)), - SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 3)); + assertThat(SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 3)) + .isEqualTo(mappings(to(0, 1, 2), to(0, 1, 2), to(0, 1, 2))); } @Test - public void testFullTaskMappingOnScaleUp() { - assertEquals( - mappings(to(0, 1, 2), to(0, 1, 2), to(0, 1, 2), to(0, 1, 2)), - SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 4)); + void testFullTaskMappingOnScaleUp() { + assertThat(SubtaskStateMapper.FULL.getNewToOldSubtasksMapping(3, 4)) + .isEqualTo(mappings(to(0, 1, 2), to(0, 1, 2), to(0, 1, 2), to(0, 1, 2))); } @Test - public void testRangeSelectorTaskMappingOnScaleDown() { + void testRangeSelectorTaskMappingOnScaleDown() { // 3 partitions: [0; 43) [43; 87) [87; 128) // 2 partitions: [0; 64) [64; 128) - assertEquals( - mappings(to(0, 1), to(1, 2)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 2)); + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 2)) + .isEqualTo(mappings(to(0, 1), to(1, 2))); - assertEquals( - mappings(to(0, 1, 2, 3, 4), to(5, 6, 7, 8, 9)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(10, 2)); + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(10, 2)) + .isEqualTo(mappings(to(0, 1, 2, 3, 4), to(5, 6, 7, 8, 9))); - assertEquals( - mappings(to(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(10, 1)); + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(10, 1)) + .isEqualTo(mappings(to(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); } @Test - public void testRangeSelectorTaskMappingOnNoScale() { - assertEquals( - mappings(to(0), to(1), to(2)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 3)); + void testRangeSelectorTaskMappingOnNoScale() { + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 3)) + .isEqualTo(mappings(to(0), to(1), to(2))); } @Test - public void testRangeSelectorTaskMappingOnScaleUp() { - assertEquals( - mappings(to(0), to(0, 1), to(1, 2), to(2)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 4)); - - assertEquals( - mappings(to(0), to(0), to(0, 1), to(1), to(1, 2), to(2), to(2)), - SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 7)); - } + void testRangeSelectorTaskMappingOnScaleUp() { + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 4)) + .isEqualTo(mappings(to(0), to(0, 1), to(1, 2), to(2))); - @Test - public void testRoundRobinTaskMappingOnScaleDown() { - assertEquals( - mappings(to(0, 4, 8), to(1, 5, 9), to(2, 6), to(3, 7)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(10, 4)); - - assertEquals( - mappings(to(0, 4), to(1), to(2), to(3)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 4)); - - assertEquals( - mappings(to(0, 2, 4), to(1, 3)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 2)); - - assertEquals( - mappings(to(0, 1, 2, 3, 4)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 1)); + assertThat(SubtaskStateMapper.RANGE.getNewToOldSubtasksMapping(3, 7)) + .isEqualTo(mappings(to(0), to(0), to(0, 1), to(1), to(1, 2), to(2), to(2))); } @Test - public void testRoundRobinTaskMappingOnNoScale() { - assertEquals( - mappings(to(0), to(1), to(2), to(3), to(4), to(5), to(6), to(7), to(8), to(9)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(10, 10)); + void testRoundRobinTaskMappingOnScaleDown() { + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(10, 4)) + .isEqualTo(mappings(to(0, 4, 8), to(1, 5, 9), to(2, 6), to(3, 7))); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 4)) + .isEqualTo(mappings(to(0, 4), to(1), to(2), to(3))); - assertEquals( - mappings(to(0), to(1), to(2), to(3), to(4)), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 5)); + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 2)) + .isEqualTo(mappings(to(0, 2, 4), to(1, 3))); - assertEquals( - mappings(to(0)), SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(1, 1)); + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 1)) + .isEqualTo(mappings(to(0, 1, 2, 3, 4))); } @Test - public void testRoundRobinTaskMappingOnScaleUp() { - assertEquals( - mappings(to(0), to(1), to(2), to(3), to(), to(), to(), to(), to(), to()), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(4, 10)); - - assertEquals( - mappings(to(0), to(1), to(2), to(3), to()), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(4, 5)); - - assertEquals( - mappings(to(0), to(1), to(), to(), to()), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(2, 5)); - - assertEquals( - mappings(to(0), to(), to(), to(), to()), - SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(1, 5)); + void testRoundRobinTaskMappingOnNoScale() { + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(10, 10)) + .isEqualTo( + mappings( + to(0), to(1), to(2), to(3), to(4), to(5), to(6), to(7), to(8), + to(9))); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(5, 5)) + .isEqualTo(mappings(to(0), to(1), to(2), to(3), to(4))); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(1, 1)) + .isEqualTo(mappings(to(0))); + } + + @Test + void testRoundRobinTaskMappingOnScaleUp() { + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(4, 10)) + .isEqualTo( + mappings(to(0), to(1), to(2), to(3), to(), to(), to(), to(), to(), to())); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(4, 5)) + .isEqualTo(mappings(to(0), to(1), to(2), to(3), to())); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(2, 5)) + .isEqualTo(mappings(to(0), to(1), to(), to(), to())); + + assertThat(SubtaskStateMapper.ROUND_ROBIN.getNewToOldSubtasksMapping(1, 5)) + .isEqualTo(mappings(to(0), to(), to(), to(), to())); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index 6bf4d96140f12..2a2599a35df47 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -16,8 +16,6 @@ */ package org.apache.flink.runtime.io.network.buffer; -import org.apache.flink.util.TestLogger; - import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufUtil; import org.apache.flink.shaded.netty4.io.netty.util.ByteProcessor; @@ -25,9 +23,10 @@ import org.apache.flink.shaded.netty4.io.netty.util.IllegalReferenceCountException; import org.apache.flink.shaded.netty4.io.netty.util.internal.PlatformDependent; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -63,24 +62,18 @@ import static org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled.unreleasableBuffer; import static org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled.wrappedBuffer; import static org.apache.flink.shaded.netty4.io.netty.util.internal.EmptyArrays.EMPTY_BYTES; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.junit.Assume.assumeFalse; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.within; +import static org.assertj.core.api.Assumptions.assumeThat; /** * An abstract test class for channel buffers. * *

Copy from netty 4.1.32.Final. */ -public abstract class AbstractByteBufTest extends TestLogger { +abstract class AbstractByteBufTest { private static final int CAPACITY = 4096; // Must be even private static final int BLOCK_SIZE = 128; @@ -100,18 +93,18 @@ protected boolean discardReadBytesDoesNotMoveWritableBytes() { return true; } - @Before - public void init() { + @BeforeEach + void init() { buffer = newBuffer(CAPACITY); seed = System.currentTimeMillis(); random = new Random(seed); } - @After - public void dispose() { + @AfterEach + void dispose() { if (buffer != null) { - assertThat(buffer.release(), is(true)); - assertThat(buffer.refCnt(), is(0)); + assertThat(buffer.release()).isTrue(); + assertThat(buffer.refCnt()).isEqualTo(0); try { buffer.release(); @@ -123,97 +116,83 @@ public void dispose() { } @Test - public void comparableInterfaceNotViolated() { - assumeFalse(buffer.isReadOnly()); + void comparableInterfaceNotViolated() { + assumeThat(buffer.isReadOnly()).isFalse(); buffer.writerIndex(buffer.readerIndex()); - assumeTrue(buffer.writableBytes() >= 4); + assumeThat(buffer.writableBytes() >= 4).isTrue(); buffer.writeLong(0); ByteBuf buffer2 = newBuffer(CAPACITY); - assumeFalse(buffer2.isReadOnly()); + assumeThat(buffer2.isReadOnly()).isFalse(); buffer2.writerIndex(buffer2.readerIndex()); // Write an unsigned integer that will cause buffer.getUnsignedInt() - // buffer2.getUnsignedInt() to underflow the // int type and wrap around on the negative side. buffer2.writeLong(0xF0000000L); - assertTrue(buffer.compareTo(buffer2) < 0); - assertTrue(buffer2.compareTo(buffer) > 0); + assumeThat(buffer.compareTo(buffer2) < 0).isTrue(); + assumeThat(buffer2.compareTo(buffer) > 0).isTrue(); buffer2.release(); } @Test - public void initialState() { - assertEquals(CAPACITY, buffer.capacity()); - assertEquals(0, buffer.readerIndex()); + void initialState() { + assertThat(buffer.capacity()).isEqualTo(CAPACITY); + assertThat(buffer.readerIndex()).isEqualTo(0); } - @Test(expected = IndexOutOfBoundsException.class) - public void readerIndexBoundaryCheck1() { - try { - buffer.writerIndex(0); - } catch (IndexOutOfBoundsException e) { - fail(); - } - buffer.readerIndex(-1); + @Test + void readerIndexBoundaryCheck1() { + buffer.writerIndex(0); + assertThatThrownBy(() -> buffer.readerIndex(-1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void readerIndexBoundaryCheck2() { - try { - buffer.writerIndex(buffer.capacity()); - } catch (IndexOutOfBoundsException e) { - fail(); - } - buffer.readerIndex(buffer.capacity() + 1); + @Test + void readerIndexBoundaryCheck2() { + buffer.writerIndex(buffer.capacity()); + assertThatThrownBy(() -> buffer.readerIndex(buffer.capacity() + 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void readerIndexBoundaryCheck3() { - try { - buffer.writerIndex(CAPACITY / 2); - } catch (IndexOutOfBoundsException e) { - fail(); - } - buffer.readerIndex(CAPACITY * 3 / 2); + @Test + void readerIndexBoundaryCheck3() { + buffer.writerIndex(CAPACITY / 2); + assertThatThrownBy(() -> buffer.readerIndex(CAPACITY * 3 / 2)) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void readerIndexBoundaryCheck4() { + void readerIndexBoundaryCheck4() { buffer.writerIndex(0); buffer.readerIndex(0); buffer.writerIndex(buffer.capacity()); buffer.readerIndex(buffer.capacity()); } - @Test(expected = IndexOutOfBoundsException.class) - public void writerIndexBoundaryCheck1() { - buffer.writerIndex(-1); + @Test + void writerIndexBoundaryCheck1() { + assertThatThrownBy(() -> buffer.writerIndex(-1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void writerIndexBoundaryCheck2() { - try { - buffer.writerIndex(CAPACITY); - buffer.readerIndex(CAPACITY); - } catch (IndexOutOfBoundsException e) { - fail(); - } - buffer.writerIndex(buffer.capacity() + 1); + @Test + void writerIndexBoundaryCheck2() { + buffer.writerIndex(CAPACITY); + buffer.readerIndex(CAPACITY); + assertThatThrownBy(() -> buffer.writerIndex(buffer.capacity() + 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void writerIndexBoundaryCheck3() { - try { - buffer.writerIndex(CAPACITY); - buffer.readerIndex(CAPACITY / 2); - } catch (IndexOutOfBoundsException e) { - fail(); - } - buffer.writerIndex(CAPACITY / 4); + @Test + void writerIndexBoundaryCheck3() { + buffer.writerIndex(CAPACITY); + buffer.readerIndex(CAPACITY / 2); + assertThatThrownBy(() -> buffer.writerIndex(CAPACITY / 4)) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void writerIndexBoundaryCheck4() { + void writerIndexBoundaryCheck4() { buffer.writerIndex(0); buffer.readerIndex(0); buffer.writerIndex(CAPACITY); @@ -221,154 +200,165 @@ public void writerIndexBoundaryCheck4() { buffer.writeBytes(ByteBuffer.wrap(EMPTY_BYTES)); } - @Test(expected = IndexOutOfBoundsException.class) - public void getBooleanBoundaryCheck1() { - buffer.getBoolean(-1); + @Test + void getBooleanBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getBoolean(-1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getBooleanBoundaryCheck2() { - buffer.getBoolean(buffer.capacity()); + @Test + void getBooleanBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getBoolean(buffer.capacity())) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getByteBoundaryCheck1() { - buffer.getByte(-1); + @Test + void getByteBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getByte(-1)).isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getByteBoundaryCheck2() { - buffer.getByte(buffer.capacity()); + @Test + void getByteBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getByte(buffer.capacity())) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getShortBoundaryCheck1() { - buffer.getShort(-1); + @Test + void getShortBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getShort(-1)).isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getShortBoundaryCheck2() { - buffer.getShort(buffer.capacity() - 1); + @Test + void getShortBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getShort(buffer.capacity() - 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getMediumBoundaryCheck1() { - buffer.getMedium(-1); + @Test + void getMediumBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getMedium(-1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getMediumBoundaryCheck2() { - buffer.getMedium(buffer.capacity() - 2); + @Test + void getMediumBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getMedium(buffer.capacity() - 2)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getIntBoundaryCheck1() { - buffer.getInt(-1); + @Test + void getIntBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getInt(-1)).isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getIntBoundaryCheck2() { - buffer.getInt(buffer.capacity() - 3); + @Test + void getIntBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getInt(buffer.capacity() - 3)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getLongBoundaryCheck1() { - buffer.getLong(-1); + @Test + void getLongBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getLong(-1)).isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getLongBoundaryCheck2() { - buffer.getLong(buffer.capacity() - 7); + @Test + void getLongBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getLong(buffer.capacity() - 7)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getByteArrayBoundaryCheck1() { - buffer.getBytes(-1, EMPTY_BYTES); + @Test + void getByteArrayBoundaryCheck1() { + assertThatThrownBy(() -> buffer.getBytes(-1, EMPTY_BYTES)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void getByteArrayBoundaryCheck2() { - buffer.getBytes(-1, EMPTY_BYTES, 0, 0); + @Test + void getByteArrayBoundaryCheck2() { + assertThatThrownBy(() -> buffer.getBytes(-1, EMPTY_BYTES, 0, 0)) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void getByteArrayBoundaryCheck3() { + void getByteArrayBoundaryCheck3() { byte[] dst = new byte[4]; buffer.setInt(0, 0x01020304); - try { - buffer.getBytes(0, dst, -1, 4); - fail(); - } catch (IndexOutOfBoundsException e) { - // Success - } + + assertThatThrownBy(() -> buffer.getBytes(0, dst, -1, 4)) + .isInstanceOf(IndexOutOfBoundsException.class); // No partial copy is expected. - assertEquals(0, dst[0]); - assertEquals(0, dst[1]); - assertEquals(0, dst[2]); - assertEquals(0, dst[3]); + assertThat(dst[0]).isEqualTo((byte) 0); + assertThat(dst[1]).isEqualTo((byte) 0); + assertThat(dst[2]).isEqualTo((byte) 0); + assertThat(dst[3]).isEqualTo((byte) 0); } @Test - public void getByteArrayBoundaryCheck4() { + void getByteArrayBoundaryCheck4() { byte[] dst = new byte[4]; buffer.setInt(0, 0x01020304); - try { - buffer.getBytes(0, dst, 1, 4); - fail(); - } catch (IndexOutOfBoundsException e) { - // Success - } + + assertThatThrownBy(() -> buffer.getBytes(0, dst, 1, 4)) + .isInstanceOf(IndexOutOfBoundsException.class); // No partial copy is expected. - assertEquals(0, dst[0]); - assertEquals(0, dst[1]); - assertEquals(0, dst[2]); - assertEquals(0, dst[3]); + assertThat(dst[0]).isEqualTo((byte) 0); + assertThat(dst[1]).isEqualTo((byte) 0); + assertThat(dst[2]).isEqualTo((byte) 0); + assertThat(dst[3]).isEqualTo((byte) 0); } - @Test(expected = IndexOutOfBoundsException.class) - public void getByteBufferBoundaryCheck() { - buffer.getBytes(-1, ByteBuffer.allocate(0)); + @Test + void getByteBufferBoundaryCheck() { + assertThatThrownBy(() -> buffer.getBytes(-1, ByteBuffer.allocate(0))) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void copyBoundaryCheck1() { - buffer.copy(-1, 0); + @Test + void copyBoundaryCheck1() { + assertThatThrownBy(() -> buffer.copy(-1, 0)).isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void copyBoundaryCheck2() { - buffer.copy(0, buffer.capacity() + 1); + @Test + void copyBoundaryCheck2() { + assertThatThrownBy(() -> buffer.copy(0, buffer.capacity() + 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void copyBoundaryCheck3() { - buffer.copy(buffer.capacity() + 1, 0); + @Test + void copyBoundaryCheck3() { + assertThatThrownBy(() -> buffer.copy(buffer.capacity() + 1, 0)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void copyBoundaryCheck4() { - buffer.copy(buffer.capacity(), 1); + @Test + void copyBoundaryCheck4() { + assertThatThrownBy(() -> buffer.copy(buffer.capacity(), 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void setIndexBoundaryCheck1() { - buffer.setIndex(-1, CAPACITY); + @Test + void setIndexBoundaryCheck1() { + assertThatThrownBy(() -> buffer.setIndex(-1, CAPACITY)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void setIndexBoundaryCheck2() { - buffer.setIndex(CAPACITY / 2, CAPACITY / 4); + @Test + void setIndexBoundaryCheck2() { + assertThatThrownBy(() -> buffer.setIndex(CAPACITY / 2, CAPACITY / 4)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void setIndexBoundaryCheck3() { - buffer.setIndex(0, CAPACITY + 1); + @Test + void setIndexBoundaryCheck3() { + assertThatThrownBy(() -> buffer.setIndex(0, CAPACITY + 1)) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void getByteBufferState() { + void getByteBufferState() { ByteBuffer dst = ByteBuffer.allocate(4); dst.position(1); dst.limit(3); @@ -379,23 +369,24 @@ public void getByteBufferState() { buffer.setByte(3, (byte) 4); buffer.getBytes(1, dst); - assertEquals(3, dst.position()); - assertEquals(3, dst.limit()); + assertThat(dst.position()).isEqualTo(3); + assertThat(dst.limit()).isEqualTo(3); dst.clear(); - assertEquals(0, dst.get(0)); - assertEquals(2, dst.get(1)); - assertEquals(3, dst.get(2)); - assertEquals(0, dst.get(3)); + assertThat(dst.get(0)).isEqualTo((byte) 0); + assertThat(dst.get(1)).isEqualTo((byte) 2); + assertThat(dst.get(2)).isEqualTo((byte) 3); + assertThat(dst.get(3)).isEqualTo((byte) 0); } - @Test(expected = IndexOutOfBoundsException.class) - public void getDirectByteBufferBoundaryCheck() { - buffer.getBytes(-1, ByteBuffer.allocateDirect(0)); + @Test + void getDirectByteBufferBoundaryCheck() { + assertThatThrownBy(() -> buffer.getBytes(-1, ByteBuffer.allocateDirect(0))) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void getDirectByteBufferState() { + void getDirectByteBufferState() { ByteBuffer dst = ByteBuffer.allocateDirect(4); dst.position(1); dst.limit(3); @@ -406,18 +397,18 @@ public void getDirectByteBufferState() { buffer.setByte(3, (byte) 4); buffer.getBytes(1, dst); - assertEquals(3, dst.position()); - assertEquals(3, dst.limit()); + assertThat(dst.position()).isEqualTo(3); + assertThat(dst.limit()).isEqualTo(3); dst.clear(); - assertEquals(0, dst.get(0)); - assertEquals(2, dst.get(1)); - assertEquals(3, dst.get(2)); - assertEquals(0, dst.get(3)); + assertThat(dst.get(0)).isEqualTo((byte) 0); + assertThat(dst.get(1)).isEqualTo((byte) 2); + assertThat(dst.get(2)).isEqualTo((byte) 3); + assertThat(dst.get(3)).isEqualTo((byte) 0); } @Test - public void testRandomByteAccess() { + void testRandomByteAccess() { for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); buffer.setByte(i, value); @@ -426,12 +417,12 @@ public void testRandomByteAccess() { random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); - assertEquals(value, buffer.getByte(i)); + assertThat(buffer.getByte(i)).isEqualTo(value); } } @Test - public void testRandomUnsignedByteAccess() { + void testRandomUnsignedByteAccess() { for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); buffer.setByte(i, value); @@ -439,18 +430,18 @@ public void testRandomUnsignedByteAccess() { random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i++) { - int value = random.nextInt() & 0xFF; - assertEquals(value, buffer.getUnsignedByte(i)); + short value = (short) (random.nextInt() & 0xFF); + assertThat(buffer.getUnsignedByte(i)).isEqualTo(value); } } @Test - public void testRandomShortAccess() { + void testRandomShortAccess() { testRandomShortAccess(true); } @Test - public void testRandomShortLEAccess() { + void testRandomShortLEAccess() { testRandomShortAccess(false); } @@ -468,15 +459,15 @@ private void testRandomShortAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 1; i += 2) { short value = (short) random.nextInt(); if (testBigEndian) { - assertEquals(value, buffer.getShort(i)); + assertThat(buffer.getShort(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getShortLE(i)); + assertThat(buffer.getShortLE(i)).isEqualTo(value); } } } @Test - public void testShortConsistentWithByteBuffer() { + void testShortConsistentWithByteBuffer() { testShortConsistentWithByteBuffer(true, true); testShortConsistentWithByteBuffer(true, false); testShortConsistentWithByteBuffer(false, true); @@ -505,20 +496,22 @@ private void testShortConsistentWithByteBuffer(boolean direct, boolean testBigEn javaBuffer.flip(); short javaActual = javaBuffer.getShort(); - assertEquals(expected, javaActual); - assertEquals( - javaActual, - testBigEndian ? buffer.getShort(bufferIndex) : buffer.getShortLE(bufferIndex)); + assertThat(javaActual).isEqualTo(expected); + assertThat(javaActual) + .isEqualTo( + testBigEndian + ? buffer.getShort(bufferIndex) + : buffer.getShortLE(bufferIndex)); } } @Test - public void testRandomUnsignedShortAccess() { + void testRandomUnsignedShortAccess() { testRandomUnsignedShortAccess(true); } @Test - public void testRandomUnsignedShortLEAccess() { + void testRandomUnsignedShortLEAccess() { testRandomUnsignedShortAccess(false); } @@ -536,20 +529,20 @@ private void testRandomUnsignedShortAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 1; i += 2) { int value = random.nextInt() & 0xFFFF; if (testBigEndian) { - assertEquals(value, buffer.getUnsignedShort(i)); + assertThat(buffer.getUnsignedShort(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getUnsignedShortLE(i)); + assertThat(buffer.getUnsignedShortLE(i)).isEqualTo(value); } } } @Test - public void testRandomMediumAccess() { + void testRandomMediumAccess() { testRandomMediumAccess(true); } @Test - public void testRandomMediumLEAccess() { + void testRandomMediumLEAccess() { testRandomMediumAccess(false); } @@ -567,20 +560,20 @@ private void testRandomMediumAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 2; i += 3) { int value = random.nextInt() << 8 >> 8; if (testBigEndian) { - assertEquals(value, buffer.getMedium(i)); + assertThat(buffer.getMedium(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getMediumLE(i)); + assertThat(buffer.getMediumLE(i)).isEqualTo(value); } } } @Test - public void testRandomUnsignedMediumAccess() { + void testRandomUnsignedMediumAccess() { testRandomUnsignedMediumAccess(true); } @Test - public void testRandomUnsignedMediumLEAccess() { + void testRandomUnsignedMediumLEAccess() { testRandomUnsignedMediumAccess(false); } @@ -598,15 +591,15 @@ private void testRandomUnsignedMediumAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 2; i += 3) { int value = random.nextInt() & 0x00FFFFFF; if (testBigEndian) { - assertEquals(value, buffer.getUnsignedMedium(i)); + assertThat(buffer.getUnsignedMedium(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getUnsignedMediumLE(i)); + assertThat(buffer.getUnsignedMediumLE(i)).isEqualTo(value); } } } @Test - public void testMediumConsistentWithByteBuffer() { + void testMediumConsistentWithByteBuffer() { testMediumConsistentWithByteBuffer(true, true); testMediumConsistentWithByteBuffer(true, false); testMediumConsistentWithByteBuffer(false, true); @@ -635,22 +628,22 @@ private void testMediumConsistentWithByteBuffer(boolean direct, boolean testBigE javaBuffer.flip(); int javaActual = javaBuffer.getInt(); - assertEquals(expected, javaActual); - assertEquals( - javaActual, - testBigEndian - ? buffer.getUnsignedMedium(bufferIndex) - : buffer.getUnsignedMediumLE(bufferIndex)); + assertThat(javaActual).isEqualTo(expected); + assertThat(javaActual) + .isEqualTo( + testBigEndian + ? buffer.getUnsignedMedium(bufferIndex) + : buffer.getUnsignedMediumLE(bufferIndex)); } } @Test - public void testRandomIntAccess() { + void testRandomIntAccess() { testRandomIntAccess(true); } @Test - public void testRandomIntLEAccess() { + void testRandomIntLEAccess() { testRandomIntAccess(false); } @@ -668,15 +661,15 @@ private void testRandomIntAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 3; i += 4) { int value = random.nextInt(); if (testBigEndian) { - assertEquals(value, buffer.getInt(i)); + assertThat(buffer.getInt(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getIntLE(i)); + assertThat(buffer.getIntLE(i)).isEqualTo(value); } } } @Test - public void testIntConsistentWithByteBuffer() { + void testIntConsistentWithByteBuffer() { testIntConsistentWithByteBuffer(true, true); testIntConsistentWithByteBuffer(true, false); testIntConsistentWithByteBuffer(false, true); @@ -705,20 +698,22 @@ private void testIntConsistentWithByteBuffer(boolean direct, boolean testBigEndi javaBuffer.flip(); int javaActual = javaBuffer.getInt(); - assertEquals(expected, javaActual); - assertEquals( - javaActual, - testBigEndian ? buffer.getInt(bufferIndex) : buffer.getIntLE(bufferIndex)); + assertThat(javaActual).isEqualTo(expected); + assertThat(javaActual) + .isEqualTo( + testBigEndian + ? buffer.getInt(bufferIndex) + : buffer.getIntLE(bufferIndex)); } } @Test - public void testRandomUnsignedIntAccess() { + void testRandomUnsignedIntAccess() { testRandomUnsignedIntAccess(true); } @Test - public void testRandomUnsignedIntLEAccess() { + void testRandomUnsignedIntLEAccess() { testRandomUnsignedIntAccess(false); } @@ -736,20 +731,20 @@ private void testRandomUnsignedIntAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 3; i += 4) { long value = random.nextInt() & 0xFFFFFFFFL; if (testBigEndian) { - assertEquals(value, buffer.getUnsignedInt(i)); + assertThat(buffer.getUnsignedInt(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getUnsignedIntLE(i)); + assertThat(buffer.getUnsignedIntLE(i)).isEqualTo(value); } } } @Test - public void testRandomLongAccess() { + void testRandomLongAccess() { testRandomLongAccess(true); } @Test - public void testRandomLongLEAccess() { + void testRandomLongLEAccess() { testRandomLongAccess(false); } @@ -767,15 +762,15 @@ private void testRandomLongAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 7; i += 8) { long value = random.nextLong(); if (testBigEndian) { - assertEquals(value, buffer.getLong(i)); + assertThat(buffer.getLong(i)).isEqualTo(value); } else { - assertEquals(value, buffer.getLongLE(i)); + assertThat(buffer.getLongLE(i)).isEqualTo(value); } } } @Test - public void testLongConsistentWithByteBuffer() { + void testLongConsistentWithByteBuffer() { testLongConsistentWithByteBuffer(true, true); testLongConsistentWithByteBuffer(true, false); testLongConsistentWithByteBuffer(false, true); @@ -804,20 +799,22 @@ private void testLongConsistentWithByteBuffer(boolean direct, boolean testBigEnd javaBuffer.flip(); long javaActual = javaBuffer.getLong(); - assertEquals(expected, javaActual); - assertEquals( - javaActual, - testBigEndian ? buffer.getLong(bufferIndex) : buffer.getLongLE(bufferIndex)); + assertThat(javaActual).isEqualTo(expected); + assertThat(javaActual) + .isEqualTo( + testBigEndian + ? buffer.getLong(bufferIndex) + : buffer.getLongLE(bufferIndex)); } } @Test - public void testRandomFloatAccess() { + void testRandomFloatAccess() { testRandomFloatAccess(true); } @Test - public void testRandomFloatLEAccess() { + void testRandomFloatLEAccess() { testRandomFloatAccess(false); } @@ -835,17 +832,17 @@ private void testRandomFloatAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 7; i += 8) { float expected = random.nextFloat(); float actual = testBigEndian ? buffer.getFloat(i) : buffer.getFloatLE(i); - assertEquals(expected, actual, 0.01); + assertThat(actual).isCloseTo(expected, within(0.01f)); } } @Test - public void testRandomDoubleAccess() { + void testRandomDoubleAccess() { testRandomDoubleAccess(true); } @Test - public void testRandomDoubleLEAccess() { + void testRandomDoubleLEAccess() { testRandomDoubleAccess(false); } @@ -863,12 +860,12 @@ private void testRandomDoubleAccess(boolean testBigEndian) { for (int i = 0; i < buffer.capacity() - 7; i += 8) { double expected = random.nextDouble(); double actual = testBigEndian ? buffer.getDouble(i) : buffer.getDoubleLE(i); - assertEquals(expected, actual, 0.01); + assertThat(actual).isCloseTo(expected, within(0.01)); } } @Test - public void testSetZero() { + void testSetZero() { buffer.clear(); while (buffer.isWritable()) { buffer.writeByte((byte) 0xFF); @@ -881,73 +878,73 @@ public void testSetZero() { } for (int i = 0; i < buffer.capacity(); i++) { - assertEquals(0, buffer.getByte(i)); + assertThat(buffer.getByte(i)).isEqualTo((byte) 0); } } @Test - public void testSequentialByteAccess() { + void testSequentialByteAccess() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); buffer.writeByte(value); } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); - assertEquals(value, buffer.readByte()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); + assertThat(buffer.readByte()).isEqualTo(value); } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialUnsignedByteAccess() { + void testSequentialUnsignedByteAccess() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); buffer.writeByte(value); } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i++) { int value = random.nextInt() & 0xFF; - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); - assertEquals(value, buffer.readUnsignedByte()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); + assertThat(buffer.readUnsignedByte()).isEqualTo((short) value); } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialShortAccess() { + void testSequentialShortAccess() { testSequentialShortAccess(true); } @Test - public void testSequentialShortLEAccess() { + void testSequentialShortLEAccess() { testSequentialShortAccess(false); } @@ -955,8 +952,8 @@ private void testSequentialShortAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 2) { short value = (short) random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeShort(value); } else { @@ -964,35 +961,35 @@ private void testSequentialShortAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i += 2) { short value = (short) random.nextInt(); - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readShort()); + assertThat(buffer.readShort()).isEqualTo(value); } else { - assertEquals(value, buffer.readShortLE()); + assertThat(buffer.readShortLE()).isEqualTo(value); } } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialUnsignedShortAccess() { + void testSequentialUnsignedShortAccess() { testSequentialUnsignedShortAccess(true); } @Test - public void testSequentialUnsignedShortLEAccess() { + void testSequentialUnsignedShortLEAccess() { testSequentialUnsignedShortAccess(true); } @@ -1000,8 +997,8 @@ private void testSequentialUnsignedShortAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 2) { short value = (short) random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeShort(value); } else { @@ -1009,35 +1006,35 @@ private void testSequentialUnsignedShortAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i += 2) { int value = random.nextInt() & 0xFFFF; - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readUnsignedShort()); + assertThat(buffer.readUnsignedShort()).isEqualTo(value); } else { - assertEquals(value, buffer.readUnsignedShortLE()); + assertThat(buffer.readUnsignedShortLE()).isEqualTo(value); } } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialMediumAccess() { + void testSequentialMediumAccess() { testSequentialMediumAccess(true); } @Test - public void testSequentialMediumLEAccess() { + void testSequentialMediumLEAccess() { testSequentialMediumAccess(false); } @@ -1045,8 +1042,8 @@ private void testSequentialMediumAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() / 3 * 3; i += 3) { int value = random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeMedium(value); } else { @@ -1054,35 +1051,35 @@ private void testSequentialMediumAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex()); - assertEquals(buffer.capacity() % 3, buffer.writableBytes()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); random.setSeed(seed); for (int i = 0; i < buffer.capacity() / 3 * 3; i += 3) { int value = random.nextInt() << 8 >> 8; - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readMedium()); + assertThat(buffer.readMedium()).isEqualTo(value); } else { - assertEquals(value, buffer.readMediumLE()); + assertThat(buffer.readMediumLE()).isEqualTo(value); } } - assertEquals(buffer.capacity() / 3 * 3, buffer.readerIndex()); - assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex()); - assertEquals(0, buffer.readableBytes()); - assertEquals(buffer.capacity() % 3, buffer.writableBytes()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); } @Test - public void testSequentialUnsignedMediumAccess() { + void testSequentialUnsignedMediumAccess() { testSequentialUnsignedMediumAccess(true); } @Test - public void testSequentialUnsignedMediumLEAccess() { + void testSequentialUnsignedMediumLEAccess() { testSequentialUnsignedMediumAccess(false); } @@ -1090,8 +1087,8 @@ private void testSequentialUnsignedMediumAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() / 3 * 3; i += 3) { int value = random.nextInt() & 0x00FFFFFF; - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeMedium(value); } else { @@ -1099,35 +1096,35 @@ private void testSequentialUnsignedMediumAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex()); - assertEquals(buffer.capacity() % 3, buffer.writableBytes()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); random.setSeed(seed); for (int i = 0; i < buffer.capacity() / 3 * 3; i += 3) { int value = random.nextInt() & 0x00FFFFFF; - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readUnsignedMedium()); + assertThat(buffer.readUnsignedMedium()).isEqualTo(value); } else { - assertEquals(value, buffer.readUnsignedMediumLE()); + assertThat(buffer.readUnsignedMediumLE()).isEqualTo(value); } } - assertEquals(buffer.capacity() / 3 * 3, buffer.readerIndex()); - assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex()); - assertEquals(0, buffer.readableBytes()); - assertEquals(buffer.capacity() % 3, buffer.writableBytes()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); + assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); } @Test - public void testSequentialIntAccess() { + void testSequentialIntAccess() { testSequentialIntAccess(true); } @Test - public void testSequentialIntLEAccess() { + void testSequentialIntLEAccess() { testSequentialIntAccess(false); } @@ -1135,8 +1132,8 @@ private void testSequentialIntAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 4) { int value = random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeInt(value); } else { @@ -1144,35 +1141,35 @@ private void testSequentialIntAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i += 4) { int value = random.nextInt(); - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readInt()); + assertThat(buffer.readInt()).isEqualTo(value); } else { - assertEquals(value, buffer.readIntLE()); + assertThat(buffer.readIntLE()).isEqualTo(value); } } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialUnsignedIntAccess() { + void testSequentialUnsignedIntAccess() { testSequentialUnsignedIntAccess(true); } @Test - public void testSequentialUnsignedIntLEAccess() { + void testSequentialUnsignedIntLEAccess() { testSequentialUnsignedIntAccess(false); } @@ -1180,8 +1177,8 @@ private void testSequentialUnsignedIntAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 4) { int value = random.nextInt(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeInt(value); } else { @@ -1189,35 +1186,35 @@ private void testSequentialUnsignedIntAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i += 4) { long value = random.nextInt() & 0xFFFFFFFFL; - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readUnsignedInt()); + assertThat(buffer.readUnsignedInt()).isEqualTo(value); } else { - assertEquals(value, buffer.readUnsignedIntLE()); + assertThat(buffer.readUnsignedIntLE()).isEqualTo(value); } } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testSequentialLongAccess() { + void testSequentialLongAccess() { testSequentialLongAccess(true); } @Test - public void testSequentialLongLEAccess() { + void testSequentialLongLEAccess() { testSequentialLongAccess(false); } @@ -1225,8 +1222,8 @@ private void testSequentialLongAccess(boolean testBigEndian) { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 8) { long value = random.nextLong(); - assertEquals(i, buffer.writerIndex()); - assertTrue(buffer.isWritable()); + assertThat(buffer.writerIndex()).isEqualTo(i); + assertThat(buffer.isWritable()).isTrue(); if (testBigEndian) { buffer.writeLong(value); } else { @@ -1234,30 +1231,30 @@ private void testSequentialLongAccess(boolean testBigEndian) { } } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isWritable()).isFalse(); random.setSeed(seed); for (int i = 0; i < buffer.capacity(); i += 8) { long value = random.nextLong(); - assertEquals(i, buffer.readerIndex()); - assertTrue(buffer.isReadable()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.isReadable()).isTrue(); if (testBigEndian) { - assertEquals(value, buffer.readLong()); + assertThat(buffer.readLong()).isEqualTo(value); } else { - assertEquals(value, buffer.readLongLE()); + assertThat(buffer.readLongLE()).isEqualTo(value); } } - assertEquals(buffer.capacity(), buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); - assertFalse(buffer.isReadable()); - assertFalse(buffer.isWritable()); + assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.isReadable()).isFalse(); + assertThat(buffer.isWritable()).isFalse(); } @Test - public void testByteArrayTransfer() { + void testByteArrayTransfer() { byte[] value = new byte[BLOCK_SIZE * 2]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); @@ -1271,13 +1268,13 @@ public void testByteArrayTransfer() { int valueOffset = random.nextInt(BLOCK_SIZE); buffer.getBytes(i, value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue[j], value[j]); + assertThat(value[j]).isEqualTo(expectedValue[j]); } } } @Test - public void testRandomByteArrayTransfer1() { + void testRandomByteArrayTransfer1() { byte[] value = new byte[BLOCK_SIZE]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); @@ -1291,13 +1288,13 @@ public void testRandomByteArrayTransfer1() { random.nextBytes(expectedValueContent); buffer.getBytes(i, value); for (int j = 0; j < BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value[j]); + assertThat(value[j]).isEqualTo(expectedValue.getByte(j)); } } } @Test - public void testRandomByteArrayTransfer2() { + void testRandomByteArrayTransfer2() { byte[] value = new byte[BLOCK_SIZE * 2]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); @@ -1312,21 +1309,21 @@ public void testRandomByteArrayTransfer2() { int valueOffset = random.nextInt(BLOCK_SIZE); buffer.getBytes(i, value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value[j]); + assertThat(value[j]).isEqualTo(expectedValue.getByte(j)); } } } @Test - public void testRandomHeapBufferTransfer1() { + void testRandomHeapBufferTransfer1() { byte[] valueContent = new byte[BLOCK_SIZE]; ByteBuf value = wrappedBuffer(valueContent); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setIndex(0, BLOCK_SIZE); buffer.setBytes(i, value); - assertEquals(BLOCK_SIZE, value.readerIndex()); - assertEquals(BLOCK_SIZE, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(BLOCK_SIZE); + assertThat(value.writerIndex()).isEqualTo(BLOCK_SIZE); } random.setSeed(seed); @@ -1336,16 +1333,16 @@ public void testRandomHeapBufferTransfer1() { random.nextBytes(expectedValueContent); value.clear(); buffer.getBytes(i, value); - assertEquals(0, value.readerIndex()); - assertEquals(BLOCK_SIZE, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(BLOCK_SIZE); for (int j = 0; j < BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } } } @Test - public void testRandomHeapBufferTransfer2() { + void testRandomHeapBufferTransfer2() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = wrappedBuffer(valueContent); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { @@ -1361,13 +1358,13 @@ public void testRandomHeapBufferTransfer2() { int valueOffset = random.nextInt(BLOCK_SIZE); buffer.getBytes(i, value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } } } @Test - public void testRandomDirectBufferTransfer() { + void testRandomDirectBufferTransfer() { byte[] tmp = new byte[BLOCK_SIZE * 2]; ByteBuf value = directBuffer(BLOCK_SIZE * 2); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { @@ -1384,7 +1381,7 @@ public void testRandomDirectBufferTransfer() { int valueOffset = random.nextInt(BLOCK_SIZE); buffer.getBytes(i, value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } } value.release(); @@ -1392,7 +1389,7 @@ public void testRandomDirectBufferTransfer() { } @Test - public void testRandomByteBufferTransfer() { + void testRandomByteBufferTransfer() { ByteBuffer value = ByteBuffer.allocate(BLOCK_SIZE * 2); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value.array()); @@ -1408,21 +1405,21 @@ public void testRandomByteBufferTransfer() { int valueOffset = random.nextInt(BLOCK_SIZE); value.clear().position(valueOffset).limit(valueOffset + BLOCK_SIZE); buffer.getBytes(i, value); - assertEquals(valueOffset + BLOCK_SIZE, value.position()); + assertThat(value.position()).isEqualTo(valueOffset + BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.get(j), value.get(j)); + assertThat(value.get(j)).isEqualTo(expectedValue.get(j)); } } } @Test - public void testSequentialByteArrayTransfer1() { + void testSequentialByteArrayTransfer1() { byte[] value = new byte[BLOCK_SIZE]; buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1430,23 +1427,23 @@ public void testSequentialByteArrayTransfer1() { byte[] expectedValue = new byte[BLOCK_SIZE]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValue); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); buffer.readBytes(value); for (int j = 0; j < BLOCK_SIZE; j++) { - assertEquals(expectedValue[j], value[j]); + assertThat(value[j]).isEqualTo(expectedValue[j]); } } } @Test - public void testSequentialByteArrayTransfer2() { + void testSequentialByteArrayTransfer2() { byte[] value = new byte[BLOCK_SIZE * 2]; buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); buffer.writeBytes(value, readerIndex, BLOCK_SIZE); } @@ -1456,27 +1453,27 @@ public void testSequentialByteArrayTransfer2() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValue); int valueOffset = random.nextInt(BLOCK_SIZE); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); buffer.readBytes(value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue[j], value[j]); + assertThat(value[j]).isEqualTo(expectedValue[j]); } } } @Test - public void testSequentialHeapBufferTransfer1() { + void testSequentialHeapBufferTransfer1() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = wrappedBuffer(valueContent); buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertEquals(0, value.readerIndex()); - assertEquals(valueContent.length, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(valueContent.length); } random.setSeed(seed); @@ -1485,32 +1482,32 @@ public void testSequentialHeapBufferTransfer1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValueContent); int valueOffset = random.nextInt(BLOCK_SIZE); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); buffer.readBytes(value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(0, value.readerIndex()); - assertEquals(valueContent.length, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(valueContent.length); } } @Test - public void testSequentialHeapBufferTransfer2() { + void testSequentialHeapBufferTransfer2() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = wrappedBuffer(valueContent); buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(readerIndex); value.writerIndex(readerIndex + BLOCK_SIZE); buffer.writeBytes(value); - assertEquals(readerIndex + BLOCK_SIZE, value.writerIndex()); - assertEquals(value.writerIndex(), value.readerIndex()); + assertThat(value.writerIndex()).isEqualTo(readerIndex + BLOCK_SIZE); + assertThat(value.readerIndex()).isEqualTo(value.writerIndex()); } random.setSeed(seed); @@ -1519,32 +1516,32 @@ public void testSequentialHeapBufferTransfer2() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValueContent); int valueOffset = random.nextInt(BLOCK_SIZE); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); value.readerIndex(valueOffset); value.writerIndex(valueOffset); buffer.readBytes(value, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(valueOffset, value.readerIndex()); - assertEquals(valueOffset + BLOCK_SIZE, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(valueOffset); + assertThat(value.writerIndex()).isEqualTo(valueOffset + BLOCK_SIZE); } } @Test - public void testSequentialDirectBufferTransfer1() { + void testSequentialDirectBufferTransfer1() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = directBuffer(BLOCK_SIZE * 2); buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertEquals(0, value.readerIndex()); - assertEquals(0, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(0); } random.setSeed(seed); @@ -1554,36 +1551,36 @@ public void testSequentialDirectBufferTransfer1() { random.nextBytes(expectedValueContent); int valueOffset = random.nextInt(BLOCK_SIZE); value.setBytes(0, valueContent); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); buffer.readBytes(value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(0, value.readerIndex()); - assertEquals(0, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(0); } value.release(); expectedValue.release(); } @Test - public void testSequentialDirectBufferTransfer2() { + void testSequentialDirectBufferTransfer2() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = directBuffer(BLOCK_SIZE * 2); buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(0); value.writerIndex(readerIndex + BLOCK_SIZE); value.readerIndex(readerIndex); buffer.writeBytes(value); - assertEquals(readerIndex + BLOCK_SIZE, value.writerIndex()); - assertEquals(value.writerIndex(), value.readerIndex()); + assertThat(value.writerIndex()).isEqualTo(readerIndex + BLOCK_SIZE); + assertThat(value.readerIndex()).isEqualTo(value.writerIndex()); } random.setSeed(seed); @@ -1593,23 +1590,23 @@ public void testSequentialDirectBufferTransfer2() { random.nextBytes(expectedValueContent); value.setBytes(0, valueContent); int valueOffset = random.nextInt(BLOCK_SIZE); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); value.readerIndex(valueOffset); value.writerIndex(valueOffset); buffer.readBytes(value, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(valueOffset, value.readerIndex()); - assertEquals(valueOffset + BLOCK_SIZE, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(valueOffset); + assertThat(value.writerIndex()).isEqualTo(valueOffset + BLOCK_SIZE); } value.release(); expectedValue.release(); } @Test - public void testSequentialByteBufferBackedHeapBufferTransfer1() { + void testSequentialByteBufferBackedHeapBufferTransfer1() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = wrappedBuffer(ByteBuffer.allocate(BLOCK_SIZE * 2)); value.writerIndex(0); @@ -1617,11 +1614,11 @@ public void testSequentialByteBufferBackedHeapBufferTransfer1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertEquals(0, value.readerIndex()); - assertEquals(0, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(0); } random.setSeed(seed); @@ -1631,19 +1628,19 @@ public void testSequentialByteBufferBackedHeapBufferTransfer1() { random.nextBytes(expectedValueContent); int valueOffset = random.nextInt(BLOCK_SIZE); value.setBytes(0, valueContent); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); buffer.readBytes(value, valueOffset, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(0, value.readerIndex()); - assertEquals(0, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.writerIndex()).isEqualTo(0); } } @Test - public void testSequentialByteBufferBackedHeapBufferTransfer2() { + void testSequentialByteBufferBackedHeapBufferTransfer2() { byte[] valueContent = new byte[BLOCK_SIZE * 2]; ByteBuf value = wrappedBuffer(ByteBuffer.allocate(BLOCK_SIZE * 2)); value.writerIndex(0); @@ -1651,15 +1648,15 @@ public void testSequentialByteBufferBackedHeapBufferTransfer2() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(0); value.writerIndex(readerIndex + BLOCK_SIZE); value.readerIndex(readerIndex); buffer.writeBytes(value); - assertEquals(readerIndex + BLOCK_SIZE, value.writerIndex()); - assertEquals(value.writerIndex(), value.readerIndex()); + assertThat(value.writerIndex()).isEqualTo(readerIndex + BLOCK_SIZE); + assertThat(value.readerIndex()).isEqualTo(value.writerIndex()); } random.setSeed(seed); @@ -1669,21 +1666,21 @@ public void testSequentialByteBufferBackedHeapBufferTransfer2() { random.nextBytes(expectedValueContent); value.setBytes(0, valueContent); int valueOffset = random.nextInt(BLOCK_SIZE); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); value.readerIndex(valueOffset); value.writerIndex(valueOffset); buffer.readBytes(value, BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.getByte(j), value.getByte(j)); + assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertEquals(valueOffset, value.readerIndex()); - assertEquals(valueOffset + BLOCK_SIZE, value.writerIndex()); + assertThat(value.readerIndex()).isEqualTo(valueOffset); + assertThat(value.writerIndex()).isEqualTo(valueOffset + BLOCK_SIZE); } } @Test - public void testSequentialByteBufferTransfer() { + void testSequentialByteBufferTransfer() { buffer.writerIndex(0); ByteBuffer value = ByteBuffer.allocate(BLOCK_SIZE * 2); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { @@ -1700,21 +1697,21 @@ public void testSequentialByteBufferTransfer() { int valueOffset = random.nextInt(BLOCK_SIZE); value.clear().position(valueOffset).limit(valueOffset + BLOCK_SIZE); buffer.readBytes(value); - assertEquals(valueOffset + BLOCK_SIZE, value.position()); + assertThat(value.position()).isEqualTo(valueOffset + BLOCK_SIZE); for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { - assertEquals(expectedValue.get(j), value.get(j)); + assertThat(value.get(j)).isEqualTo(expectedValue.get(j)); } } } @Test - public void testSequentialCopiedBufferTransfer1() { + void testSequentialCopiedBufferTransfer1() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { byte[] value = new byte[BLOCK_SIZE]; random.nextBytes(value); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1722,26 +1719,26 @@ public void testSequentialCopiedBufferTransfer1() { byte[] expectedValue = new byte[BLOCK_SIZE]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValue); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); ByteBuf actualValue = buffer.readBytes(BLOCK_SIZE); - assertEquals(wrappedBuffer(expectedValue), actualValue); + assertThat(actualValue).isEqualTo(wrappedBuffer(expectedValue)); // Make sure if it is a copied buffer. actualValue.setByte(0, (byte) (actualValue.getByte(0) + 1)); - assertFalse(buffer.getByte(i) == actualValue.getByte(0)); + assertThat(buffer.getByte(i) == actualValue.getByte(0)).isFalse(); actualValue.release(); } } @Test - public void testSequentialSlice1() { + void testSequentialSlice1() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { byte[] value = new byte[BLOCK_SIZE]; random.nextBytes(value); - assertEquals(0, buffer.readerIndex()); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1749,26 +1746,21 @@ public void testSequentialSlice1() { byte[] expectedValue = new byte[BLOCK_SIZE]; for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(expectedValue); - assertEquals(i, buffer.readerIndex()); - assertEquals(CAPACITY, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY); ByteBuf actualValue = buffer.readSlice(BLOCK_SIZE); - assertEquals(buffer.order(), actualValue.order()); - assertEquals(wrappedBuffer(expectedValue), actualValue); + assertThat(actualValue.order()).isEqualTo(buffer.order()); + assertThat(actualValue).isEqualTo(wrappedBuffer(expectedValue)); // Make sure if it is a sliced buffer. actualValue.setByte(0, (byte) (actualValue.getByte(0) + 1)); - assertEquals(buffer.getByte(i), actualValue.getByte(0)); + assertThat(actualValue.getByte(0)).isEqualTo(buffer.getByte(i)); } } @Test - public void testWriteZero() { - try { - buffer.writeZero(-1); - fail(); - } catch (IllegalArgumentException e) { - // Expected - } + void testWriteZero() { + assertThatThrownBy(() -> buffer.writeZero(-1)).isInstanceOf(IllegalArgumentException.class); buffer.clear(); while (buffer.isWritable()) { @@ -1782,16 +1774,16 @@ public void testWriteZero() { i += length; } - assertEquals(0, buffer.readerIndex()); - assertEquals(buffer.capacity(), buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); for (int i = 0; i < buffer.capacity(); i++) { - assertEquals(0, buffer.getByte(i)); + assertThat(buffer.getByte(i)).isEqualTo((byte) 0); } } @Test - public void testDiscardReadBytes() { + void testDiscardReadBytes() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i += 4) { buffer.writeInt(i); @@ -1807,40 +1799,40 @@ public void testDiscardReadBytes() { buffer.writerIndex(CAPACITY / 2); buffer.discardReadBytes(); - assertEquals(0, buffer.readerIndex()); - assertEquals(CAPACITY / 2, buffer.writerIndex()); - assertEquals(copy.slice(0, CAPACITY / 2), buffer.slice(0, CAPACITY / 2)); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2); + assertThat(buffer.slice(0, CAPACITY / 2)).isEqualTo(copy.slice(0, CAPACITY / 2)); buffer.resetReaderIndex(); - assertEquals(CAPACITY / 4, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(CAPACITY / 4); buffer.resetWriterIndex(); - assertEquals(CAPACITY / 3, buffer.writerIndex()); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 3); // Make sure bytes after writerIndex is not copied. buffer.readerIndex(1); buffer.writerIndex(CAPACITY / 2); buffer.discardReadBytes(); - assertEquals(0, buffer.readerIndex()); - assertEquals(CAPACITY / 2 - 1, buffer.writerIndex()); - assertEquals(copy.slice(1, CAPACITY / 2 - 1), buffer.slice(0, CAPACITY / 2 - 1)); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2 - 1); + assertThat(buffer.slice(0, CAPACITY / 2 - 1)).isEqualTo(copy.slice(1, CAPACITY / 2 - 1)); if (discardReadBytesDoesNotMoveWritableBytes()) { // If writable bytes were copied, the test should fail to avoid unnecessary memory // bandwidth consumption. - assertFalse( - copy.slice(CAPACITY / 2, CAPACITY / 2) - .equals(buffer.slice(CAPACITY / 2 - 1, CAPACITY / 2))); + assertThat( + copy.slice(CAPACITY / 2, CAPACITY / 2) + .equals(buffer.slice(CAPACITY / 2 - 1, CAPACITY / 2))) + .isFalse(); } else { - assertEquals( - copy.slice(CAPACITY / 2, CAPACITY / 2), - buffer.slice(CAPACITY / 2 - 1, CAPACITY / 2)); + assertThat(buffer.slice(CAPACITY / 2 - 1, CAPACITY / 2)) + .isEqualTo(copy.slice(CAPACITY / 2, CAPACITY / 2)); } // Marks also should be relocated. buffer.resetReaderIndex(); - assertEquals(CAPACITY / 4 - 1, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(CAPACITY / 4 - 1); buffer.resetWriterIndex(); - assertEquals(CAPACITY / 3 - 1, buffer.writerIndex()); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 3 - 1); copy.release(); } @@ -1849,7 +1841,7 @@ public void testDiscardReadBytes() { * chunk at once. */ @Test - public void testDiscardReadBytes2() { + void testDiscardReadBytes2() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity(); i++) { buffer.writeByte((byte) i); @@ -1859,25 +1851,24 @@ public void testDiscardReadBytes2() { // Discard the first (CAPACITY / 2 - 1) bytes. buffer.setIndex(CAPACITY / 2 - 1, CAPACITY - 1); buffer.discardReadBytes(); - assertEquals(0, buffer.readerIndex()); - assertEquals(CAPACITY / 2, buffer.writerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2); for (int i = 0; i < CAPACITY / 2; i++) { - assertEquals( - copy.slice(CAPACITY / 2 - 1 + i, CAPACITY / 2 - i), - buffer.slice(i, CAPACITY / 2 - i)); + assertThat(buffer.slice(i, CAPACITY / 2 - i)) + .isEqualTo(copy.slice(CAPACITY / 2 - 1 + i, CAPACITY / 2 - i)); } copy.release(); } @Test - public void testStreamTransfer1() throws Exception { + void testStreamTransfer1() throws Exception { byte[] expected = new byte[buffer.capacity()]; random.nextBytes(expected); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { ByteArrayInputStream in = new ByteArrayInputStream(expected, i, BLOCK_SIZE); - assertEquals(BLOCK_SIZE, buffer.setBytes(i, in, BLOCK_SIZE)); - assertEquals(-1, buffer.setBytes(i, in, 0)); + assertThat(buffer.setBytes(i, in, BLOCK_SIZE)).isEqualTo(BLOCK_SIZE); + assertThat(buffer.setBytes(i, in, 0)).isEqualTo(-1); } ByteArrayOutputStream out = new ByteArrayOutputStream(); @@ -1885,34 +1876,34 @@ public void testStreamTransfer1() throws Exception { buffer.getBytes(i, out, BLOCK_SIZE); } - assertTrue(Arrays.equals(expected, out.toByteArray())); + assertThat(Arrays.equals(expected, out.toByteArray())).isTrue(); } @Test - public void testStreamTransfer2() throws Exception { + void testStreamTransfer2() throws Exception { byte[] expected = new byte[buffer.capacity()]; random.nextBytes(expected); buffer.clear(); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { ByteArrayInputStream in = new ByteArrayInputStream(expected, i, BLOCK_SIZE); - assertEquals(i, buffer.writerIndex()); + assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(in, BLOCK_SIZE); - assertEquals(i + BLOCK_SIZE, buffer.writerIndex()); + assertThat(buffer.writerIndex()).isEqualTo(i + BLOCK_SIZE); } ByteArrayOutputStream out = new ByteArrayOutputStream(); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { - assertEquals(i, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i); buffer.readBytes(out, BLOCK_SIZE); - assertEquals(i + BLOCK_SIZE, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(i + BLOCK_SIZE); } - assertTrue(Arrays.equals(expected, out.toByteArray())); + assertThat(Arrays.equals(expected, out.toByteArray())).isTrue(); } @Test - public void testCopy() { + void testCopy() { for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); buffer.setByte(i, value); @@ -1924,24 +1915,24 @@ public void testCopy() { // Make sure all properties are copied. ByteBuf copy = buffer.copy(); - assertEquals(0, copy.readerIndex()); - assertEquals(buffer.readableBytes(), copy.writerIndex()); - assertEquals(buffer.readableBytes(), copy.capacity()); - assertSame(buffer.order(), copy.order()); + assertThat(copy.readerIndex()).isEqualTo(0); + assertThat(copy.writerIndex()).isEqualTo(buffer.readableBytes()); + assertThat(copy.capacity()).isEqualTo(buffer.readableBytes()); + assertThat(copy.order()).isSameAs(buffer.order()); for (int i = 0; i < copy.capacity(); i++) { - assertEquals(buffer.getByte(i + readerIndex), copy.getByte(i)); + assertThat(copy.getByte(i)).isEqualTo(buffer.getByte(i + readerIndex)); } // Make sure the buffer content is independent from each other. buffer.setByte(readerIndex, (byte) (buffer.getByte(readerIndex) + 1)); - assertTrue(buffer.getByte(readerIndex) != copy.getByte(0)); + assertThat(buffer.getByte(readerIndex) != copy.getByte(0)).isTrue(); copy.setByte(1, (byte) (copy.getByte(1) + 1)); - assertTrue(buffer.getByte(readerIndex + 1) != copy.getByte(1)); + assertThat(buffer.getByte(readerIndex + 1) != copy.getByte(1)).isTrue(); copy.release(); } @Test - public void testDuplicate() { + void testDuplicate() { for (int i = 0; i < buffer.capacity(); i++) { byte value = (byte) random.nextInt(); buffer.setByte(i, value); @@ -1953,101 +1944,101 @@ public void testDuplicate() { // Make sure all properties are copied. ByteBuf duplicate = buffer.duplicate(); - assertSame(buffer.order(), duplicate.order()); - assertEquals(buffer.readableBytes(), duplicate.readableBytes()); - assertEquals(0, buffer.compareTo(duplicate)); + assertThat(duplicate.order()).isSameAs(buffer.order()); + assertThat(duplicate.readableBytes()).isEqualTo(buffer.readableBytes()); + assertThat(buffer.compareTo(duplicate)).isEqualTo(0); // Make sure the buffer content is shared. buffer.setByte(readerIndex, (byte) (buffer.getByte(readerIndex) + 1)); - assertEquals(buffer.getByte(readerIndex), duplicate.getByte(duplicate.readerIndex())); + assertThat(duplicate.getByte(duplicate.readerIndex())) + .isEqualTo(buffer.getByte(readerIndex)); duplicate.setByte( duplicate.readerIndex(), (byte) (duplicate.getByte(duplicate.readerIndex()) + 1)); - assertEquals(buffer.getByte(readerIndex), duplicate.getByte(duplicate.readerIndex())); + assertThat(duplicate.getByte(duplicate.readerIndex())) + .isEqualTo(buffer.getByte(readerIndex)); } @Test - public void testSliceEndianness() throws Exception { - assertEquals(buffer.order(), buffer.slice(0, buffer.capacity()).order()); - assertEquals(buffer.order(), buffer.slice(0, buffer.capacity() - 1).order()); - assertEquals(buffer.order(), buffer.slice(1, buffer.capacity() - 1).order()); - assertEquals(buffer.order(), buffer.slice(1, buffer.capacity() - 2).order()); + void testSliceEndianness() throws Exception { + assertThat(buffer.slice(0, buffer.capacity()).order()).isEqualTo(buffer.order()); + assertThat(buffer.slice(0, buffer.capacity() - 1).order()).isEqualTo(buffer.order()); + assertThat(buffer.slice(1, buffer.capacity() - 1).order()).isEqualTo(buffer.order()); + assertThat(buffer.slice(1, buffer.capacity() - 2).order()).isEqualTo(buffer.order()); } @Test - public void testSliceIndex() throws Exception { - assertEquals(0, buffer.slice(0, buffer.capacity()).readerIndex()); - assertEquals(0, buffer.slice(0, buffer.capacity() - 1).readerIndex()); - assertEquals(0, buffer.slice(1, buffer.capacity() - 1).readerIndex()); - assertEquals(0, buffer.slice(1, buffer.capacity() - 2).readerIndex()); + void testSliceIndex() throws Exception { + assertThat(buffer.slice(0, buffer.capacity()).readerIndex()).isEqualTo(0); + assertThat(buffer.slice(0, buffer.capacity() - 1).readerIndex()).isEqualTo(0); + assertThat(buffer.slice(1, buffer.capacity() - 1).readerIndex()).isEqualTo(0); + assertThat(buffer.slice(1, buffer.capacity() - 2).readerIndex()).isEqualTo(0); - assertEquals(buffer.capacity(), buffer.slice(0, buffer.capacity()).writerIndex()); - assertEquals(buffer.capacity() - 1, buffer.slice(0, buffer.capacity() - 1).writerIndex()); - assertEquals(buffer.capacity() - 1, buffer.slice(1, buffer.capacity() - 1).writerIndex()); - assertEquals(buffer.capacity() - 2, buffer.slice(1, buffer.capacity() - 2).writerIndex()); + assertThat(buffer.slice(0, buffer.capacity()).writerIndex()).isEqualTo(buffer.capacity()); + assertThat(buffer.slice(0, buffer.capacity() - 1).writerIndex()) + .isEqualTo(buffer.capacity() - 1); + assertThat(buffer.slice(1, buffer.capacity() - 1).writerIndex()) + .isEqualTo(buffer.capacity() - 1); + assertThat(buffer.slice(1, buffer.capacity() - 2).writerIndex()) + .isEqualTo(buffer.capacity() - 2); } @Test - public void testRetainedSliceIndex() throws Exception { + void testRetainedSliceIndex() throws Exception { ByteBuf retainedSlice = buffer.retainedSlice(0, buffer.capacity()); - assertEquals(0, retainedSlice.readerIndex()); + assertThat(retainedSlice.readerIndex()).isEqualTo(0); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, buffer.capacity() - 1); - assertEquals(0, retainedSlice.readerIndex()); + assertThat(retainedSlice.readerIndex()).isEqualTo(0); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 1); - assertEquals(0, retainedSlice.readerIndex()); + assertThat(retainedSlice.readerIndex()).isEqualTo(0); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 2); - assertEquals(0, retainedSlice.readerIndex()); + assertThat(retainedSlice.readerIndex()).isEqualTo(0); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, buffer.capacity()); - assertEquals(buffer.capacity(), retainedSlice.writerIndex()); + assertThat(retainedSlice.writerIndex()).isEqualTo(buffer.capacity()); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, buffer.capacity() - 1); - assertEquals(buffer.capacity() - 1, retainedSlice.writerIndex()); + assertThat(retainedSlice.writerIndex()).isEqualTo(buffer.capacity() - 1); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 1); - assertEquals(buffer.capacity() - 1, retainedSlice.writerIndex()); + assertThat(retainedSlice.writerIndex()).isEqualTo(buffer.capacity() - 1); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 2); - assertEquals(buffer.capacity() - 2, retainedSlice.writerIndex()); + assertThat(retainedSlice.writerIndex()).isEqualTo(buffer.capacity() - 2); retainedSlice.release(); } @Test @SuppressWarnings("ObjectEqualsNull") - public void testEquals() { - assertFalse(buffer.equals(null)); - assertFalse(buffer.equals(new Object())); + void testEquals() { + assertThat(buffer.equals(null)).isFalse(); + assertThat(buffer.equals(new Object())).isFalse(); byte[] value = new byte[32]; buffer.setIndex(0, value.length); random.nextBytes(value); buffer.setBytes(0, value); - assertEquals(buffer, wrappedBuffer(value)); - assertEquals(buffer, wrappedBuffer(value).order(LITTLE_ENDIAN)); + assertThat(wrappedBuffer(value)).isEqualTo(buffer); + assertThat(wrappedBuffer(value).order(LITTLE_ENDIAN)).isEqualTo(buffer); value[0]++; - assertFalse(buffer.equals(wrappedBuffer(value))); - assertFalse(buffer.equals(wrappedBuffer(value).order(LITTLE_ENDIAN))); + assertThat(buffer.equals(wrappedBuffer(value))).isFalse(); + assertThat(buffer.equals(wrappedBuffer(value).order(LITTLE_ENDIAN))).isFalse(); } @Test - public void testCompareTo() { - try { - buffer.compareTo(null); - fail(); - } catch (NullPointerException e) { - // Expected - } + void testCompareTo() { + assertThatThrownBy(() -> buffer.compareTo(null)).isInstanceOf(NullPointerException.class); // Fill the random stuff byte[] value = new byte[32]; @@ -2062,33 +2053,34 @@ public void testCompareTo() { buffer.setIndex(0, value.length); buffer.setBytes(0, value); - assertEquals(0, buffer.compareTo(wrappedBuffer(value))); - assertEquals(0, buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN))); + assertThat(buffer.compareTo(wrappedBuffer(value))).isEqualTo(0); + assertThat(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN))).isEqualTo(0); value[0]++; - assertTrue(buffer.compareTo(wrappedBuffer(value)) < 0); - assertTrue(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0); + assertThat(buffer.compareTo(wrappedBuffer(value)) < 0).isTrue(); + assertThat(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0).isTrue(); value[0] -= 2; - assertTrue(buffer.compareTo(wrappedBuffer(value)) > 0); - assertTrue(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) > 0); + assertThat(buffer.compareTo(wrappedBuffer(value)) > 0).isTrue(); + assertThat(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) > 0).isTrue(); value[0]++; - assertTrue(buffer.compareTo(wrappedBuffer(value, 0, 31)) > 0); - assertTrue(buffer.compareTo(wrappedBuffer(value, 0, 31).order(LITTLE_ENDIAN)) > 0); - assertTrue(buffer.slice(0, 31).compareTo(wrappedBuffer(value)) < 0); - assertTrue(buffer.slice(0, 31).compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0); + assertThat(buffer.compareTo(wrappedBuffer(value, 0, 31)) > 0).isTrue(); + assertThat(buffer.compareTo(wrappedBuffer(value, 0, 31).order(LITTLE_ENDIAN)) > 0).isTrue(); + assertThat(buffer.slice(0, 31).compareTo(wrappedBuffer(value)) < 0).isTrue(); + assertThat(buffer.slice(0, 31).compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0) + .isTrue(); ByteBuf retainedSlice = buffer.retainedSlice(0, 31); - assertTrue(retainedSlice.compareTo(wrappedBuffer(value)) < 0); + assertThat(retainedSlice.compareTo(wrappedBuffer(value)) < 0).isTrue(); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, 31); - assertTrue(retainedSlice.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0); + assertThat(retainedSlice.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0).isTrue(); retainedSlice.release(); } @Test - public void testCompareTo2() { + void testCompareTo2() { byte[] bytes = {1, 2, 3, 4}; byte[] bytesReversed = {4, 3, 2, 1}; @@ -2098,10 +2090,10 @@ public void testCompareTo2() { ByteBuf buf3 = newBuffer(4).clear().writeBytes(bytes).order(ByteOrder.BIG_ENDIAN); ByteBuf buf4 = newBuffer(4).clear().writeBytes(bytesReversed).order(ByteOrder.BIG_ENDIAN); try { - assertEquals(buf1.compareTo(buf2), buf3.compareTo(buf4)); - assertEquals(buf2.compareTo(buf1), buf4.compareTo(buf3)); - assertEquals(buf1.compareTo(buf3), buf2.compareTo(buf4)); - assertEquals(buf3.compareTo(buf1), buf4.compareTo(buf2)); + assertThat(buf3.compareTo(buf4)).isEqualTo(buf1.compareTo(buf2)); + assertThat(buf4.compareTo(buf3)).isEqualTo(buf2.compareTo(buf1)); + assertThat(buf2.compareTo(buf4)).isEqualTo(buf1.compareTo(buf3)); + assertThat(buf4.compareTo(buf2)).isEqualTo(buf3.compareTo(buf1)); } finally { buf1.release(); buf2.release(); @@ -2111,16 +2103,17 @@ public void testCompareTo2() { } @Test - public void testToString() { + void testToString() { ByteBuf copied = copiedBuffer("Hello, World!", CharsetUtil.ISO_8859_1); buffer.clear(); buffer.writeBytes(copied); - assertEquals("Hello, World!", buffer.toString(CharsetUtil.ISO_8859_1)); + assertThat(buffer.toString(CharsetUtil.ISO_8859_1)).isEqualTo("Hello, World!"); copied.release(); } - @Test(timeout = 10000) - public void testToStringMultipleThreads() throws Throwable { + @Test + @Timeout(10) + void testToStringMultipleThreads() throws Throwable { buffer.clear(); buffer.writeBytes("Hello, World!".getBytes(CharsetUtil.ISO_8859_1)); @@ -2136,9 +2129,8 @@ public void run() { try { while (errorRef.get() == null && counter.decrementAndGet() > 0) { - assertEquals( - "Hello, World!", - buffer.toString(CharsetUtil.ISO_8859_1)); + assertThat(buffer.toString(CharsetUtil.ISO_8859_1)) + .isEqualTo("Hello, World!"); } } catch (Throwable cause) { errorRef.compareAndSet(null, cause); @@ -2162,7 +2154,7 @@ public void run() { } @Test - public void testIndexOf() { + void testIndexOf() { buffer.clear(); buffer.writeByte((byte) 1); buffer.writeByte((byte) 2); @@ -2170,15 +2162,15 @@ public void testIndexOf() { buffer.writeByte((byte) 2); buffer.writeByte((byte) 1); - assertEquals(-1, buffer.indexOf(1, 4, (byte) 1)); - assertEquals(-1, buffer.indexOf(4, 1, (byte) 1)); - assertEquals(1, buffer.indexOf(1, 4, (byte) 2)); - assertEquals(3, buffer.indexOf(4, 1, (byte) 2)); + assertThat(buffer.indexOf(1, 4, (byte) 1)).isEqualTo(-1); + assertThat(buffer.indexOf(4, 1, (byte) 1)).isEqualTo(-1); + assertThat(buffer.indexOf(1, 4, (byte) 2)).isEqualTo(1); + assertThat(buffer.indexOf(4, 1, (byte) 2)).isEqualTo(3); } @Test - public void testNioBuffer1() { - assumeTrue(buffer.nioBufferCount() == 1); + void testNioBuffer1() { + assumeThat(buffer.nioBufferCount() == 1).isTrue(); byte[] value = new byte[buffer.capacity()]; random.nextBytes(value); @@ -2189,8 +2181,8 @@ public void testNioBuffer1() { } @Test - public void testToByteBuffer2() { - assumeTrue(buffer.nioBufferCount() == 1); + void testToByteBuffer2() { + assumeThat(buffer.nioBufferCount() == 1).isTrue(); byte[] value = new byte[buffer.capacity()]; random.nextBytes(value); @@ -2207,41 +2199,37 @@ private static void assertRemainingEquals(ByteBuffer expected, ByteBuffer actual int remaining = expected.remaining(); int remaining2 = actual.remaining(); - assertEquals(remaining, remaining2); + assumeThat(remaining2).isEqualTo(remaining); byte[] array1 = new byte[remaining]; byte[] array2 = new byte[remaining2]; expected.get(array1); actual.get(array2); - assertArrayEquals(array1, array2); + assertThat(array2).isEqualTo(array1); } @Test - public void testToByteBuffer3() { - assumeTrue(buffer.nioBufferCount() == 1); + void testToByteBuffer3() { + assumeThat(buffer.nioBufferCount() == 1).isTrue(); - assertEquals(buffer.order(), buffer.nioBuffer().order()); + assertThat(buffer.nioBuffer().order()).isEqualTo(buffer.order()); } @Test - public void testSkipBytes1() { + void testSkipBytes1() { buffer.setIndex(CAPACITY / 4, CAPACITY / 2); buffer.skipBytes(CAPACITY / 4); - assertEquals(CAPACITY / 4 * 2, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(CAPACITY / 4 * 2); - try { - buffer.skipBytes(CAPACITY / 4 + 1); - fail(); - } catch (IndexOutOfBoundsException e) { - // Expected - } + assertThatThrownBy(() -> buffer.skipBytes(CAPACITY / 4 + 1)) + .isInstanceOf(IndexOutOfBoundsException.class); // Should remain unchanged. - assertEquals(CAPACITY / 4 * 2, buffer.readerIndex()); + assertThat(buffer.readerIndex()).isEqualTo(CAPACITY / 4 * 2); } @Test - public void testHashCode() { + void testHashCode() { ByteBuf elemA = buffer(15); ByteBuf elemB = directBuffer(15); elemA.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0, 1, 2, 3, 4, 5}); @@ -2251,25 +2239,25 @@ public void testHashCode() { set.add(elemA); set.add(elemB); - assertEquals(2, set.size()); + assertThat(set.size()).isEqualTo(2); ByteBuf elemACopy = elemA.copy(); - assertTrue(set.contains(elemACopy)); + assertThat(set.contains(elemACopy)).isTrue(); ByteBuf elemBCopy = elemB.copy(); - assertTrue(set.contains(elemBCopy)); + assertThat(set.contains(elemBCopy)).isTrue(); buffer.clear(); buffer.writeBytes(elemA.duplicate()); - assertTrue(set.remove(buffer)); - assertFalse(set.contains(elemA)); - assertEquals(1, set.size()); + assertThat(set.remove(buffer)).isTrue(); + assertThat(set.contains(elemA)).isFalse(); + assertThat(set.size()).isEqualTo(1); buffer.clear(); buffer.writeBytes(elemB.duplicate()); - assertTrue(set.remove(buffer)); - assertFalse(set.contains(elemB)); - assertEquals(0, set.size()); + assertThat(set.remove(buffer)).isTrue(); + assertThat(set.contains(elemB)).isFalse(); + assertThat(set.size()).isEqualTo(0); elemA.release(); elemB.release(); elemACopy.release(); @@ -2278,14 +2266,14 @@ public void testHashCode() { // Test case for https://github.com/netty/netty/issues/325 @Test - public void testDiscardAllReadBytes() { + void testDiscardAllReadBytes() { buffer.writerIndex(buffer.capacity()); buffer.readerIndex(buffer.writerIndex()); buffer.discardReadBytes(); } @Test - public void testForEachByte() { + void testForEachByte() { buffer.clear(); for (int i = 0; i < CAPACITY; i++) { buffer.writeByte(i + 1); @@ -2294,25 +2282,25 @@ public void testForEachByte() { final AtomicInteger lastIndex = new AtomicInteger(); buffer.setIndex(CAPACITY / 4, CAPACITY * 3 / 4); assertThat( - buffer.forEachByte( - new ByteProcessor() { - int i = CAPACITY / 4; - - @Override - public boolean process(byte value) throws Exception { - assertThat(value, is((byte) (i + 1))); - lastIndex.set(i); - i++; - return true; - } - }), - is(-1)); + buffer.forEachByte( + new ByteProcessor() { + int i = CAPACITY / 4; + + @Override + public boolean process(byte value) throws Exception { + assertThat(value).isEqualTo((byte) (i + 1)); + lastIndex.set(i); + i++; + return true; + } + })) + .isEqualTo(-1); - assertThat(lastIndex.get(), is(CAPACITY * 3 / 4 - 1)); + assertThat(lastIndex.get()).isEqualTo(CAPACITY * 3 / 4 - 1); } @Test - public void testForEachByteAbort() { + void testForEachByteAbort() { buffer.clear(); for (int i = 0; i < CAPACITY; i++) { buffer.writeByte(i + 1); @@ -2320,28 +2308,28 @@ public void testForEachByteAbort() { final int stop = CAPACITY / 2; assertThat( - buffer.forEachByte( - CAPACITY / 3, - CAPACITY / 3, - new ByteProcessor() { - int i = CAPACITY / 3; - - @Override - public boolean process(byte value) throws Exception { - assertThat(value, is((byte) (i + 1))); - if (i == stop) { - return false; - } + buffer.forEachByte( + CAPACITY / 3, + CAPACITY / 3, + new ByteProcessor() { + int i = CAPACITY / 3; + + @Override + public boolean process(byte value) throws Exception { + assertThat(value).isEqualTo((byte) (i + 1)); + if (i == stop) { + return false; + } - i++; - return true; - } - }), - is(stop)); + i++; + return true; + } + })) + .isEqualTo(stop); } @Test - public void testForEachByteDesc() { + void testForEachByteDesc() { buffer.clear(); for (int i = 0; i < CAPACITY; i++) { buffer.writeByte(i + 1); @@ -2349,27 +2337,27 @@ public void testForEachByteDesc() { final AtomicInteger lastIndex = new AtomicInteger(); assertThat( - buffer.forEachByteDesc( - CAPACITY / 4, - CAPACITY * 2 / 4, - new ByteProcessor() { - int i = CAPACITY * 3 / 4 - 1; - - @Override - public boolean process(byte value) throws Exception { - assertThat(value, is((byte) (i + 1))); - lastIndex.set(i); - i--; - return true; - } - }), - is(-1)); + buffer.forEachByteDesc( + CAPACITY / 4, + CAPACITY * 2 / 4, + new ByteProcessor() { + int i = CAPACITY * 3 / 4 - 1; + + @Override + public boolean process(byte value) throws Exception { + assertThat(value).isEqualTo((byte) (i + 1)); + lastIndex.set(i); + i--; + return true; + } + })) + .isEqualTo(-1); - assertThat(lastIndex.get(), is(CAPACITY / 4)); + assertThat(lastIndex.get()).isEqualTo(CAPACITY / 4); } @Test - public void testInternalNioBuffer() { + void testInternalNioBuffer() { testInternalNioBuffer(128); testInternalNioBuffer(1024); testInternalNioBuffer(4 * 1024); @@ -2381,29 +2369,29 @@ public void testInternalNioBuffer() { private void testInternalNioBuffer(int a) { ByteBuf buffer = newBuffer(2); ByteBuffer buf = buffer.internalNioBuffer(buffer.readerIndex(), 1); - assertEquals(1, buf.remaining()); + assertThat(buf.remaining()).isEqualTo(1); byte[] data = new byte[a]; PlatformDependent.threadLocalRandom().nextBytes(data); buffer.writeBytes(data); buf = buffer.internalNioBuffer(buffer.readerIndex(), a); - assertEquals(a, buf.remaining()); + assertThat(buf.remaining()).isEqualTo(a); for (int i = 0; i < a; i++) { - assertEquals(data[i], buf.get()); + assertThat(buf.get()).isEqualTo(data[i]); } - assertFalse(buf.hasRemaining()); + assertThat(buf.hasRemaining()).isFalse(); buffer.release(); } @Test - public void testDuplicateReadGatheringByteChannelMultipleThreads() throws Exception { + void testDuplicateReadGatheringByteChannelMultipleThreads() throws Exception { testReadGatheringByteChannelMultipleThreads(false); } @Test - public void testSliceReadGatheringByteChannelMultipleThreads() throws Exception { + void testSliceReadGatheringByteChannelMultipleThreads() throws Exception { testReadGatheringByteChannelMultipleThreads(true); } @@ -2438,7 +2426,7 @@ public void run() { return; } } - assertArrayEquals(bytes, channel.writtenBytes()); + assertThat(channel.writtenBytes()).isEqualTo(bytes); latch.countDown(); } try { @@ -2450,18 +2438,18 @@ public void run() { }) .start(); } - assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); barrier.await(5, TimeUnit.SECONDS); buffer.release(); } @Test - public void testDuplicateReadOutputStreamMultipleThreads() throws Exception { + void testDuplicateReadOutputStreamMultipleThreads() throws Exception { testReadOutputStreamMultipleThreads(false); } @Test - public void testSliceReadOutputStreamMultipleThreads() throws Exception { + void testSliceReadOutputStreamMultipleThreads() throws Exception { testReadOutputStreamMultipleThreads(true); } @@ -2495,7 +2483,7 @@ public void run() { return; } } - assertArrayEquals(bytes, out.toByteArray()); + assertThat(out.toByteArray()).isEqualTo(bytes); latch.countDown(); } try { @@ -2507,18 +2495,18 @@ public void run() { }) .start(); } - assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); barrier.await(5, TimeUnit.SECONDS); buffer.release(); } @Test - public void testDuplicateBytesInArrayMultipleThreads() throws Exception { + void testDuplicateBytesInArrayMultipleThreads() throws Exception { testBytesInArrayMultipleThreads(false); } @Test - public void testSliceBytesInArrayMultipleThreads() throws Exception { + void testSliceBytesInArrayMultipleThreads() throws Exception { testBytesInArrayMultipleThreads(true); } @@ -2547,11 +2535,11 @@ public void run() { byte[] array = new byte[8]; buf.readBytes(array); - assertArrayEquals(bytes, array); + assertThat(array).isEqualTo(bytes); Arrays.fill(array, (byte) 0); buf.getBytes(0, array); - assertArrayEquals(bytes, array); + assertThat(array).isEqualTo(bytes); latch.countDown(); } @@ -2564,19 +2552,19 @@ public void run() { }) .start(); } - assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertThat(latch.await(10, TimeUnit.SECONDS)).isTrue(); barrier.await(5, TimeUnit.SECONDS); - assertNull(cause.get()); + assertThat(cause.get()).isNull(); buffer.release(); } - @Test(expected = IndexOutOfBoundsException.class) - public void readByteThrowsIndexOutOfBoundsException() { + @Test + void readByteThrowsIndexOutOfBoundsException() { final ByteBuf buffer = newBuffer(8); try { buffer.writeByte(0); - assertEquals((byte) 0, buffer.readByte()); - buffer.readByte(); + assertThat(buffer.readByte()).isEqualTo((byte) 0); + assertThatThrownBy(buffer::readByte).isInstanceOf(IndexOutOfBoundsException.class); } finally { buffer.release(); } @@ -2584,29 +2572,29 @@ public void readByteThrowsIndexOutOfBoundsException() { @Test @SuppressWarnings("ForLoopThatDoesntUseLoopVariable") - public void testNioBufferExposeOnlyRegion() { + void testNioBufferExposeOnlyRegion() { final ByteBuf buffer = newBuffer(8); byte[] data = new byte[8]; random.nextBytes(data); buffer.writeBytes(data); ByteBuffer nioBuf = buffer.nioBuffer(1, data.length - 2); - assertEquals(0, nioBuf.position()); - assertEquals(6, nioBuf.remaining()); + assertThat(nioBuf.position()).isEqualTo(0); + assertThat(nioBuf.remaining()).isEqualTo(6); for (int i = 1; nioBuf.hasRemaining(); i++) { - assertEquals(data[i], nioBuf.get()); + assertThat(nioBuf.get()).isEqualTo(data[i]); } buffer.release(); } @Test - public void ensureWritableWithForceDoesNotThrow() { + void ensureWritableWithForceDoesNotThrow() { ensureWritableDoesNotThrow(true); } @Test - public void ensureWritableWithOutForceDoesNotThrow() { + void ensureWritableWithOutForceDoesNotThrow() { ensureWritableDoesNotThrow(false); } @@ -2621,10 +2609,10 @@ private void ensureWritableDoesNotThrow(boolean force) { // - https://github.com/netty/netty/issues/2587 // - https://github.com/netty/netty/issues/2580 @Test - public void testLittleEndianWithExpand() { + void testLittleEndianWithExpand() { ByteBuf buffer = newBuffer(0).order(LITTLE_ENDIAN); buffer.writeInt(0x12345678); - assertEquals("78563412", ByteBufUtil.hexDump(buffer)); + assertThat(ByteBufUtil.hexDump(buffer)).isEqualTo("78563412"); buffer.release(); } @@ -2634,758 +2622,884 @@ private ByteBuf releasedBuffer() { // Clear the buffer so we are sure the reader and writer indices are 0. // This is important as we may return a slice from newBuffer(...). buffer.clear(); - assertTrue(buffer.release()); + assertThat(buffer.release()).isTrue(); return buffer; } - @Test(expected = IllegalReferenceCountException.class) - public void testDiscardReadBytesAfterRelease() { - releasedBuffer().discardReadBytes(); + @Test + void testDiscardReadBytesAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().discardReadBytes()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testDiscardSomeReadBytesAfterRelease() { - releasedBuffer().discardSomeReadBytes(); + @Test + void testDiscardSomeReadBytesAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().discardSomeReadBytes()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testEnsureWritableAfterRelease() { - releasedBuffer().ensureWritable(16); + @Test + void testEnsureWritableAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().ensureWritable(16)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBooleanAfterRelease() { - releasedBuffer().getBoolean(0); + @Test + void testGetBooleanAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getBoolean(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetByteAfterRelease() { - releasedBuffer().getByte(0); + @Test + void testGetByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getByte(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedByteAfterRelease() { - releasedBuffer().getUnsignedByte(0); + @Test + void testGetUnsignedByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedByte(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetShortAfterRelease() { - releasedBuffer().getShort(0); + @Test + void testGetShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getShort(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetShortLEAfterRelease() { - releasedBuffer().getShortLE(0); + @Test + void testGetShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getShortLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedShortAfterRelease() { - releasedBuffer().getUnsignedShort(0); + @Test + void testGetUnsignedShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedShort(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedShortLEAfterRelease() { - releasedBuffer().getUnsignedShortLE(0); + @Test + void testGetUnsignedShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedShortLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetMediumAfterRelease() { - releasedBuffer().getMedium(0); + @Test + void testGetMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getMedium(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetMediumLEAfterRelease() { - releasedBuffer().getMediumLE(0); + @Test + void testGetMediumLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getMediumLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedMediumAfterRelease() { - releasedBuffer().getUnsignedMedium(0); + @Test + void testGetUnsignedMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedMedium(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetIntAfterRelease() { - releasedBuffer().getInt(0); + @Test + void testGetIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getInt(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetIntLEAfterRelease() { - releasedBuffer().getIntLE(0); + @Test + void testGetIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getIntLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedIntAfterRelease() { - releasedBuffer().getUnsignedInt(0); + @Test + void testGetUnsignedIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedInt(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetUnsignedIntLEAfterRelease() { - releasedBuffer().getUnsignedIntLE(0); + @Test + void testGetUnsignedIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getUnsignedIntLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetLongAfterRelease() { - releasedBuffer().getLong(0); + @Test + void testGetLongAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getLong(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetLongLEAfterRelease() { - releasedBuffer().getLongLE(0); + @Test + void testGetLongLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getLongLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetCharAfterRelease() { - releasedBuffer().getChar(0); + @Test + void testGetCharAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getChar(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetFloatAfterRelease() { - releasedBuffer().getFloat(0); + @Test + void testGetFloatAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getFloat(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetFloatLEAfterRelease() { - releasedBuffer().getFloatLE(0); + @Test + void testGetFloatLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getFloatLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetDoubleAfterRelease() { - releasedBuffer().getDouble(0); + @Test + void testGetDoubleAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getDouble(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetDoubleLEAfterRelease() { - releasedBuffer().getDoubleLE(0); + @Test + void testGetDoubleLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().getDoubleLE(0)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease() { + @Test + void testGetBytesAfterRelease() { ByteBuf buffer = buffer(8); try { - releasedBuffer().getBytes(0, buffer); + assertThatThrownBy(() -> releasedBuffer().getBytes(0, buffer)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease2() { + @Test + void testGetBytesAfterRelease2() { ByteBuf buffer = buffer(); try { - releasedBuffer().getBytes(0, buffer, 1); + assertThatThrownBy(() -> releasedBuffer().getBytes(0, buffer, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease3() { + @Test + void testGetBytesAfterRelease3() { ByteBuf buffer = buffer(); try { - releasedBuffer().getBytes(0, buffer, 0, 1); + assertThatThrownBy(() -> releasedBuffer().getBytes(0, buffer, 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease4() { - releasedBuffer().getBytes(0, new byte[8]); + @Test + void testGetBytesAfterRelease4() { + assertThatThrownBy(() -> releasedBuffer().getBytes(0, new byte[8])) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease5() { - releasedBuffer().getBytes(0, new byte[8], 0, 1); + @Test + void testGetBytesAfterRelease5() { + assertThatThrownBy(() -> releasedBuffer().getBytes(0, new byte[8], 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease6() { - releasedBuffer().getBytes(0, ByteBuffer.allocate(8)); + @Test + void testGetBytesAfterRelease6() { + assertThatThrownBy(() -> releasedBuffer().getBytes(0, ByteBuffer.allocate(8))) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease7() throws IOException { - releasedBuffer().getBytes(0, new ByteArrayOutputStream(), 1); + @Test + void testGetBytesAfterRelease7() { + assertThatThrownBy(() -> releasedBuffer().getBytes(0, new ByteArrayOutputStream(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testGetBytesAfterRelease8() throws IOException { - releasedBuffer().getBytes(0, new DevNullGatheringByteChannel(), 1); + @Test + void testGetBytesAfterRelease8() { + assertThatThrownBy(() -> releasedBuffer().getBytes(0, new DevNullGatheringByteChannel(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBooleanAfterRelease() { - releasedBuffer().setBoolean(0, true); + @Test + void testSetBooleanAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setBoolean(0, true)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetByteAfterRelease() { - releasedBuffer().setByte(0, 1); + @Test + void testSetByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setByte(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetShortAfterRelease() { - releasedBuffer().setShort(0, 1); + @Test + void testSetShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setShort(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetShortLEAfterRelease() { - releasedBuffer().setShortLE(0, 1); + @Test + void testSetShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setShortLE(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetMediumAfterRelease() { - releasedBuffer().setMedium(0, 1); + @Test + void testSetMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setMedium(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetMediumLEAfterRelease() { - releasedBuffer().setMediumLE(0, 1); + @Test + void testSetMediumLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setMediumLE(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetIntAfterRelease() { - releasedBuffer().setInt(0, 1); + @Test + void testSetIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setInt(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetIntLEAfterRelease() { - releasedBuffer().setIntLE(0, 1); + @Test + void testSetIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setIntLE(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetLongAfterRelease() { - releasedBuffer().setLong(0, 1); + @Test + void testSetLongAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setLong(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetLongLEAfterRelease() { - releasedBuffer().setLongLE(0, 1); + @Test + void testSetLongLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setLongLE(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetCharAfterRelease() { - releasedBuffer().setChar(0, 1); + @Test + void testSetCharAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setChar(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetFloatAfterRelease() { - releasedBuffer().setFloat(0, 1); + @Test + void testSetFloatAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setFloat(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetDoubleAfterRelease() { - releasedBuffer().setDouble(0, 1); + @Test + void testSetDoubleAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setDouble(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease() { + @Test + void testSetBytesAfterRelease() { ByteBuf buffer = buffer(); try { - releasedBuffer().setBytes(0, buffer); + assertThatThrownBy(() -> releasedBuffer().setBytes(0, buffer)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease2() { + @Test + void testSetBytesAfterRelease2() { ByteBuf buffer = buffer(); try { - releasedBuffer().setBytes(0, buffer, 1); + assertThatThrownBy(() -> releasedBuffer().setBytes(0, buffer, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease3() { + @Test + void testSetBytesAfterRelease3() { ByteBuf buffer = buffer(); try { - releasedBuffer().setBytes(0, buffer, 0, 1); + assertThatThrownBy(() -> releasedBuffer().setBytes(0, buffer, 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testSetUsAsciiCharSequenceAfterRelease() { - testSetCharSequenceAfterRelease0(CharsetUtil.US_ASCII); + @Test + void testSetUsAsciiCharSequenceAfterRelease() { + assertThatThrownBy(() -> testSetCharSequenceAfterRelease0(CharsetUtil.US_ASCII)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetIso88591CharSequenceAfterRelease() { - testSetCharSequenceAfterRelease0(CharsetUtil.ISO_8859_1); + @Test + void testSetIso88591CharSequenceAfterRelease() { + assertThatThrownBy(() -> testSetCharSequenceAfterRelease0(CharsetUtil.ISO_8859_1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetUtf8CharSequenceAfterRelease() { - testSetCharSequenceAfterRelease0(CharsetUtil.UTF_8); + @Test + void testSetUtf8CharSequenceAfterRelease() { + assertThatThrownBy(() -> testSetCharSequenceAfterRelease0(CharsetUtil.UTF_8)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetUtf16CharSequenceAfterRelease() { - testSetCharSequenceAfterRelease0(CharsetUtil.UTF_16); + @Test + void testSetUtf16CharSequenceAfterRelease() { + assertThatThrownBy(() -> testSetCharSequenceAfterRelease0(CharsetUtil.UTF_16)) + .isInstanceOf(IllegalReferenceCountException.class); } private void testSetCharSequenceAfterRelease0(Charset charset) { releasedBuffer().setCharSequence(0, "x", charset); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease4() { - releasedBuffer().setBytes(0, new byte[8]); + @Test + void testSetBytesAfterRelease4() { + assertThatThrownBy(() -> releasedBuffer().setBytes(0, new byte[8])) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease5() { - releasedBuffer().setBytes(0, new byte[8], 0, 1); + @Test + void testSetBytesAfterRelease5() { + assertThatThrownBy(() -> releasedBuffer().setBytes(0, new byte[8], 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease6() { - releasedBuffer().setBytes(0, ByteBuffer.allocate(8)); + @Test + void testSetBytesAfterRelease6() { + assertThatThrownBy(() -> releasedBuffer().setBytes(0, ByteBuffer.allocate(8))) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease7() throws IOException { - releasedBuffer().setBytes(0, new ByteArrayInputStream(new byte[8]), 1); + @Test + void testSetBytesAfterRelease7() { + assertThatThrownBy( + () -> + releasedBuffer() + .setBytes(0, new ByteArrayInputStream(new byte[8]), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetBytesAfterRelease8() throws IOException { - releasedBuffer().setBytes(0, new TestScatteringByteChannel(), 1); + @Test + void testSetBytesAfterRelease8() { + assertThatThrownBy(() -> releasedBuffer().setBytes(0, new TestScatteringByteChannel(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSetZeroAfterRelease() { - releasedBuffer().setZero(0, 1); + @Test + void testSetZeroAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().setZero(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBooleanAfterRelease() { - releasedBuffer().readBoolean(); + @Test + void testReadBooleanAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readBoolean()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadByteAfterRelease() { - releasedBuffer().readByte(); + @Test + void testReadByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readByte()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedByteAfterRelease() { - releasedBuffer().readUnsignedByte(); + @Test + void testReadUnsignedByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedByte()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadShortAfterRelease() { - releasedBuffer().readShort(); + @Test + void testReadShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readShort()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadShortLEAfterRelease() { - releasedBuffer().readShortLE(); + @Test + void testReadShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readShortLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedShortAfterRelease() { - releasedBuffer().readUnsignedShort(); + @Test + void testReadUnsignedShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedShort()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedShortLEAfterRelease() { - releasedBuffer().readUnsignedShortLE(); + @Test + void testReadUnsignedShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedShortLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadMediumAfterRelease() { - releasedBuffer().readMedium(); + @Test + void testReadMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readMedium()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadMediumLEAfterRelease() { - releasedBuffer().readMediumLE(); + @Test + void testReadMediumLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readMediumLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedMediumAfterRelease() { - releasedBuffer().readUnsignedMedium(); + @Test + void testReadUnsignedMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedMedium()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedMediumLEAfterRelease() { - releasedBuffer().readUnsignedMediumLE(); + @Test + void testReadUnsignedMediumLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedMediumLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadIntAfterRelease() { - releasedBuffer().readInt(); + @Test + void testReadIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readInt()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadIntLEAfterRelease() { - releasedBuffer().readIntLE(); + @Test + void testReadIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readIntLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedIntAfterRelease() { - releasedBuffer().readUnsignedInt(); + @Test + void testReadUnsignedIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedInt()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadUnsignedIntLEAfterRelease() { - releasedBuffer().readUnsignedIntLE(); + @Test + void testReadUnsignedIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readUnsignedIntLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadLongAfterRelease() { - releasedBuffer().readLong(); + @Test + void testReadLongAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readLong()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadLongLEAfterRelease() { - releasedBuffer().readLongLE(); + @Test + void testReadLongLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readLongLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadCharAfterRelease() { - releasedBuffer().readChar(); + @Test + void testReadCharAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readChar()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadFloatAfterRelease() { - releasedBuffer().readFloat(); + @Test + void testReadFloatAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readFloat()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadFloatLEAfterRelease() { - releasedBuffer().readFloatLE(); + @Test + void testReadFloatLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readFloatLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadDoubleAfterRelease() { - releasedBuffer().readDouble(); + @Test + void testReadDoubleAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readDouble()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadDoubleLEAfterRelease() { - releasedBuffer().readDoubleLE(); + @Test + void testReadDoubleLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readDoubleLE()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease() { - releasedBuffer().readBytes(1); + @Test + void testReadBytesAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().readBytes(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease2() { + @Test + void testReadBytesAfterRelease2() { ByteBuf buffer = buffer(8); try { - releasedBuffer().readBytes(buffer); + assertThatThrownBy(() -> releasedBuffer().readBytes(buffer)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease3() { + @Test + void testReadBytesAfterRelease3() { ByteBuf buffer = buffer(8); try { - releasedBuffer().readBytes(buffer); + assertThatThrownBy(() -> releasedBuffer().readBytes(buffer)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease4() { + @Test + void testReadBytesAfterRelease4() { ByteBuf buffer = buffer(8); try { - releasedBuffer().readBytes(buffer, 0, 1); + assertThatThrownBy(() -> releasedBuffer().readBytes(buffer, 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease5() { - releasedBuffer().readBytes(new byte[8]); + @Test + void testReadBytesAfterRelease5() { + assertThatThrownBy(() -> releasedBuffer().readBytes(new byte[8])) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease6() { - releasedBuffer().readBytes(new byte[8], 0, 1); + @Test + void testReadBytesAfterRelease6() { + assertThatThrownBy(() -> releasedBuffer().readBytes(new byte[8], 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease7() { - releasedBuffer().readBytes(ByteBuffer.allocate(8)); + @Test + void testReadBytesAfterRelease7() { + assertThatThrownBy(() -> releasedBuffer().readBytes(ByteBuffer.allocate(8))) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease8() throws IOException { - releasedBuffer().readBytes(new ByteArrayOutputStream(), 1); + @Test + void testReadBytesAfterRelease8() { + assertThatThrownBy(() -> releasedBuffer().readBytes(new ByteArrayOutputStream(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease9() throws IOException { - releasedBuffer().readBytes(new ByteArrayOutputStream(), 1); + @Test + void testReadBytesAfterRelease9() { + assertThatThrownBy(() -> releasedBuffer().readBytes(new ByteArrayOutputStream(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testReadBytesAfterRelease10() throws IOException { - releasedBuffer().readBytes(new DevNullGatheringByteChannel(), 1); + @Test + void testReadBytesAfterRelease10() { + assertThatThrownBy(() -> releasedBuffer().readBytes(new DevNullGatheringByteChannel(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBooleanAfterRelease() { - releasedBuffer().writeBoolean(true); + @Test + void testWriteBooleanAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeBoolean(true)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteByteAfterRelease() { - releasedBuffer().writeByte(1); + @Test + void testWriteByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeByte(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteShortAfterRelease() { - releasedBuffer().writeShort(1); + @Test + void testWriteShortAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeShort(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteShortLEAfterRelease() { - releasedBuffer().writeShortLE(1); + @Test + void testWriteShortLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeShortLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteMediumAfterRelease() { - releasedBuffer().writeMedium(1); + @Test + void testWriteMediumAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeMedium(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteMediumLEAfterRelease() { - releasedBuffer().writeMediumLE(1); + @Test + void testWriteMediumLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeMediumLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteIntAfterRelease() { - releasedBuffer().writeInt(1); + @Test + void testWriteIntAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeInt(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteIntLEAfterRelease() { - releasedBuffer().writeIntLE(1); + @Test + void testWriteIntLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeIntLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteLongAfterRelease() { - releasedBuffer().writeLong(1); + @Test + void testWriteLongAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeLong(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteLongLEAfterRelease() { - releasedBuffer().writeLongLE(1); + @Test + void testWriteLongLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeLongLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteCharAfterRelease() { - releasedBuffer().writeChar(1); + @Test + void testWriteCharAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeChar(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteFloatAfterRelease() { - releasedBuffer().writeFloat(1); + @Test + void testWriteFloatAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeFloat(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteFloatLEAfterRelease() { - releasedBuffer().writeFloatLE(1); + @Test + void testWriteFloatLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeFloatLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteDoubleAfterRelease() { - releasedBuffer().writeDouble(1); + @Test + void testWriteDoubleAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeDouble(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteDoubleLEAfterRelease() { - releasedBuffer().writeDoubleLE(1); + @Test + void testWriteDoubleLEAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeDoubleLE(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease() { + @Test + void testWriteBytesAfterRelease() { ByteBuf buffer = buffer(8); try { - releasedBuffer().writeBytes(buffer); + assertThatThrownBy(() -> releasedBuffer().writeBytes(buffer)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease2() { + @Test + void testWriteBytesAfterRelease2() { ByteBuf buffer = copiedBuffer(new byte[8]); try { - releasedBuffer().writeBytes(buffer, 1); + assertThatThrownBy(() -> releasedBuffer().writeBytes(buffer, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease3() { + @Test + void testWriteBytesAfterRelease3() { ByteBuf buffer = buffer(8); try { - releasedBuffer().writeBytes(buffer, 0, 1); + assertThatThrownBy(() -> releasedBuffer().writeBytes(buffer, 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } finally { buffer.release(); } } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease4() { - releasedBuffer().writeBytes(new byte[8]); + @Test + void testWriteBytesAfterRelease4() { + assertThatThrownBy(() -> releasedBuffer().writeBytes(new byte[8])) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease5() { - releasedBuffer().writeBytes(new byte[8], 0, 1); + @Test + void testWriteBytesAfterRelease5() { + assertThatThrownBy(() -> releasedBuffer().writeBytes(new byte[8], 0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease6() { - releasedBuffer().writeBytes(ByteBuffer.allocate(8)); + @Test + void testWriteBytesAfterRelease6() { + assertThatThrownBy(() -> releasedBuffer().writeBytes(ByteBuffer.allocate(8))) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease7() throws IOException { - releasedBuffer().writeBytes(new ByteArrayInputStream(new byte[8]), 1); + @Test + void testWriteBytesAfterRelease7() { + assertThatThrownBy( + () -> releasedBuffer().writeBytes(new ByteArrayInputStream(new byte[8]), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteBytesAfterRelease8() throws IOException { - releasedBuffer().writeBytes(new TestScatteringByteChannel(), 1); + @Test + void testWriteBytesAfterRelease8() { + assertThatThrownBy(() -> releasedBuffer().writeBytes(new TestScatteringByteChannel(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteZeroAfterRelease() throws IOException { - releasedBuffer().writeZero(1); + @Test + void testWriteZeroAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().writeZero(1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteUsAsciiCharSequenceAfterRelease() { - testWriteCharSequenceAfterRelease0(CharsetUtil.US_ASCII); + @Test + void testWriteUsAsciiCharSequenceAfterRelease() { + assertThatThrownBy(() -> testWriteCharSequenceAfterRelease0(CharsetUtil.US_ASCII)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteIso88591CharSequenceAfterRelease() { - testWriteCharSequenceAfterRelease0(CharsetUtil.ISO_8859_1); + @Test + void testWriteIso88591CharSequenceAfterRelease() { + assertThatThrownBy(() -> testWriteCharSequenceAfterRelease0(CharsetUtil.ISO_8859_1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteUtf8CharSequenceAfterRelease() { - testWriteCharSequenceAfterRelease0(CharsetUtil.UTF_8); + @Test + void testWriteUtf8CharSequenceAfterRelease() { + assertThatThrownBy(() -> testWriteCharSequenceAfterRelease0(CharsetUtil.UTF_8)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testWriteUtf16CharSequenceAfterRelease() { - testWriteCharSequenceAfterRelease0(CharsetUtil.UTF_16); + @Test + void testWriteUtf16CharSequenceAfterRelease() { + assertThatThrownBy(() -> testWriteCharSequenceAfterRelease0(CharsetUtil.UTF_16)) + .isInstanceOf(IllegalReferenceCountException.class); } private void testWriteCharSequenceAfterRelease0(Charset charset) { releasedBuffer().writeCharSequence("x", charset); } - @Test(expected = IllegalReferenceCountException.class) - public void testForEachByteAfterRelease() { - releasedBuffer().forEachByte(new TestByteProcessor()); + @Test + void testForEachByteAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().forEachByte(new TestByteProcessor())) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testForEachByteAfterRelease1() { - releasedBuffer().forEachByte(0, 1, new TestByteProcessor()); + @Test + void testForEachByteAfterRelease1() { + assertThatThrownBy(() -> releasedBuffer().forEachByte(0, 1, new TestByteProcessor())) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testForEachByteDescAfterRelease() { - releasedBuffer().forEachByteDesc(new TestByteProcessor()); + @Test + void testForEachByteDescAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().forEachByteDesc(new TestByteProcessor())) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testForEachByteDescAfterRelease1() { - releasedBuffer().forEachByteDesc(0, 1, new TestByteProcessor()); + @Test + void testForEachByteDescAfterRelease1() { + assertThatThrownBy(() -> releasedBuffer().forEachByteDesc(0, 1, new TestByteProcessor())) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testCopyAfterRelease() { - releasedBuffer().copy(); + @Test + void testCopyAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().copy()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testCopyAfterRelease1() { - releasedBuffer().copy(); + @Test + void testCopyAfterRelease1() { + assertThatThrownBy(() -> releasedBuffer().copy()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testNioBufferAfterRelease() { - releasedBuffer().nioBuffer(); + @Test + void testNioBufferAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().nioBuffer()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testNioBufferAfterRelease1() { - releasedBuffer().nioBuffer(0, 1); + @Test + void testNioBufferAfterRelease1() { + assertThatThrownBy(() -> releasedBuffer().nioBuffer(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testInternalNioBufferAfterRelease() { + @Test + void testInternalNioBufferAfterRelease() { ByteBuf releasedBuffer = releasedBuffer(); - releasedBuffer.internalNioBuffer(releasedBuffer.readerIndex(), 1); + assertThatThrownBy(() -> releasedBuffer.internalNioBuffer(releasedBuffer.readerIndex(), 1)) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testNioBuffersAfterRelease() { - releasedBuffer().nioBuffers(); + @Test + void testNioBuffersAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().nioBuffers()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testNioBuffersAfterRelease2() { - releasedBuffer().nioBuffers(0, 1); + @Test + void testNioBuffersAfterRelease2() { + assertThatThrownBy(() -> releasedBuffer().nioBuffers(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } @Test - public void testArrayAfterRelease() { + void testArrayAfterRelease() { ByteBuf buf = releasedBuffer(); if (buf.hasArray()) { - try { - buf.array(); - fail(); - } catch (IllegalReferenceCountException e) { - // expected - } + assertThatThrownBy(buf::array).isInstanceOf(IllegalReferenceCountException.class); } } @Test - public void testMemoryAddressAfterRelease() { + void testMemoryAddressAfterRelease() { ByteBuf buf = releasedBuffer(); if (buf.hasMemoryAddress()) { - try { - buf.memoryAddress(); - fail(); - } catch (IllegalReferenceCountException e) { - // expected - } + assertThatThrownBy(buf::memoryAddress) + .isInstanceOf(IllegalReferenceCountException.class); } } - @Test(expected = IllegalReferenceCountException.class) - public void testSliceAfterRelease() { - releasedBuffer().slice(); + @Test + void testSliceAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().slice()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testSliceAfterRelease2() { - releasedBuffer().slice(0, 1); + @Test + void testSliceAfterRelease2() { + assertThatThrownBy(() -> releasedBuffer().slice(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } private static void assertSliceFailAfterRelease(ByteBuf... bufs) { @@ -3395,25 +3509,20 @@ private static void assertSliceFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - try { - assertEquals(0, buf.refCnt()); - buf.slice(); - fail(); - } catch (IllegalReferenceCountException ignored) { - // as expected - } + assertThat(buf.refCnt()).isEqualTo(0); + assertThatThrownBy(buf::slice).isInstanceOf(IllegalReferenceCountException.class); } } @Test - public void testSliceAfterReleaseRetainedSlice() { + void testSliceAfterReleaseRetainedSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); assertSliceFailAfterRelease(buf, buf2); } @Test - public void testSliceAfterReleaseRetainedSliceDuplicate() { + void testSliceAfterReleaseRetainedSliceDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); ByteBuf buf3 = buf2.duplicate(); @@ -3421,7 +3530,7 @@ public void testSliceAfterReleaseRetainedSliceDuplicate() { } @Test - public void testSliceAfterReleaseRetainedSliceRetainedDuplicate() { + void testSliceAfterReleaseRetainedSliceRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); ByteBuf buf3 = buf2.retainedDuplicate(); @@ -3429,28 +3538,30 @@ public void testSliceAfterReleaseRetainedSliceRetainedDuplicate() { } @Test - public void testSliceAfterReleaseRetainedDuplicate() { + void testSliceAfterReleaseRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); assertSliceFailAfterRelease(buf, buf2); } @Test - public void testSliceAfterReleaseRetainedDuplicateSlice() { + void testSliceAfterReleaseRetainedDuplicateSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); ByteBuf buf3 = buf2.slice(0, 1); assertSliceFailAfterRelease(buf, buf2, buf3); } - @Test(expected = IllegalReferenceCountException.class) - public void testRetainedSliceAfterRelease() { - releasedBuffer().retainedSlice(); + @Test + void testRetainedSliceAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().retainedSlice()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testRetainedSliceAfterRelease2() { - releasedBuffer().retainedSlice(0, 1); + @Test + void testRetainedSliceAfterRelease2() { + assertThatThrownBy(() -> releasedBuffer().retainedSlice(0, 1)) + .isInstanceOf(IllegalReferenceCountException.class); } private static void assertRetainedSliceFailAfterRelease(ByteBuf... bufs) { @@ -3460,25 +3571,21 @@ private static void assertRetainedSliceFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - try { - assertEquals(0, buf.refCnt()); - buf.retainedSlice(); - fail(); - } catch (IllegalReferenceCountException ignored) { - // as expected - } + assertThat(buf.refCnt()).isEqualTo(0); + assertThatThrownBy(buf::retainedSlice) + .isInstanceOf(IllegalReferenceCountException.class); } } @Test - public void testRetainedSliceAfterReleaseRetainedSlice() { + void testRetainedSliceAfterReleaseRetainedSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); assertRetainedSliceFailAfterRelease(buf, buf2); } @Test - public void testRetainedSliceAfterReleaseRetainedSliceDuplicate() { + void testRetainedSliceAfterReleaseRetainedSliceDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); ByteBuf buf3 = buf2.duplicate(); @@ -3486,7 +3593,7 @@ public void testRetainedSliceAfterReleaseRetainedSliceDuplicate() { } @Test - public void testRetainedSliceAfterReleaseRetainedSliceRetainedDuplicate() { + void testRetainedSliceAfterReleaseRetainedSliceRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); ByteBuf buf3 = buf2.retainedDuplicate(); @@ -3494,28 +3601,30 @@ public void testRetainedSliceAfterReleaseRetainedSliceRetainedDuplicate() { } @Test - public void testRetainedSliceAfterReleaseRetainedDuplicate() { + void testRetainedSliceAfterReleaseRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); assertRetainedSliceFailAfterRelease(buf, buf2); } @Test - public void testRetainedSliceAfterReleaseRetainedDuplicateSlice() { + void testRetainedSliceAfterReleaseRetainedDuplicateSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); ByteBuf buf3 = buf2.slice(0, 1); assertRetainedSliceFailAfterRelease(buf, buf2, buf3); } - @Test(expected = IllegalReferenceCountException.class) - public void testDuplicateAfterRelease() { - releasedBuffer().duplicate(); + @Test + void testDuplicateAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().duplicate()) + .isInstanceOf(IllegalReferenceCountException.class); } - @Test(expected = IllegalReferenceCountException.class) - public void testRetainedDuplicateAfterRelease() { - releasedBuffer().retainedDuplicate(); + @Test + void testRetainedDuplicateAfterRelease() { + assertThatThrownBy(() -> releasedBuffer().retainedDuplicate()) + .isInstanceOf(IllegalReferenceCountException.class); } private static void assertDuplicateFailAfterRelease(ByteBuf... bufs) { @@ -3525,18 +3634,13 @@ private static void assertDuplicateFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - try { - assertEquals(0, buf.refCnt()); - buf.duplicate(); - fail(); - } catch (IllegalReferenceCountException ignored) { - // as expected - } + assertThat(buf.refCnt()).isEqualTo(0); + assertThatThrownBy(buf::duplicate).isInstanceOf(IllegalReferenceCountException.class); } } @Test - public void testDuplicateAfterReleaseRetainedSliceDuplicate() { + void testDuplicateAfterReleaseRetainedSliceDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); ByteBuf buf3 = buf2.duplicate(); @@ -3544,14 +3648,14 @@ public void testDuplicateAfterReleaseRetainedSliceDuplicate() { } @Test - public void testDuplicateAfterReleaseRetainedDuplicate() { + void testDuplicateAfterReleaseRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); assertDuplicateFailAfterRelease(buf, buf2); } @Test - public void testDuplicateAfterReleaseRetainedDuplicateSlice() { + void testDuplicateAfterReleaseRetainedDuplicateSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); ByteBuf buf3 = buf2.slice(0, 1); @@ -3565,53 +3669,51 @@ private static void assertRetainedDuplicateFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - try { - assertEquals(0, buf.refCnt()); - buf.retainedDuplicate(); - fail(); - } catch (IllegalReferenceCountException ignored) { - // as expected - } + assertThat(buf.refCnt()).isEqualTo(0); + assertThatThrownBy(buf::retainedDuplicate) + .isInstanceOf(IllegalReferenceCountException.class); } } @Test - public void testRetainedDuplicateAfterReleaseRetainedDuplicate() { + void testRetainedDuplicateAfterReleaseRetainedDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedDuplicate(); assertRetainedDuplicateFailAfterRelease(buf, buf2); } @Test - public void testRetainedDuplicateAfterReleaseDuplicate() { + void testRetainedDuplicateAfterReleaseDuplicate() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.duplicate(); assertRetainedDuplicateFailAfterRelease(buf, buf2); } @Test - public void testRetainedDuplicateAfterReleaseRetainedSlice() { + void testRetainedDuplicateAfterReleaseRetainedSlice() { ByteBuf buf = newBuffer(1); ByteBuf buf2 = buf.retainedSlice(0, 1); assertRetainedDuplicateFailAfterRelease(buf, buf2); } @Test - public void testSliceRelease() { + void testSliceRelease() { ByteBuf buf = newBuffer(8); - assertEquals(1, buf.refCnt()); - assertTrue(buf.slice().release()); - assertEquals(0, buf.refCnt()); + assertThat(buf.refCnt()).isEqualTo(1); + assertThat(buf.slice().release()).isTrue(); + assertThat(buf.refCnt()).isEqualTo(0); } - @Test(expected = IndexOutOfBoundsException.class) - public void testReadSliceOutOfBounds() { - testReadSliceOutOfBounds(false); + @Test + void testReadSliceOutOfBounds() { + assertThatThrownBy(() -> testReadSliceOutOfBounds(false)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testReadRetainedSliceOutOfBounds() { - testReadSliceOutOfBounds(true); + @Test + void testReadRetainedSliceOutOfBounds() { + assertThatThrownBy(() -> testReadSliceOutOfBounds(false)) + .isInstanceOf(IndexOutOfBoundsException.class); } private void testReadSliceOutOfBounds(boolean retainedSlice) { @@ -3623,29 +3725,29 @@ private void testReadSliceOutOfBounds(boolean retainedSlice) { } else { buf.readSlice(51); } - fail(); + fail("Expect exception but not thrown"); } finally { buf.release(); } } @Test - public void testWriteUsAsciiCharSequenceExpand() { + void testWriteUsAsciiCharSequenceExpand() { testWriteCharSequenceExpand(CharsetUtil.US_ASCII); } @Test - public void testWriteUtf8CharSequenceExpand() { + void testWriteUtf8CharSequenceExpand() { testWriteCharSequenceExpand(CharsetUtil.UTF_8); } @Test - public void testWriteIso88591CharSequenceExpand() { + void testWriteIso88591CharSequenceExpand() { testWriteCharSequenceExpand(CharsetUtil.ISO_8859_1); } @Test - public void testWriteUtf16CharSequenceExpand() { + void testWriteUtf16CharSequenceExpand() { testWriteCharSequenceExpand(CharsetUtil.UTF_16); } @@ -3655,30 +3757,34 @@ private void testWriteCharSequenceExpand(Charset charset) { int writerIndex = buf.capacity() - 1; buf.writerIndex(writerIndex); int written = buf.writeCharSequence("AB", charset); - assertEquals(writerIndex, buf.writerIndex() - written); + assertThat(buf.writerIndex() - written).isEqualTo(writerIndex); } finally { buf.release(); } } - @Test(expected = IndexOutOfBoundsException.class) - public void testSetUsAsciiCharSequenceNoExpand() { - testSetCharSequenceNoExpand(CharsetUtil.US_ASCII); + @Test + void testSetUsAsciiCharSequenceNoExpand() { + assertThatThrownBy(() -> testSetCharSequenceNoExpand(CharsetUtil.US_ASCII)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testSetUtf8CharSequenceNoExpand() { - testSetCharSequenceNoExpand(CharsetUtil.UTF_8); + @Test + void testSetUtf8CharSequenceNoExpand() { + assertThatThrownBy(() -> testSetCharSequenceNoExpand(CharsetUtil.UTF_8)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testSetIso88591CharSequenceNoExpand() { - testSetCharSequenceNoExpand(CharsetUtil.ISO_8859_1); + @Test + void testSetIso88591CharSequenceNoExpand() { + assertThatThrownBy(() -> testSetCharSequenceNoExpand(CharsetUtil.ISO_8859_1)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testSetUtf16CharSequenceNoExpand() { - testSetCharSequenceNoExpand(CharsetUtil.UTF_16); + @Test + void testSetUtf16CharSequenceNoExpand() { + assertThatThrownBy(() -> testSetCharSequenceNoExpand(CharsetUtil.UTF_16)) + .isInstanceOf(IndexOutOfBoundsException.class); } private void testSetCharSequenceNoExpand(Charset charset) { @@ -3691,22 +3797,22 @@ private void testSetCharSequenceNoExpand(Charset charset) { } @Test - public void testSetUsAsciiCharSequence() { + void testSetUsAsciiCharSequence() { testSetGetCharSequence(CharsetUtil.US_ASCII); } @Test - public void testSetUtf8CharSequence() { + void testSetUtf8CharSequence() { testSetGetCharSequence(CharsetUtil.UTF_8); } @Test - public void testSetIso88591CharSequence() { + void testSetIso88591CharSequence() { testSetGetCharSequence(CharsetUtil.ISO_8859_1); } @Test - public void testSetUtf16CharSequence() { + void testSetUtf16CharSequence() { testSetGetCharSequence(CharsetUtil.UTF_16); } @@ -3726,27 +3832,28 @@ private void testSetGetCharSequence(Charset charset) { CharBuffer sequence = CharsetUtil.US_ASCII.equals(charset) ? ASCII_CHARS : EXTENDED_ASCII_CHARS; int bytes = buf.setCharSequence(1, sequence, charset); - assertEquals(sequence, CharBuffer.wrap(buf.getCharSequence(1, bytes, charset))); + CharSequence actual = CharBuffer.wrap(buf.getCharSequence(1, bytes, charset)); + assertThat(actual).isEqualTo(sequence); buf.release(); } @Test - public void testWriteReadUsAsciiCharSequence() { + void testWriteReadUsAsciiCharSequence() { testWriteReadCharSequence(CharsetUtil.US_ASCII); } @Test - public void testWriteReadUtf8CharSequence() { + void testWriteReadUtf8CharSequence() { testWriteReadCharSequence(CharsetUtil.UTF_8); } @Test - public void testWriteReadIso88591CharSequence() { + void testWriteReadIso88591CharSequence() { testWriteReadCharSequence(CharsetUtil.ISO_8859_1); } @Test - public void testWriteReadUtf16CharSequence() { + void testWriteReadUtf16CharSequence() { testWriteReadCharSequence(CharsetUtil.UTF_16); } @@ -3757,52 +3864,61 @@ private void testWriteReadCharSequence(Charset charset) { buf.writerIndex(1); int bytes = buf.writeCharSequence(sequence, charset); buf.readerIndex(1); - assertEquals(sequence, CharBuffer.wrap(buf.readCharSequence(bytes, charset))); + CharSequence actual = CharBuffer.wrap(buf.readCharSequence(bytes, charset)); + assertThat(actual).isEqualTo(sequence); buf.release(); } - @Test(expected = IndexOutOfBoundsException.class) - public void testRetainedSliceIndexOutOfBounds() { - testSliceOutOfBounds(true, true, true); + @Test + void testRetainedSliceIndexOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(true, true, true)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testRetainedSliceLengthOutOfBounds() { - testSliceOutOfBounds(true, true, false); + @Test + void testRetainedSliceLengthOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(true, true, false)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testMixedSliceAIndexOutOfBounds() { - testSliceOutOfBounds(true, false, true); + @Test + void testMixedSliceAIndexOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(true, false, true)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testMixedSliceALengthOutOfBounds() { - testSliceOutOfBounds(true, false, false); + @Test + void testMixedSliceALengthOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(true, false, false)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testMixedSliceBIndexOutOfBounds() { - testSliceOutOfBounds(false, true, true); + @Test + void testMixedSliceBIndexOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(false, true, true)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testMixedSliceBLengthOutOfBounds() { - testSliceOutOfBounds(false, true, false); + @Test + void testMixedSliceBLengthOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(false, true, false)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testSliceIndexOutOfBounds() { - testSliceOutOfBounds(false, false, true); + @Test + void testSliceIndexOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(false, false, true)) + .isInstanceOf(IndexOutOfBoundsException.class); } - @Test(expected = IndexOutOfBoundsException.class) - public void testSliceLengthOutOfBounds() { - testSliceOutOfBounds(false, false, false); + @Test + void testSliceLengthOutOfBounds() { + assertThatThrownBy(() -> testSliceOutOfBounds(false, false, false)) + .isInstanceOf(IndexOutOfBoundsException.class); } @Test - public void testRetainedSliceAndRetainedDuplicateContentIsExpected() { + void testRetainedSliceAndRetainedDuplicateContentIsExpected() { ByteBuf buf = newBuffer(8).resetWriterIndex(); ByteBuf expected1 = newBuffer(6).resetWriterIndex(); ByteBuf expected2 = newBuffer(5).resetWriterIndex(); @@ -3815,8 +3931,8 @@ public void testRetainedSliceAndRetainedDuplicateContentIsExpected() { expected4.writeBytes(new byte[] {5, 6, 7}); ByteBuf slice1 = buf.retainedSlice(buf.readerIndex() + 1, 6); - assertEquals(0, slice1.compareTo(expected1)); - assertEquals(0, slice1.compareTo(buf.slice(buf.readerIndex() + 1, 6))); + assertThat(slice1.compareTo(expected1)).isEqualTo(0); + assertThat(slice1.compareTo(buf.slice(buf.readerIndex() + 1, 6))).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -3824,49 +3940,49 @@ public void testRetainedSliceAndRetainedDuplicateContentIsExpected() { slice1.readByte(); ByteBuf dup1 = slice1.retainedDuplicate(); - assertEquals(0, dup1.compareTo(expected2)); - assertEquals(0, dup1.compareTo(slice1.duplicate())); + assertThat(dup1.compareTo(expected2)).isEqualTo(0); + assertThat(dup1.compareTo(slice1.duplicate())).isEqualTo(0); // Advance the reader index on dup1. dup1.readByte(); ByteBuf dup2 = dup1.duplicate(); - assertEquals(0, dup2.compareTo(expected3)); + assertThat(dup2.compareTo(expected3)).isEqualTo(0); // Advance the reader index on dup2. dup2.readByte(); ByteBuf slice2 = dup2.retainedSlice(dup2.readerIndex(), 3); - assertEquals(0, slice2.compareTo(expected4)); - assertEquals(0, slice2.compareTo(dup2.slice(dup2.readerIndex(), 3))); + assertThat(slice2.compareTo(expected4)).isEqualTo(0); + assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex(), 3))).isEqualTo(0); // Cleanup the expected buffers used for testing. - assertTrue(expected1.release()); - assertTrue(expected2.release()); - assertTrue(expected3.release()); - assertTrue(expected4.release()); + assertThat(expected1.release()).isTrue(); + assertThat(expected2.release()).isTrue(); + assertThat(expected3.release()).isTrue(); + assertThat(expected4.release()).isTrue(); slice2.release(); dup2.release(); - assertEquals(slice2.refCnt(), dup2.refCnt()); - assertEquals(dup2.refCnt(), dup1.refCnt()); + assertThat(dup2.refCnt()).isEqualTo(slice2.refCnt()); + assertThat(dup1.refCnt()).isEqualTo(dup2.refCnt()); // The handler is now done with the original slice - assertTrue(slice1.release()); + assertThat(slice1.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, slice1.refCnt()); - assertEquals(0, slice2.refCnt()); - assertEquals(0, dup1.refCnt()); - assertEquals(0, dup2.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(slice1.refCnt()).isEqualTo(0); + assertThat(slice2.refCnt()).isEqualTo(0); + assertThat(dup1.refCnt()).isEqualTo(0); + assertThat(dup2.refCnt()).isEqualTo(0); } @Test - public void testRetainedDuplicateAndRetainedSliceContentIsExpected() { + void testRetainedDuplicateAndRetainedSliceContentIsExpected() { ByteBuf buf = newBuffer(8).resetWriterIndex(); ByteBuf expected1 = newBuffer(6).resetWriterIndex(); ByteBuf expected2 = newBuffer(5).resetWriterIndex(); @@ -3877,8 +3993,8 @@ public void testRetainedDuplicateAndRetainedSliceContentIsExpected() { expected3.writeBytes(new byte[] {5, 6, 7}); ByteBuf dup1 = buf.retainedDuplicate(); - assertEquals(0, dup1.compareTo(buf)); - assertEquals(0, dup1.compareTo(buf.slice())); + assertThat(dup1.compareTo(buf)).isEqualTo(0); + assertThat(dup1.compareTo(buf.slice())).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -3886,243 +4002,245 @@ public void testRetainedDuplicateAndRetainedSliceContentIsExpected() { dup1.readByte(); ByteBuf slice1 = dup1.retainedSlice(dup1.readerIndex(), 6); - assertEquals(0, slice1.compareTo(expected1)); - assertEquals(0, slice1.compareTo(slice1.duplicate())); + assertThat(slice1.compareTo(expected1)).isEqualTo(0); + assertThat(slice1.compareTo(slice1.duplicate())).isEqualTo(0); // Advance the reader index on slice1. slice1.readByte(); ByteBuf dup2 = slice1.duplicate(); - assertEquals(0, dup2.compareTo(slice1)); + assertThat(dup2.compareTo(slice1)).isEqualTo(0); // Advance the reader index on dup2. dup2.readByte(); ByteBuf slice2 = dup2.retainedSlice(dup2.readerIndex() + 1, 3); - assertEquals(0, slice2.compareTo(expected3)); - assertEquals(0, slice2.compareTo(dup2.slice(dup2.readerIndex() + 1, 3))); + assertThat(slice2.compareTo(expected3)).isEqualTo(0); + assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex() + 1, 3))).isEqualTo(0); // Cleanup the expected buffers used for testing. - assertTrue(expected1.release()); - assertTrue(expected2.release()); - assertTrue(expected3.release()); + assertThat(expected1.release()).isTrue(); + assertThat(expected2.release()).isTrue(); + assertThat(expected3.release()).isTrue(); slice2.release(); slice1.release(); - assertEquals(slice2.refCnt(), dup2.refCnt()); - assertEquals(dup2.refCnt(), slice1.refCnt()); + assertThat(dup2.refCnt()).isEqualTo(slice2.refCnt()); + assertThat(slice1.refCnt()).isEqualTo(dup2.refCnt()); // The handler is now done with the original slice - assertTrue(dup1.release()); + assertThat(dup1.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, slice1.refCnt()); - assertEquals(0, slice2.refCnt()); - assertEquals(0, dup1.refCnt()); - assertEquals(0, dup2.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(slice1.refCnt()).isEqualTo(0); + assertThat(slice2.refCnt()).isEqualTo(0); + assertThat(dup1.refCnt()).isEqualTo(0); + assertThat(dup2.refCnt()).isEqualTo(0); } @Test - public void testRetainedSliceContents() { + void testRetainedSliceContents() { testSliceContents(true); } @Test - public void testMultipleLevelRetainedSlice1() { + void testMultipleLevelRetainedSlice1() { testMultipleLevelRetainedSliceWithNonRetained(true, true); } @Test - public void testMultipleLevelRetainedSlice2() { + void testMultipleLevelRetainedSlice2() { testMultipleLevelRetainedSliceWithNonRetained(true, false); } @Test - public void testMultipleLevelRetainedSlice3() { + void testMultipleLevelRetainedSlice3() { testMultipleLevelRetainedSliceWithNonRetained(false, true); } @Test - public void testMultipleLevelRetainedSlice4() { + void testMultipleLevelRetainedSlice4() { testMultipleLevelRetainedSliceWithNonRetained(false, false); } @Test - public void testRetainedSliceReleaseOriginal1() { + void testRetainedSliceReleaseOriginal1() { testSliceReleaseOriginal(true, true); } @Test - public void testRetainedSliceReleaseOriginal2() { + void testRetainedSliceReleaseOriginal2() { testSliceReleaseOriginal(true, false); } @Test - public void testRetainedSliceReleaseOriginal3() { + void testRetainedSliceReleaseOriginal3() { testSliceReleaseOriginal(false, true); } @Test - public void testRetainedSliceReleaseOriginal4() { + void testRetainedSliceReleaseOriginal4() { testSliceReleaseOriginal(false, false); } @Test - public void testRetainedDuplicateReleaseOriginal1() { + void testRetainedDuplicateReleaseOriginal1() { testDuplicateReleaseOriginal(true, true); } @Test - public void testRetainedDuplicateReleaseOriginal2() { + void testRetainedDuplicateReleaseOriginal2() { testDuplicateReleaseOriginal(true, false); } @Test - public void testRetainedDuplicateReleaseOriginal3() { + void testRetainedDuplicateReleaseOriginal3() { testDuplicateReleaseOriginal(false, true); } @Test - public void testRetainedDuplicateReleaseOriginal4() { + void testRetainedDuplicateReleaseOriginal4() { testDuplicateReleaseOriginal(false, false); } @Test - public void testMultipleRetainedSliceReleaseOriginal1() { + void testMultipleRetainedSliceReleaseOriginal1() { testMultipleRetainedSliceReleaseOriginal(true, true); } @Test - public void testMultipleRetainedSliceReleaseOriginal2() { + void testMultipleRetainedSliceReleaseOriginal2() { testMultipleRetainedSliceReleaseOriginal(true, false); } @Test - public void testMultipleRetainedSliceReleaseOriginal3() { + void testMultipleRetainedSliceReleaseOriginal3() { testMultipleRetainedSliceReleaseOriginal(false, true); } @Test - public void testMultipleRetainedSliceReleaseOriginal4() { + void testMultipleRetainedSliceReleaseOriginal4() { testMultipleRetainedSliceReleaseOriginal(false, false); } @Test - public void testMultipleRetainedDuplicateReleaseOriginal1() { + void testMultipleRetainedDuplicateReleaseOriginal1() { testMultipleRetainedDuplicateReleaseOriginal(true, true); } @Test - public void testMultipleRetainedDuplicateReleaseOriginal2() { + void testMultipleRetainedDuplicateReleaseOriginal2() { testMultipleRetainedDuplicateReleaseOriginal(true, false); } @Test - public void testMultipleRetainedDuplicateReleaseOriginal3() { + void testMultipleRetainedDuplicateReleaseOriginal3() { testMultipleRetainedDuplicateReleaseOriginal(false, true); } @Test - public void testMultipleRetainedDuplicateReleaseOriginal4() { + void testMultipleRetainedDuplicateReleaseOriginal4() { testMultipleRetainedDuplicateReleaseOriginal(false, false); } @Test - public void testSliceContents() { + void testSliceContents() { testSliceContents(false); } @Test - public void testRetainedDuplicateContents() { + void testRetainedDuplicateContents() { testDuplicateContents(true); } @Test - public void testDuplicateContents() { + void testDuplicateContents() { testDuplicateContents(false); } @Test - public void testDuplicateCapacityChange() { + void testDuplicateCapacityChange() { testDuplicateCapacityChange(false); } @Test - public void testRetainedDuplicateCapacityChange() { + void testRetainedDuplicateCapacityChange() { testDuplicateCapacityChange(true); } - @Test(expected = UnsupportedOperationException.class) - public void testSliceCapacityChange() { - testSliceCapacityChange(false); + @Test + void testSliceCapacityChange() { + assertThatThrownBy(() -> testSliceCapacityChange(false)) + .isInstanceOf(UnsupportedOperationException.class); } - @Test(expected = UnsupportedOperationException.class) - public void testRetainedSliceCapacityChange() { - testSliceCapacityChange(true); + @Test + void testRetainedSliceCapacityChange() { + assertThatThrownBy(() -> testSliceCapacityChange(true)) + .isInstanceOf(UnsupportedOperationException.class); } @Test - public void testRetainedSliceUnreleasable1() { + void testRetainedSliceUnreleasable1() { testRetainedSliceUnreleasable(true, true); } @Test - public void testRetainedSliceUnreleasable2() { + void testRetainedSliceUnreleasable2() { testRetainedSliceUnreleasable(true, false); } @Test - public void testRetainedSliceUnreleasable3() { + void testRetainedSliceUnreleasable3() { testRetainedSliceUnreleasable(false, true); } @Test - public void testRetainedSliceUnreleasable4() { + void testRetainedSliceUnreleasable4() { testRetainedSliceUnreleasable(false, false); } @Test - public void testReadRetainedSliceUnreleasable1() { + void testReadRetainedSliceUnreleasable1() { testReadRetainedSliceUnreleasable(true, true); } @Test - public void testReadRetainedSliceUnreleasable2() { + void testReadRetainedSliceUnreleasable2() { testReadRetainedSliceUnreleasable(true, false); } @Test - public void testReadRetainedSliceUnreleasable3() { + void testReadRetainedSliceUnreleasable3() { testReadRetainedSliceUnreleasable(false, true); } @Test - public void testReadRetainedSliceUnreleasable4() { + void testReadRetainedSliceUnreleasable4() { testReadRetainedSliceUnreleasable(false, false); } @Test - public void testRetainedDuplicateUnreleasable1() { + void testRetainedDuplicateUnreleasable1() { testRetainedDuplicateUnreleasable(true, true); } @Test - public void testRetainedDuplicateUnreleasable2() { + void testRetainedDuplicateUnreleasable2() { testRetainedDuplicateUnreleasable(true, false); } @Test - public void testRetainedDuplicateUnreleasable3() { + void testRetainedDuplicateUnreleasable3() { testRetainedDuplicateUnreleasable(false, true); } @Test - public void testRetainedDuplicateUnreleasable4() { + void testRetainedDuplicateUnreleasable4() { testRetainedDuplicateUnreleasable(false, false); } @@ -4132,12 +4250,12 @@ private void testRetainedSliceUnreleasable( ByteBuf buf1 = initRetainedSlice ? buf.retainedSlice() : buf.slice().retain(); ByteBuf buf2 = unreleasableBuffer(buf1); ByteBuf buf3 = finalRetainedSlice ? buf2.retainedSlice() : buf2.slice().retain(); - assertFalse(buf3.release()); - assertFalse(buf2.release()); + assertThat(buf3.release()).isFalse(); + assertThat(buf2.release()).isFalse(); buf1.release(); - assertTrue(buf.release()); - assertEquals(0, buf1.refCnt()); - assertEquals(0, buf.refCnt()); + assertThat(buf.release()).isTrue(); + assertThat(buf1.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isEqualTo(0); } private void testReadRetainedSliceUnreleasable( @@ -4149,12 +4267,12 @@ private void testReadRetainedSliceUnreleasable( finalRetainedSlice ? buf2.readRetainedSlice(buf2.readableBytes()) : buf2.readSlice(buf2.readableBytes()).retain(); - assertFalse(buf3.release()); - assertFalse(buf2.release()); + assertThat(buf3.release()).isFalse(); + assertThat(buf2.release()).isFalse(); buf1.release(); - assertTrue(buf.release()); - assertEquals(0, buf1.refCnt()); - assertEquals(0, buf.refCnt()); + assertThat(buf.release()).isTrue(); + assertThat(buf1.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isEqualTo(0); } private void testRetainedDuplicateUnreleasable( @@ -4164,12 +4282,12 @@ private void testRetainedDuplicateUnreleasable( ByteBuf buf2 = unreleasableBuffer(buf1); ByteBuf buf3 = finalRetainedDuplicate ? buf2.retainedDuplicate() : buf2.duplicate().retain(); - assertFalse(buf3.release()); - assertFalse(buf2.release()); + assertThat(buf3.release()).isFalse(); + assertThat(buf2.release()).isFalse(); buf1.release(); - assertTrue(buf.release()); - assertEquals(0, buf1.refCnt()); - assertEquals(0, buf.refCnt()); + assertThat(buf.release()).isTrue(); + assertThat(buf1.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isEqualTo(0); } private void testDuplicateCapacityChange(boolean retainedDuplicate) { @@ -4177,9 +4295,9 @@ private void testDuplicateCapacityChange(boolean retainedDuplicate) { ByteBuf dup = retainedDuplicate ? buf.retainedDuplicate() : buf.duplicate(); try { dup.capacity(10); - assertEquals(buf.capacity(), dup.capacity()); + assertThat(dup.capacity()).isEqualTo(buf.capacity()); dup.capacity(5); - assertEquals(buf.capacity(), dup.capacity()); + assertThat(dup.capacity()).isEqualTo(buf.capacity()); } finally { if (retainedDuplicate) { dup.release(); @@ -4212,8 +4330,8 @@ private void testSliceOutOfBounds( ? buf.retainedSlice(buf.readerIndex() + 1, 2) : buf.slice(buf.readerIndex() + 1, 2); try { - assertEquals(2, slice.capacity()); - assertEquals(2, slice.maxCapacity()); + assertThat(slice.capacity()).isEqualTo(2); + assertThat(slice.maxCapacity()).isEqualTo(2); final int index = indexOutOfBounds ? 3 : 0; final int length = indexOutOfBounds ? 0 : 3; if (finalRetainedSlice) { @@ -4240,12 +4358,12 @@ private void testSliceContents(boolean retainedSlice) { ? buf.retainedSlice(buf.readerIndex() + 3, 3) : buf.slice(buf.readerIndex() + 3, 3); try { - assertEquals(0, slice.compareTo(expected)); - assertEquals(0, slice.compareTo(slice.duplicate())); + assertThat(slice.compareTo(expected)).isEqualTo(0); + assertThat(slice.compareTo(slice.duplicate())).isEqualTo(0); ByteBuf b = slice.retainedDuplicate(); - assertEquals(0, slice.compareTo(b)); + assertThat(slice.compareTo(b)).isEqualTo(0); b.release(); - assertEquals(0, slice.compareTo(slice.slice(0, slice.capacity()))); + assertThat(slice.compareTo(slice.slice(0, slice.capacity()))).isEqualTo(0); } finally { if (retainedSlice) { slice.release(); @@ -4266,7 +4384,7 @@ private void testSliceReleaseOriginal(boolean retainedSlice1, boolean retainedSl retainedSlice1 ? buf.retainedSlice(buf.readerIndex() + 5, 3) : buf.slice(buf.readerIndex() + 5, 3).retain(); - assertEquals(0, slice1.compareTo(expected1)); + assertThat(slice1.compareTo(expected1)).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -4274,24 +4392,24 @@ private void testSliceReleaseOriginal(boolean retainedSlice1, boolean retainedSl retainedSlice2 ? slice1.retainedSlice(slice1.readerIndex() + 1, 2) : slice1.slice(slice1.readerIndex() + 1, 2).retain(); - assertEquals(0, slice2.compareTo(expected2)); + assertThat(slice2.compareTo(expected2)).isEqualTo(0); // Cleanup the expected buffers used for testing. - assertTrue(expected1.release()); - assertTrue(expected2.release()); + assertThat(expected1.release()).isTrue(); + assertThat(expected2.release()).isTrue(); // The handler created a slice of the slice and is now done with it. slice2.release(); // The handler is now done with the original slice - assertTrue(slice1.release()); + assertThat(slice1.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, slice1.refCnt()); - assertEquals(0, slice2.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(slice1.refCnt()).isEqualTo(0); + assertThat(slice2.refCnt()).isEqualTo(0); } private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boolean doSlice2) { @@ -4309,60 +4427,60 @@ private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boo expected4DupSlice.writeBytes(new byte[] {4}); ByteBuf slice1 = buf.retainedSlice(buf.readerIndex() + 1, 6); - assertEquals(0, slice1.compareTo(expected1)); + assertThat(slice1.compareTo(expected1)).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf slice2 = slice1.retainedSlice(slice1.readerIndex() + 1, 4); - assertEquals(0, slice2.compareTo(expected2)); - assertEquals(0, slice2.compareTo(slice2.duplicate())); - assertEquals(0, slice2.compareTo(slice2.slice())); + assertThat(slice2.compareTo(expected2)).isEqualTo(0); + assertThat(slice2.compareTo(slice2.duplicate())).isEqualTo(0); + assertThat(slice2.compareTo(slice2.slice())).isEqualTo(0); ByteBuf tmpBuf = slice2.retainedDuplicate(); - assertEquals(0, slice2.compareTo(tmpBuf)); + assertThat(slice2.compareTo(tmpBuf)).isEqualTo(0); tmpBuf.release(); tmpBuf = slice2.retainedSlice(); - assertEquals(0, slice2.compareTo(tmpBuf)); + assertThat(slice2.compareTo(tmpBuf)).isEqualTo(0); tmpBuf.release(); ByteBuf slice3 = doSlice1 ? slice2.slice(slice2.readerIndex() + 1, 2) : slice2.duplicate(); if (doSlice1) { - assertEquals(0, slice3.compareTo(expected3)); + assertThat(slice3.compareTo(expected3)).isEqualTo(0); } else { - assertEquals(0, slice3.compareTo(expected2)); + assertThat(slice3.compareTo(expected2)).isEqualTo(0); } ByteBuf slice4 = doSlice2 ? slice3.slice(slice3.readerIndex() + 1, 1) : slice3.duplicate(); if (doSlice1 && doSlice2) { - assertEquals(0, slice4.compareTo(expected4SliceSlice)); + assertThat(slice4.compareTo(expected4SliceSlice)).isEqualTo(0); } else if (doSlice2) { - assertEquals(0, slice4.compareTo(expected4DupSlice)); + assertThat(slice4.compareTo(expected4DupSlice)).isEqualTo(0); } else { - assertEquals(0, slice3.compareTo(slice4)); + assertThat(slice3.compareTo(slice4)).isEqualTo(0); } // Cleanup the expected buffers used for testing. - assertTrue(expected1.release()); - assertTrue(expected2.release()); - assertTrue(expected3.release()); - assertTrue(expected4SliceSlice.release()); - assertTrue(expected4DupSlice.release()); + assertThat(expected1.release()).isTrue(); + assertThat(expected2.release()).isTrue(); + assertThat(expected3.release()).isTrue(); + assertThat(expected4SliceSlice.release()).isTrue(); + assertThat(expected4DupSlice.release()).isTrue(); // Slice 4, 3, and 2 should effectively "share" a reference count. slice4.release(); - assertEquals(slice3.refCnt(), slice2.refCnt()); - assertEquals(slice3.refCnt(), slice4.refCnt()); + assertThat(slice2.refCnt()).isEqualTo(slice3.refCnt()); + assertThat(slice4.refCnt()).isEqualTo(slice3.refCnt()); // Slice 1 should also release the original underlying buffer without throwing exceptions - assertTrue(slice1.release()); + assertThat(slice1.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, slice1.refCnt()); - assertEquals(0, slice2.refCnt()); - assertEquals(0, slice3.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(slice1.refCnt()).isEqualTo(0); + assertThat(slice2.refCnt()).isEqualTo(0); + assertThat(slice3.refCnt()).isEqualTo(0); } private void testDuplicateReleaseOriginal( @@ -4372,28 +4490,28 @@ private void testDuplicateReleaseOriginal( buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); expected.writeBytes(buf, buf.readerIndex(), buf.readableBytes()); ByteBuf dup1 = retainedDuplicate1 ? buf.retainedDuplicate() : buf.duplicate().retain(); - assertEquals(0, dup1.compareTo(expected)); + assertThat(dup1.compareTo(expected)).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf dup2 = retainedDuplicate2 ? dup1.retainedDuplicate() : dup1.duplicate().retain(); - assertEquals(0, dup2.compareTo(expected)); + assertThat(dup2.compareTo(expected)).isEqualTo(0); // Cleanup the expected buffers used for testing. - assertTrue(expected.release()); + assertThat(expected.release()).isTrue(); // The handler created a slice of the slice and is now done with it. dup2.release(); // The handler is now done with the original slice - assertTrue(dup1.release()); + assertThat(dup1.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, dup1.refCnt()); - assertEquals(0, dup2.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(dup1.refCnt()).isEqualTo(0); + assertThat(dup2.refCnt()).isEqualTo(0); } private void testMultipleRetainedSliceReleaseOriginal( @@ -4410,7 +4528,7 @@ private void testMultipleRetainedSliceReleaseOriginal( retainedSlice1 ? buf.retainedSlice(buf.readerIndex() + 5, 3) : buf.slice(buf.readerIndex() + 5, 3).retain(); - assertEquals(0, slice1.compareTo(expected1)); + assertThat(slice1.compareTo(expected1)).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -4418,32 +4536,32 @@ private void testMultipleRetainedSliceReleaseOriginal( retainedSlice2 ? slice1.retainedSlice(slice1.readerIndex() + 1, 2) : slice1.slice(slice1.readerIndex() + 1, 2).retain(); - assertEquals(0, slice2.compareTo(expected2)); + assertThat(slice2.compareTo(expected2)).isEqualTo(0); // The handler created a slice of the slice and is now done with it. slice2.release(); ByteBuf slice3 = slice1.retainedSlice(slice1.readerIndex(), 2); - assertEquals(0, slice3.compareTo(expected3)); + assertThat(slice3.compareTo(expected3)).isEqualTo(0); // The handler created another slice of the slice and is now done with it. slice3.release(); // The handler is now done with the original slice - assertTrue(slice1.release()); + assertThat(slice1.release()).isTrue(); // Cleanup the expected buffers used for testing. - assertTrue(expected1.release()); - assertTrue(expected2.release()); - assertTrue(expected3.release()); + assertThat(expected1.release()).isTrue(); + assertThat(expected2.release()).isTrue(); + assertThat(expected3.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, slice1.refCnt()); - assertEquals(0, slice2.refCnt()); - assertEquals(0, slice3.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(slice1.refCnt()).isEqualTo(0); + assertThat(slice2.refCnt()).isEqualTo(0); + assertThat(slice3.refCnt()).isEqualTo(0); } private void testMultipleRetainedDuplicateReleaseOriginal( @@ -4453,44 +4571,44 @@ private void testMultipleRetainedDuplicateReleaseOriginal( buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); expected.writeBytes(buf, buf.readerIndex(), buf.readableBytes()); ByteBuf dup1 = retainedDuplicate1 ? buf.retainedDuplicate() : buf.duplicate().retain(); - assertEquals(0, dup1.compareTo(expected)); + assertThat(dup1.compareTo(expected)).isEqualTo(0); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf dup2 = retainedDuplicate2 ? dup1.retainedDuplicate() : dup1.duplicate().retain(); - assertEquals(0, dup2.compareTo(expected)); - assertEquals(0, dup2.compareTo(dup2.duplicate())); - assertEquals(0, dup2.compareTo(dup2.slice())); + assertThat(dup2.compareTo(expected)).isEqualTo(0); + assertThat(dup2.compareTo(dup2.duplicate())).isEqualTo(0); + assertThat(dup2.compareTo(dup2.slice())).isEqualTo(0); ByteBuf tmpBuf = dup2.retainedDuplicate(); - assertEquals(0, dup2.compareTo(tmpBuf)); + assertThat(dup2.compareTo(tmpBuf)).isEqualTo(0); tmpBuf.release(); tmpBuf = dup2.retainedSlice(); - assertEquals(0, dup2.compareTo(tmpBuf)); + assertThat(dup2.compareTo(tmpBuf)).isEqualTo(0); tmpBuf.release(); // The handler created a slice of the slice and is now done with it. dup2.release(); ByteBuf dup3 = dup1.retainedDuplicate(); - assertEquals(0, dup3.compareTo(expected)); + assertThat(dup3.compareTo(expected)).isEqualTo(0); // The handler created another slice of the slice and is now done with it. dup3.release(); // The handler is now done with the original slice - assertTrue(dup1.release()); + assertThat(dup1.release()).isTrue(); // Cleanup the expected buffers used for testing. - assertTrue(expected.release()); + assertThat(expected.release()).isTrue(); // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertEquals(0, buf.refCnt()); - assertEquals(0, dup1.refCnt()); - assertEquals(0, dup2.refCnt()); - assertEquals(0, dup3.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); + assertThat(dup1.refCnt()).isEqualTo(0); + assertThat(dup2.refCnt()).isEqualTo(0); + assertThat(dup3.refCnt()).isEqualTo(0); } private void testDuplicateContents(boolean retainedDuplicate) { @@ -4498,12 +4616,13 @@ private void testDuplicateContents(boolean retainedDuplicate) { buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); ByteBuf dup = retainedDuplicate ? buf.retainedDuplicate() : buf.duplicate(); try { - assertEquals(0, dup.compareTo(buf)); - assertEquals(0, dup.compareTo(dup.duplicate())); + assertThat(dup.compareTo(buf)).isEqualTo(0); + assertThat(dup.compareTo(dup.duplicate())).isEqualTo(0); ByteBuf b = dup.retainedDuplicate(); - assertEquals(0, dup.compareTo(b)); + assertThat(dup.compareTo(b)).isEqualTo(0); b.release(); - assertEquals(0, dup.compareTo(dup.slice(dup.readerIndex(), dup.readableBytes()))); + assertThat(dup.compareTo(dup.slice(dup.readerIndex(), dup.readableBytes()))) + .isEqualTo(0); } finally { if (retainedDuplicate) { dup.release(); @@ -4513,45 +4632,45 @@ private void testDuplicateContents(boolean retainedDuplicate) { } @Test - public void testDuplicateRelease() { + void testDuplicateRelease() { ByteBuf buf = newBuffer(8); - assertEquals(1, buf.refCnt()); - assertTrue(buf.duplicate().release()); - assertEquals(0, buf.refCnt()); + assertThat(buf.refCnt()).isEqualTo(1); + assertThat(buf.duplicate().release()).isTrue(); + assertThat(buf.refCnt()).isEqualTo(0); } // Test-case trying to reproduce: // https://github.com/netty/netty/issues/2843 @Test - public void testRefCnt() throws Exception { + void testRefCnt() throws Exception { testRefCnt0(false); } // Test-case trying to reproduce: // https://github.com/netty/netty/issues/2843 @Test - public void testRefCnt2() throws Exception { + void testRefCnt2() throws Exception { testRefCnt0(true); } @Test - public void testEmptyNioBuffers() throws Exception { + void testEmptyNioBuffers() throws Exception { ByteBuf buffer = newBuffer(8); buffer.clear(); - assertFalse(buffer.isReadable()); + assertThat(buffer.isReadable()).isFalse(); ByteBuffer[] nioBuffers = buffer.nioBuffers(); - assertEquals(1, nioBuffers.length); - assertFalse(nioBuffers[0].hasRemaining()); + assertThat(nioBuffers.length).isEqualTo(1); + assertThat(nioBuffers[0].hasRemaining()).isFalse(); buffer.release(); } @Test - public void testGetReadOnlyDirectDst() { + void testGetReadOnlyDirectDst() { testGetReadOnlyDst(true); } @Test - public void testGetReadOnlyHeapDst() { + void testGetReadOnlyHeapDst() { testGetReadOnlyDst(false); } @@ -4566,18 +4685,14 @@ private void testGetReadOnlyDst(boolean direct) { ? ByteBuffer.allocateDirect(bytes.length) : ByteBuffer.allocate(bytes.length); ByteBuffer readOnlyDst = dst.asReadOnlyBuffer(); - try { - buffer.getBytes(0, readOnlyDst); - fail(); - } catch (ReadOnlyBufferException e) { - // expected - } - assertEquals(0, readOnlyDst.position()); + assertThatThrownBy(() -> buffer.getBytes(0, readOnlyDst)) + .isInstanceOf(ReadOnlyBufferException.class); + assertThat(readOnlyDst.position()).isEqualTo(0); buffer.release(); } @Test - public void testReadBytesAndWriteBytesWithFileChannel() throws IOException { + void testReadBytesAndWriteBytesWithFileChannel() throws IOException { File file = File.createTempFile("file-channel", ".tmp"); RandomAccessFile randomAccessFile = null; try { @@ -4594,21 +4709,21 @@ public void testReadBytesAndWriteBytesWithFileChannel() throws IOException { buffer.writeBytes(bytes); int oldReaderIndex = buffer.readerIndex(); - assertEquals(len, buffer.readBytes(channel, 10, len)); - assertEquals(oldReaderIndex + len, buffer.readerIndex()); - assertEquals(channelPosition, channel.position()); + assertThat(buffer.readBytes(channel, 10, len)).isEqualTo(len); + assertThat(buffer.readerIndex()).isEqualTo(oldReaderIndex + len); + assertThat(channel.position()).isEqualTo(channelPosition); ByteBuf buffer2 = newBuffer(len); buffer2.resetReaderIndex(); buffer2.resetWriterIndex(); int oldWriterIndex = buffer2.writerIndex(); - assertEquals(len, buffer2.writeBytes(channel, 10, len)); - assertEquals(channelPosition, channel.position()); - assertEquals(oldWriterIndex + len, buffer2.writerIndex()); - assertEquals('a', buffer2.getByte(0)); - assertEquals('b', buffer2.getByte(1)); - assertEquals('c', buffer2.getByte(2)); - assertEquals('d', buffer2.getByte(3)); + assertThat(buffer2.writeBytes(channel, 10, len)).isEqualTo(len); + assertThat(channel.position()).isEqualTo(channelPosition); + assertThat(buffer2.writerIndex()).isEqualTo(oldWriterIndex + len); + assertThat((char) buffer2.getByte(0)).isEqualTo('a'); + assertThat((char) buffer2.getByte(1)).isEqualTo('b'); + assertThat((char) buffer2.getByte(2)).isEqualTo('c'); + assertThat((char) buffer2.getByte(3)).isEqualTo('d'); buffer.release(); buffer2.release(); } finally { @@ -4620,7 +4735,7 @@ public void testReadBytesAndWriteBytesWithFileChannel() throws IOException { } @Test - public void testGetBytesAndSetBytesWithFileChannel() throws IOException { + void testGetBytesAndSetBytesWithFileChannel() throws IOException { File file = File.createTempFile("file-channel", ".tmp"); RandomAccessFile randomAccessFile = null; try { @@ -4637,22 +4752,22 @@ public void testGetBytesAndSetBytesWithFileChannel() throws IOException { buffer.writeBytes(bytes); int oldReaderIndex = buffer.readerIndex(); - assertEquals(len, buffer.getBytes(oldReaderIndex, channel, 10, len)); - assertEquals(oldReaderIndex, buffer.readerIndex()); - assertEquals(channelPosition, channel.position()); + assertThat(buffer.getBytes(oldReaderIndex, channel, 10, len)).isEqualTo(len); + assertThat(buffer.readerIndex()).isEqualTo(oldReaderIndex); + assertThat(channel.position()).isEqualTo(channelPosition); ByteBuf buffer2 = newBuffer(len); buffer2.resetReaderIndex(); buffer2.resetWriterIndex(); int oldWriterIndex = buffer2.writerIndex(); - assertEquals(buffer2.setBytes(oldWriterIndex, channel, 10, len), len); - assertEquals(channelPosition, channel.position()); + assertThat(buffer2.setBytes(oldWriterIndex, channel, 10, len)).isEqualTo(len); + assertThat(channel.position()).isEqualTo(channelPosition); - assertEquals(oldWriterIndex, buffer2.writerIndex()); - assertEquals('a', buffer2.getByte(oldWriterIndex)); - assertEquals('b', buffer2.getByte(oldWriterIndex + 1)); - assertEquals('c', buffer2.getByte(oldWriterIndex + 2)); - assertEquals('d', buffer2.getByte(oldWriterIndex + 3)); + assertThat(buffer2.writerIndex()).isEqualTo(oldWriterIndex); + assertThat((char) buffer2.getByte(oldWriterIndex)).isEqualTo('a'); + assertThat((char) buffer2.getByte(oldWriterIndex + 1)).isEqualTo('b'); + assertThat((char) buffer2.getByte(oldWriterIndex + 2)).isEqualTo('c'); + assertThat((char) buffer2.getByte(oldWriterIndex + 3)).isEqualTo('d'); buffer.release(); buffer2.release(); @@ -4665,22 +4780,22 @@ public void testGetBytesAndSetBytesWithFileChannel() throws IOException { } @Test - public void testReadBytes() { + void testReadBytes() { ByteBuf buffer = newBuffer(8); byte[] bytes = new byte[8]; buffer.writeBytes(bytes); ByteBuf buffer2 = buffer.readBytes(4); - assertSame(buffer.alloc(), buffer2.alloc()); - assertEquals(4, buffer.readerIndex()); - assertTrue(buffer.release()); - assertEquals(0, buffer.refCnt()); - assertTrue(buffer2.release()); - assertEquals(0, buffer2.refCnt()); + assertThat(buffer2.alloc()).isSameAs(buffer.alloc()); + assertThat(buffer.readerIndex()).isEqualTo(4); + assertThat(buffer.release()).isTrue(); + assertThat(buffer.refCnt()).isEqualTo(0); + assertThat(buffer2.release()).isTrue(); + assertThat(buffer2.refCnt()).isEqualTo(0); } @Test - public void testForEachByteDesc2() { + void testForEachByteDesc2() { byte[] expected = {1, 2, 3, 4}; ByteBuf buf = newBuffer(expected.length); try { @@ -4697,15 +4812,15 @@ public boolean process(byte value) throws Exception { return true; } }); - assertEquals(-1, i); - assertArrayEquals(expected, bytes); + assertThat(i).isEqualTo(-1); + assertThat(bytes).isEqualTo(expected); } finally { buf.release(); } } @Test - public void testForEachByte2() { + void testForEachByte2() { byte[] expected = {1, 2, 3, 4}; ByteBuf buf = newBuffer(expected.length); try { @@ -4722,22 +4837,23 @@ public boolean process(byte value) throws Exception { return true; } }); - assertEquals(-1, i); - assertArrayEquals(expected, bytes); + assertThat(i).isEqualTo(-1); + assertThat(bytes).isEqualTo(expected); } finally { buf.release(); } } - @Test(expected = IndexOutOfBoundsException.class) - public void testGetBytesByteBuffer() { + @Test + void testGetBytesByteBuffer() { byte[] bytes = {'a', 'b', 'c', 'd', 'e', 'f', 'g'}; // Ensure destination buffer is bigger then what is in the ByteBuf. ByteBuffer nioBuffer = ByteBuffer.allocate(bytes.length + 1); ByteBuf buffer = newBuffer(bytes.length); try { buffer.writeBytes(bytes); - buffer.getBytes(buffer.readerIndex(), nioBuffer); + assertThatThrownBy(() -> buffer.getBytes(buffer.readerIndex(), nioBuffer)) + .isInstanceOf(IndexOutOfBoundsException.class); } finally { buffer.release(); } @@ -4749,7 +4865,7 @@ private void testRefCnt0(final boolean parameter) throws Exception { final CountDownLatch innerLatch = new CountDownLatch(1); final ByteBuf buffer = newBuffer(4); - assertEquals(1, buffer.refCnt()); + assertThat(buffer.refCnt()).isEqualTo(1); final AtomicInteger cnt = new AtomicInteger(Integer.MAX_VALUE); Thread t1 = new Thread( @@ -4762,7 +4878,7 @@ public void run() { } else { released = buffer.release(); } - assertTrue(released); + assertThat(released).isTrue(); Thread t2 = new Thread( new Runnable() { @@ -4785,7 +4901,7 @@ public void run() { t1.start(); latch.await(); - assertEquals(0, cnt.get()); + assertThat(cnt.get()).isEqualTo(0); innerLatch.countDown(); } } @@ -4907,60 +5023,62 @@ public boolean process(byte value) throws Exception { } } - @Test(expected = IllegalArgumentException.class) - public void testCapacityEnforceMaxCapacity() { + @Test + void testCapacityEnforceMaxCapacity() { ByteBuf buffer = newBuffer(3, 13); - assertEquals(13, buffer.maxCapacity()); - assertEquals(3, buffer.capacity()); + assertThat(buffer.maxCapacity()).isEqualTo(13); + assertThat(buffer.capacity()).isEqualTo(3); try { - buffer.capacity(14); + assertThatThrownBy(() -> buffer.capacity(14)) + .isInstanceOf(IllegalArgumentException.class); } finally { buffer.release(); } } - @Test(expected = IllegalArgumentException.class) - public void testCapacityNegative() { + @Test + void testCapacityNegative() { ByteBuf buffer = newBuffer(3, 13); - assertEquals(13, buffer.maxCapacity()); - assertEquals(3, buffer.capacity()); + assertThat(buffer.maxCapacity()).isEqualTo(13); + assertThat(buffer.capacity()).isEqualTo(3); try { - buffer.capacity(-1); + assertThatThrownBy(() -> buffer.capacity(-1)) + .isInstanceOf(IllegalArgumentException.class); } finally { buffer.release(); } } @Test - public void testCapacityDecrease() { + void testCapacityDecrease() { ByteBuf buffer = newBuffer(3, 13); - assertEquals(13, buffer.maxCapacity()); - assertEquals(3, buffer.capacity()); + assertThat(buffer.maxCapacity()).isEqualTo(13); + assertThat(buffer.capacity()).isEqualTo(3); try { buffer.capacity(2); - assertEquals(2, buffer.capacity()); - assertEquals(13, buffer.maxCapacity()); + assertThat(buffer.capacity()).isEqualTo(2); + assertThat(buffer.maxCapacity()).isEqualTo(13); } finally { buffer.release(); } } @Test - public void testCapacityIncrease() { + void testCapacityIncrease() { ByteBuf buffer = newBuffer(3, 13); - assertEquals(13, buffer.maxCapacity()); - assertEquals(3, buffer.capacity()); + assertThat(buffer.maxCapacity()).isEqualTo(13); + assertThat(buffer.capacity()).isEqualTo(3); try { buffer.capacity(4); - assertEquals(4, buffer.capacity()); - assertEquals(13, buffer.maxCapacity()); + assertThat(buffer.capacity()).isEqualTo(4); + assertThat(buffer.maxCapacity()).isEqualTo(13); } finally { buffer.release(); } } - @Test(expected = IndexOutOfBoundsException.class) - public void testReaderIndexLargerThanWriterIndex() { + @Test + void testReaderIndexLargerThanWriterIndex() { String content1 = "hello"; String content2 = "world"; int length = content1.length() + content2.length(); @@ -4971,10 +5089,11 @@ public void testReaderIndexLargerThanWriterIndex() { buffer.skipBytes(content1.length()); buffer.writeCharSequence(content2, CharsetUtil.US_ASCII); buffer.skipBytes(content2.length()); - assertTrue(buffer.readerIndex() <= buffer.writerIndex()); + assertThat(buffer.readerIndex()).isLessThanOrEqualTo(buffer.writerIndex()); try { - buffer.resetWriterIndex(); + assertThatThrownBy(buffer::resetWriterIndex) + .isInstanceOf(IndexOutOfBoundsException.class); } finally { buffer.release(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java index b927917e81f9b..2a83b32a9c518 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java @@ -20,7 +20,7 @@ import org.apache.flink.core.memory.MemorySegment; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.annotation.Nullable; @@ -30,10 +30,8 @@ import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link BufferBuilder}. */ public class BufferBuilderAndConsumerTest { @@ -41,40 +39,41 @@ public class BufferBuilderAndConsumerTest { private static final int BUFFER_SIZE = BUFFER_INT_SIZE * Integer.BYTES; @Test - public void referenceCounting() { + void referenceCounting() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); - assertEquals(3 * Integer.BYTES, bufferBuilder.appendAndCommit(toByteBuffer(1, 2, 3))); + assertThat(bufferBuilder.appendAndCommit(toByteBuffer(1, 2, 3))) + .isEqualTo(3 * Integer.BYTES); bufferBuilder.close(); Buffer buffer = bufferConsumer.build(); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); buffer.recycleBuffer(); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); bufferConsumer.close(); - assertTrue(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isTrue(); } @Test - public void append() { + void append() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); int[] intsToWrite = new int[] {0, 1, 2, 3, 42}; ByteBuffer bytesToWrite = toByteBuffer(intsToWrite); - assertEquals(bytesToWrite.limit(), bufferBuilder.appendAndCommit(bytesToWrite)); + assertThat(bufferBuilder.appendAndCommit(bytesToWrite)).isEqualTo(bytesToWrite.limit()); - assertEquals(bytesToWrite.limit(), bytesToWrite.position()); - assertFalse(bufferBuilder.isFull()); + assertThat(bytesToWrite.position()).isEqualTo(bytesToWrite.limit()); + assertThat(bufferBuilder.isFull()).isFalse(); assertContent(bufferConsumer, intsToWrite); } @Test - public void multipleAppends() { + void multipleAppends() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); @@ -86,7 +85,7 @@ public void multipleAppends() { } @Test - public void multipleNotCommittedAppends() { + void multipleNotCommittedAppends() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); @@ -102,33 +101,34 @@ public void multipleNotCommittedAppends() { } @Test - public void appendOverSize() { + void appendOverSize() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); ByteBuffer bytesToWrite = toByteBuffer(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 42); - assertEquals(BUFFER_SIZE, bufferBuilder.appendAndCommit(bytesToWrite)); + assertThat(bufferBuilder.appendAndCommit(bytesToWrite)).isEqualTo(BUFFER_SIZE); - assertTrue(bufferBuilder.isFull()); + assertThat(bufferBuilder.isFull()).isTrue(); assertContent(bufferConsumer, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); bufferBuilder = createBufferBuilder(); bufferConsumer = bufferBuilder.createBufferConsumer(); - assertEquals(Integer.BYTES, bufferBuilder.appendAndCommit(bytesToWrite)); + assertThat(bufferBuilder.appendAndCommit(bytesToWrite)).isEqualTo(Integer.BYTES); - assertFalse(bufferBuilder.isFull()); + assertThat(bufferBuilder.isFull()).isFalse(); assertContent(bufferConsumer, 42); } - @Test(expected = IllegalStateException.class) - public void creatingBufferConsumerTwice() { + @Test + void creatingBufferConsumerTwice() { BufferBuilder bufferBuilder = createBufferBuilder(); bufferBuilder.createBufferConsumer(); - bufferBuilder.createBufferConsumer(); + assertThatThrownBy(bufferBuilder::createBufferConsumer) + .isInstanceOf(IllegalStateException.class); } @Test - public void copy() { + void copy() { BufferBuilder bufferBuilder = createBufferBuilder(); BufferConsumer bufferConsumer1 = bufferBuilder.createBufferConsumer(); @@ -154,16 +154,16 @@ public void copy() { } @Test - public void buildEmptyBuffer() { + void buildEmptyBuffer() { try (BufferBuilder bufferBuilder = createBufferBuilder()) { Buffer buffer = buildSingleBuffer(bufferBuilder); - assertEquals(0, buffer.getSize()); + assertThat(buffer.getSize()).isEqualTo(0); assertContent(buffer, FreeingBufferRecycler.INSTANCE); } } @Test - public void buildingBufferMultipleTimes() { + void buildingBufferMultipleTimes() { try (BufferBuilder bufferBuilder = createBufferBuilder()) { try (BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer()) { bufferBuilder.appendAndCommit(toByteBuffer(0, 1)); @@ -190,45 +190,43 @@ public void buildingBufferMultipleTimes() { } @Test - public void emptyIsFinished() { + void emptyIsFinished() { testIsFinished(0); } @Test - public void partiallyFullIsFinished() { + void partiallyFullIsFinished() { testIsFinished(BUFFER_INT_SIZE / 2); } @Test - public void fullIsFinished() { + void fullIsFinished() { testIsFinished(BUFFER_INT_SIZE); } @Test - public void testWritableBytes() { + void testWritableBytes() { BufferBuilder bufferBuilder = createBufferBuilder(); - assertEquals(bufferBuilder.getMaxCapacity(), bufferBuilder.getWritableBytes()); + assertThat(bufferBuilder.getWritableBytes()).isEqualTo(bufferBuilder.getMaxCapacity()); ByteBuffer byteBuffer = toByteBuffer(1, 2, 3); bufferBuilder.append(byteBuffer); - assertEquals( - bufferBuilder.getMaxCapacity() - byteBuffer.position(), - bufferBuilder.getWritableBytes()); + assertThat(bufferBuilder.getWritableBytes()) + .isEqualTo(bufferBuilder.getMaxCapacity() - byteBuffer.position()); - assertEquals( - bufferBuilder.getMaxCapacity() - byteBuffer.position(), - bufferBuilder.getWritableBytes()); + assertThat(bufferBuilder.getWritableBytes()) + .isEqualTo(bufferBuilder.getMaxCapacity() - byteBuffer.position()); } @Test - public void testWritableBytesWhenFull() { + void testWritableBytesWhenFull() { BufferBuilder bufferBuilder = createBufferBuilder(); bufferBuilder.append(toByteBuffer(new int[bufferBuilder.getMaxCapacity()])); - assertEquals(0, bufferBuilder.getWritableBytes()); + assertThat(bufferBuilder.getWritableBytes()).isEqualTo(0); } @Test - public void recycleWithoutConsumer() { + void recycleWithoutConsumer() { // given: Recycler with the counter of recycle invocation. CountedRecycler recycler = new CountedRecycler(); BufferBuilder bufferBuilder = @@ -238,11 +236,11 @@ public void recycleWithoutConsumer() { bufferBuilder.close(); // then: Recycling successfully finished. - assertEquals(1, recycler.recycleInvocationCounter); + assertThat(recycler.recycleInvocationCounter).isEqualTo(1); } @Test - public void recycleConsumerAndBufferBuilder() { + void recycleConsumerAndBufferBuilder() { // given: Recycler with the counter of recycling invocation. CountedRecycler recycler = new CountedRecycler(); BufferBuilder bufferBuilder = @@ -255,55 +253,55 @@ public void recycleConsumerAndBufferBuilder() { bufferBuilder.close(); // then: Nothing happened because BufferBuilder has already consumer. - assertEquals(0, recycler.recycleInvocationCounter); + assertThat(recycler.recycleInvocationCounter).isEqualTo(0); // when: Close the consumer. bufferConsumer.close(); // then: Recycling successfully finished. - assertEquals(1, recycler.recycleInvocationCounter); + assertThat(recycler.recycleInvocationCounter).isEqualTo(1); } @Test - public void trimToAvailableSize() { + void trimToAvailableSize() { BufferBuilder bufferBuilder = createBufferBuilder(); - assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); bufferBuilder.trim(BUFFER_SIZE / 2); - assertEquals(BUFFER_SIZE / 2, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE / 2); bufferBuilder.trim(0); - assertEquals(0, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(0); } @Test - public void trimToNegativeSize() { + void trimToNegativeSize() { BufferBuilder bufferBuilder = createBufferBuilder(); - assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); bufferBuilder.trim(-1); - assertEquals(0, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(0); } @Test - public void trimToSizeLessThanWritten() { + void trimToSizeLessThanWritten() { BufferBuilder bufferBuilder = createBufferBuilder(); - assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); bufferBuilder.append(toByteBuffer(1, 2, 3)); bufferBuilder.trim(4); // Should be minimum possible size = 3 * int == 12. - assertEquals(12, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(12); } @Test - public void trimToSizeGreaterThanMax() { + void trimToSizeGreaterThanMax() { BufferBuilder bufferBuilder = createBufferBuilder(); - assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); bufferBuilder.trim(BUFFER_SIZE + 1); - assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity()); + assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); } private static void testIsFinished(int writes) { @@ -311,33 +309,33 @@ private static void testIsFinished(int writes) { BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer(); for (int i = 0; i < writes; i++) { - assertEquals(Integer.BYTES, bufferBuilder.appendAndCommit(toByteBuffer(42))); + assertThat(bufferBuilder.appendAndCommit(toByteBuffer(42))).isEqualTo(Integer.BYTES); } int expectedWrittenBytes = writes * Integer.BYTES; - assertFalse(bufferBuilder.isFinished()); - assertFalse(bufferConsumer.isFinished()); - assertEquals(0, bufferConsumer.getWrittenBytes()); + assertThat(bufferBuilder.isFinished()).isFalse(); + assertThat(bufferConsumer.isFinished()).isFalse(); + assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(0); bufferConsumer.build(); - assertFalse(bufferBuilder.isFinished()); - assertFalse(bufferConsumer.isFinished()); - assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); + assertThat(bufferBuilder.isFinished()).isFalse(); + assertThat(bufferConsumer.isFinished()).isFalse(); + assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(expectedWrittenBytes); int actualWrittenBytes = bufferBuilder.finish(); - assertEquals(expectedWrittenBytes, actualWrittenBytes); - assertTrue(bufferBuilder.isFinished()); - assertFalse(bufferConsumer.isFinished()); - assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); + assertThat(actualWrittenBytes).isEqualTo(expectedWrittenBytes); + assertThat(bufferBuilder.isFinished()).isTrue(); + assertThat(bufferConsumer.isFinished()).isFalse(); + assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(expectedWrittenBytes); actualWrittenBytes = bufferBuilder.finish(); - assertEquals(expectedWrittenBytes, actualWrittenBytes); - assertTrue(bufferBuilder.isFinished()); - assertFalse(bufferConsumer.isFinished()); - assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); + assertThat(actualWrittenBytes).isEqualTo(expectedWrittenBytes); + assertThat(bufferBuilder.isFinished()).isTrue(); + assertThat(bufferConsumer.isFinished()).isFalse(); + assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(expectedWrittenBytes); - assertEquals(0, bufferConsumer.build().getSize()); - assertTrue(bufferConsumer.isFinished()); + assertThat(bufferConsumer.build().getSize()).isEqualTo(0); + assertThat(bufferBuilder.isFinished()).isTrue(); } public static ByteBuffer toByteBuffer(int... data) { @@ -347,11 +345,11 @@ public static ByteBuffer toByteBuffer(int... data) { } private static void assertContent(BufferConsumer actualConsumer, int... expected) { - assertFalse(actualConsumer.isFinished()); + assertThat(actualConsumer.isFinished()).isFalse(); Buffer buffer = actualConsumer.build(); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); assertContent(buffer, FreeingBufferRecycler.INSTANCE, expected); - assertEquals(expected.length * Integer.BYTES, buffer.getSize()); + assertThat(buffer.getSize()).isEqualTo(expected.length * Integer.BYTES); buffer.recycleBuffer(); } @@ -360,10 +358,10 @@ public static void assertContent( IntBuffer actualIntBuffer = actualBuffer.getNioBufferReadable().asIntBuffer(); int[] actual = new int[actualIntBuffer.limit()]; actualIntBuffer.get(actual); - assertArrayEquals(expected, actual); + assertThat(actual).containsExactly(expected); if (recycler != null) { - assertEquals(recycler, actualBuffer.getRecycler()); + assertThat(actualBuffer.getRecycler()).isEqualTo(recycler); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java index a8ef32f5224a7..2fdd6b9d8c9b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java @@ -25,7 +25,7 @@ import java.nio.ByteOrder; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** Utility class for create {@link BufferBuilder}, {@link BufferConsumer} and {@link Buffer}. */ public class BufferBuilderTestUtils { @@ -120,7 +120,7 @@ public static void validateBufferWithAscendingInts(Buffer buffer, int numInts, i final ByteBuffer bb = buffer.getNioBufferReadable().order(ByteOrder.LITTLE_ENDIAN); for (int i = 0; i < numInts; i++) { - assertEquals(nextValue++, bb.getInt()); + assertThat(bb.getInt()).isEqualTo(nextValue++); } } @@ -140,7 +140,7 @@ public static void validateBufferWithAscendingLongs( final ByteBuffer bb = buffer.getNioBufferReadable().order(ByteOrder.LITTLE_ENDIAN); for (int i = 0; i < numLongs; i++) { - assertEquals(nextValue++, bb.getLong()); + assertThat(bb.getLong()).isEqualTo(nextValue++); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java index d269151205bd6..f51103ee2a547 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java @@ -22,24 +22,22 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; import java.util.Collection; -import static junit.framework.TestCase.assertEquals; -import static junit.framework.TestCase.assertFalse; -import static junit.framework.TestCase.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link BufferCompressor} and {@link BufferDecompressor}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class BufferCompressionTest { private static final int BUFFER_SIZE = 4 * 1024 * 1024; @@ -95,111 +93,143 @@ public BufferCompressionTest( this.bufferToCompress = createBufferAndFillWithLongValues(isDirect); } - @Test - public void testCompressAndDecompressNetWorkBuffer() { + @TestTemplate + void testCompressAndDecompressNetWorkBuffer() { Buffer compressedBuffer = compress(compressor, bufferToCompress, compressToOriginalBuffer); - assertTrue(compressedBuffer.isCompressed()); + assertThat(compressedBuffer.isCompressed()).isTrue(); Buffer decompressedBuffer = decompress(decompressor, compressedBuffer, decompressToOriginalBuffer); - assertFalse(decompressedBuffer.isCompressed()); + assertThat(decompressedBuffer.isCompressed()).isFalse(); verifyDecompressionResult(decompressedBuffer, 0, NUM_LONGS); } - @Test - public void testCompressAndDecompressReadOnlySlicedNetworkBuffer() { + @TestTemplate + void testCompressAndDecompressReadOnlySlicedNetworkBuffer() { int offset = NUM_LONGS / 4 * 8; int length = NUM_LONGS / 2 * 8; Buffer readOnlySlicedBuffer = bufferToCompress.readOnlySlice(offset, length); Buffer compressedBuffer = compress(compressor, readOnlySlicedBuffer, compressToOriginalBuffer); - assertTrue(compressedBuffer.isCompressed()); + assertThat(compressedBuffer.isCompressed()).isTrue(); Buffer decompressedBuffer = decompress(decompressor, compressedBuffer, decompressToOriginalBuffer); - assertFalse(decompressedBuffer.isCompressed()); + assertThat(decompressedBuffer.isCompressed()).isFalse(); verifyDecompressionResult(decompressedBuffer, NUM_LONGS / 4, NUM_LONGS / 2); } - @Test(expected = IllegalArgumentException.class) - public void testCompressEmptyBuffer() { - compress(compressor, bufferToCompress.readOnlySlice(0, 0), compressToOriginalBuffer); + @TestTemplate + void testCompressEmptyBuffer() { + assertThatThrownBy( + () -> + compress( + compressor, + bufferToCompress.readOnlySlice(0, 0), + compressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testDecompressEmptyBuffer() { + @TestTemplate + void testDecompressEmptyBuffer() { Buffer readOnlySlicedBuffer = bufferToCompress.readOnlySlice(0, 0); readOnlySlicedBuffer.setCompressed(true); - decompress(decompressor, readOnlySlicedBuffer, decompressToOriginalBuffer); + assertThatThrownBy( + () -> + decompress( + decompressor, + readOnlySlicedBuffer, + decompressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testCompressBufferWithNonZeroReadOffset() { + @TestTemplate + void testCompressBufferWithNonZeroReadOffset() { bufferToCompress.setReaderIndex(1); - compress(compressor, bufferToCompress, compressToOriginalBuffer); + assertThatThrownBy(() -> compress(compressor, bufferToCompress, compressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testDecompressBufferWithNonZeroReadOffset() { + @TestTemplate + void testDecompressBufferWithNonZeroReadOffset() { bufferToCompress.setReaderIndex(1); bufferToCompress.setCompressed(true); - decompress(decompressor, bufferToCompress, decompressToOriginalBuffer); + assertThatThrownBy( + () -> + decompress( + decompressor, bufferToCompress, decompressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testCompressNull() { - compress(compressor, null, compressToOriginalBuffer); + @TestTemplate + void testCompressNull() { + assertThatThrownBy(() -> compress(compressor, null, compressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testDecompressNull() { - decompress(decompressor, null, decompressToOriginalBuffer); + @TestTemplate + void testDecompressNull() { + assertThatThrownBy(() -> decompress(decompressor, null, decompressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testCompressCompressedBuffer() { + @TestTemplate + void testCompressCompressedBuffer() { bufferToCompress.setCompressed(true); - compress(compressor, bufferToCompress, compressToOriginalBuffer); + assertThatThrownBy(() -> compress(compressor, bufferToCompress, compressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testDecompressUncompressedBuffer() { - decompress(decompressor, bufferToCompress, decompressToOriginalBuffer); + @TestTemplate + void testDecompressUncompressedBuffer() { + assertThatThrownBy( + () -> + decompress( + decompressor, bufferToCompress, decompressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testCompressEvent() throws IOException { - compress( - compressor, - EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE, false), - compressToOriginalBuffer); + @TestTemplate + void testCompressEvent() { + assertThatThrownBy( + () -> + compress( + compressor, + EventSerializer.toBuffer( + EndOfPartitionEvent.INSTANCE, false), + compressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalArgumentException.class) - public void testDecompressEvent() throws IOException { - decompress( - decompressor, - EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE, false), - decompressToOriginalBuffer); + @TestTemplate + void testDecompressEvent() { + assertThatThrownBy( + () -> + decompress( + decompressor, + EventSerializer.toBuffer( + EndOfPartitionEvent.INSTANCE, false), + decompressToOriginalBuffer)) + .isInstanceOf(IllegalArgumentException.class); } - @Test - public void testDataSizeGrowsAfterCompression() { + @TestTemplate + void testDataSizeGrowsAfterCompression() { int numBytes = 1; Buffer readOnlySlicedBuffer = bufferToCompress.readOnlySlice(BUFFER_SIZE / 2, numBytes); Buffer compressedBuffer = compress(compressor, readOnlySlicedBuffer, compressToOriginalBuffer); - assertFalse(compressedBuffer.isCompressed()); - assertEquals(readOnlySlicedBuffer, compressedBuffer); - assertEquals(numBytes, compressedBuffer.readableBytes()); + assertThat(compressedBuffer.isCompressed()).isFalse(); + assertThat(compressedBuffer).isEqualTo(readOnlySlicedBuffer); + assertThat(compressedBuffer.readableBytes()).isEqualTo(numBytes); } private static Buffer createBufferAndFillWithLongValues(boolean isDirect) { @@ -220,7 +250,7 @@ private static Buffer createBufferAndFillWithLongValues(boolean isDirect) { private static void verifyDecompressionResult(Buffer buffer, long start, int numLongs) { ByteBuffer byteBuffer = buffer.getNioBufferReadable().order(ByteOrder.LITTLE_ENDIAN); for (int i = 0; i < numLongs; ++i) { - assertEquals(start + i, byteBuffer.getLong()); + assertThat(byteBuffer.getLong()).isEqualTo(start + i); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java index 7122d3045eb58..190643f411825 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java @@ -21,20 +21,18 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.partition.PrioritizedDeque; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; import static java.util.Objects.requireNonNull; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest.assertContent; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest.toByteBuffer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link BufferConsumerWithPartialRecordLength}. */ -public class BufferConsumerWithPartialRecordLengthTest { +class BufferConsumerWithPartialRecordLengthTest { private static final int BUFFER_INT_SIZE = 4; private static final int BUFFER_SIZE = BUFFER_INT_SIZE * Integer.BYTES; private final PrioritizedDeque buffers = @@ -42,74 +40,74 @@ public class BufferConsumerWithPartialRecordLengthTest { private BufferBuilder builder = null; - @After - public void clear() { + @AfterEach + void clear() { buffers.clear(); builder = null; } @Test - public void partialRecordTestCase() { + void partialRecordTestCase() { writeToBuffer(toByteBuffer(0, 1, 2, 3, 42)); - assertEquals(buffers.size(), 2); + assertThat(buffers).hasSize(2); // buffer starts with a full record BufferConsumerWithPartialRecordLength consumer1 = buffers.poll(); - assertEquals(0, requireNonNull(consumer1).getPartialRecordLength()); - assertTrue(consumer1.cleanupPartialRecord()); + assertThat(requireNonNull(consumer1).getPartialRecordLength()).isEqualTo(0); + assertThat(consumer1.cleanupPartialRecord()).isTrue(); assertContent(consumer1.build(), FreeingBufferRecycler.INSTANCE, 0, 1, 2, 3); // buffer starts with partial record, partial record ends within the buffer // skip the partial record, return an empty buffer BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); - assertTrue(requireNonNull(consumer2).cleanupPartialRecord()); - assertEquals(consumer2.build().readableBytes(), 0); + assertThat(requireNonNull(consumer2).cleanupPartialRecord()).isTrue(); + assertThat(consumer2.build().readableBytes()).isEqualTo(0); } @Test - public void partialLongRecordSpanningBufferTestCase() { + void partialLongRecordSpanningBufferTestCase() { writeToBuffer(toByteBuffer(0, 1, 2, 3, 4, 5, 6, 7, 42)); writeToBuffer(toByteBuffer(8, 9)); - assertEquals(buffers.size(), 3); + assertThat(buffers).hasSize(3); buffers.poll(); // long partial record spanning over the entire buffer, clean up not successful BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); - assertEquals(BUFFER_SIZE, requireNonNull(consumer2).getPartialRecordLength()); - assertFalse(consumer2.cleanupPartialRecord()); - assertEquals(consumer2.build().readableBytes(), 0); + assertThat(requireNonNull(consumer2).getPartialRecordLength()).isEqualTo(BUFFER_SIZE); + assertThat(consumer2.cleanupPartialRecord()).isFalse(); + assertThat(consumer2.build().readableBytes()).isEqualTo(0); BufferConsumerWithPartialRecordLength consumer3 = buffers.poll(); - assertTrue(requireNonNull(consumer3).cleanupPartialRecord()); + assertThat(requireNonNull(consumer3).cleanupPartialRecord()).isTrue(); assertContent(consumer3.build(), FreeingBufferRecycler.INSTANCE, 8, 9); } @Test - public void partialLongRecordEndsWithFullBufferTestCase() { + void partialLongRecordEndsWithFullBufferTestCase() { writeToBuffer(toByteBuffer(0, 1, 2, 3, 4, 5, 6, 42)); writeToBuffer(toByteBuffer(8, 9)); - assertEquals(buffers.size(), 3); + assertThat(buffers).hasSize(3); buffers.poll(); // long partial record ends at the end of the buffer, clean up not successful BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); - assertEquals(BUFFER_SIZE, requireNonNull(consumer2).getPartialRecordLength()); - assertFalse(consumer2.cleanupPartialRecord()); - assertEquals(consumer2.build().readableBytes(), 0); + assertThat(requireNonNull(consumer2).getPartialRecordLength()).isEqualTo(BUFFER_SIZE); + assertThat(consumer2.cleanupPartialRecord()).isFalse(); + assertThat(consumer2.build().readableBytes()).isEqualTo(0); BufferConsumerWithPartialRecordLength consumer3 = buffers.poll(); - assertTrue(requireNonNull(consumer3).cleanupPartialRecord()); + assertThat(requireNonNull(consumer3).cleanupPartialRecord()).isTrue(); assertContent(consumer3.build(), FreeingBufferRecycler.INSTANCE, 8, 9); } @Test - public void readPositionNotAtTheBeginningOfTheBufferTestCase() { + void readPositionNotAtTheBeginningOfTheBufferTestCase() { writeToBuffer(toByteBuffer(0, 1, 2, 3, 42)); - assertEquals(buffers.size(), 2); + assertThat(buffers).hasSize(2); buffers.poll(); BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); @@ -117,8 +115,8 @@ public void readPositionNotAtTheBeginningOfTheBufferTestCase() { // read not start from the beginning of the buffer writeToBuffer(toByteBuffer(8, 9)); - assertEquals(4, consumer2.getPartialRecordLength()); - assertTrue(consumer2.cleanupPartialRecord()); + assertThat(consumer2.getPartialRecordLength()).isEqualTo(4); + assertThat(consumer2.cleanupPartialRecord()).isTrue(); assertContent(consumer2.build(), FreeingBufferRecycler.INSTANCE, 8, 9); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java index 19bd2e7eae3e8..0152476ce2e32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java @@ -20,11 +20,9 @@ import org.apache.flink.core.memory.MemorySegment; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -32,18 +30,14 @@ import java.util.Random; import java.util.stream.Stream; -import static org.hamcrest.Matchers.isOneOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Tests for the creation of {@link LocalBufferPool} instances from the {@link NetworkBufferPool} * factory. */ -public class BufferPoolFactoryTest { +class BufferPoolFactoryTest { private static final int numBuffers = 1024; @@ -51,18 +45,18 @@ public class BufferPoolFactoryTest { private NetworkBufferPool networkBufferPool; - @Rule public ExpectedException expectedException = ExpectedException.none(); - - @Before - public void setupNetworkBufferPool() { + @BeforeEach + void setupNetworkBufferPool() { networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize); } - @After - public void verifyAllBuffersReturned() { + @AfterEach + void verifyAllBuffersReturned() { String msg = "Did not return all buffers to network buffer pool after test."; try { - assertEquals(msg, numBuffers, networkBufferPool.getNumberOfAvailableMemorySegments()); + assertThat(networkBufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(numBuffers); } finally { // in case buffers have actually been requested, we must release them again networkBufferPool.destroyAllBufferPools(); @@ -72,24 +66,28 @@ public void verifyAllBuffersReturned() { /** Tests creating one buffer pool which requires more buffers than available. */ @Test - public void testRequireMoreThanPossible1() throws IOException { - expectedException.expect(IOException.class); - expectedException.expectMessage("Insufficient number of network buffers"); - - networkBufferPool.createBufferPool( - networkBufferPool.getTotalNumberOfMemorySegments() + 1, Integer.MAX_VALUE); + void testRequireMoreThanPossible1() throws IOException { + assertThatThrownBy( + () -> + networkBufferPool.createBufferPool( + networkBufferPool.getTotalNumberOfMemorySegments() + 1, + Integer.MAX_VALUE)) + .isInstanceOf(IOException.class) + .hasMessageContaining("Insufficient number of network buffers"); } /** Tests creating two buffer pools which together require more buffers than available. */ @Test - public void testRequireMoreThanPossible2() throws IOException { - expectedException.expect(IOException.class); - expectedException.expectMessage("Insufficient number of network buffers"); - + void testRequireMoreThanPossible2() throws IOException { BufferPool bufferPool = null; try { bufferPool = networkBufferPool.createBufferPool(numBuffers / 2 + 1, numBuffers); - networkBufferPool.createBufferPool(numBuffers / 2 + 1, numBuffers); + assertThatThrownBy( + () -> + networkBufferPool.createBufferPool( + numBuffers / 2 + 1, numBuffers)) + .isInstanceOf(IOException.class) + .hasMessageContaining("Insufficient number of network buffers"); } finally { if (bufferPool != null) { bufferPool.lazyDestroy(); @@ -103,7 +101,7 @@ public void testRequireMoreThanPossible2() throws IOException { * {@link LocalBufferPool} creation. */ @Test - public void testOverprovisioned() throws IOException { + void testOverprovisioned() throws IOException { // note: this is also the minimum number of buffers reserved for pool2 int buffersToTakeFromPool1 = numBuffers / 2 + 1; // note: this is also the minimum number of buffers reserved for pool1 @@ -117,48 +115,50 @@ public void testOverprovisioned() throws IOException { // take more buffers than the minimum required for (int i = 0; i < buffersToTakeFromPool1; ++i) { Buffer buffer = bufferPool1.requestBuffer(); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); buffers.add(buffer); } - assertEquals(buffersToTakeFromPool1, bufferPool1.bestEffortGetNumOfUsedBuffers()); - assertEquals(numBuffers, bufferPool1.getNumBuffers()); + assertThat(bufferPool1.bestEffortGetNumOfUsedBuffers()) + .isEqualTo(buffersToTakeFromPool1); + assertThat(bufferPool1.getNumBuffers()).isEqualTo(numBuffers); // create a second pool which requires more buffers than are available at the moment bufferPool2 = networkBufferPool.createBufferPool(buffersToTakeFromPool1, numBuffers); - assertEquals( - bufferPool2.getNumberOfRequiredMemorySegments(), bufferPool2.getNumBuffers()); - assertEquals( - bufferPool1.getNumberOfRequiredMemorySegments(), bufferPool1.getNumBuffers()); - assertNull(bufferPool1.requestBuffer()); + assertThat(bufferPool2.getNumBuffers()) + .isEqualTo(bufferPool2.getNumberOfRequiredMemorySegments()); + assertThat(bufferPool1.getNumBuffers()) + .isEqualTo(bufferPool1.getNumberOfRequiredMemorySegments()); + assertThat(bufferPool1.requestBuffer()).isNull(); // take all remaining buffers for (int i = 0; i < buffersToTakeFromPool2; ++i) { Buffer buffer = bufferPool2.requestBuffer(); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); buffers.add(buffer); } - assertEquals(buffersToTakeFromPool2, bufferPool2.bestEffortGetNumOfUsedBuffers()); + assertThat(bufferPool2.bestEffortGetNumOfUsedBuffers()) + .isEqualTo(buffersToTakeFromPool2); // we should be able to get one more but this is currently given out to bufferPool1 and // taken by buffer1 - assertNull(bufferPool2.requestBuffer()); + assertThat(bufferPool2.requestBuffer()).isNull(); // as soon as one excess buffer of bufferPool1 is recycled, it should be available for // bufferPool2 buffers.remove(0).recycleBuffer(); // recycle returns the excess buffer to the network buffer pool from where it's eagerly // fetched by pool 2 - assertEquals(0, networkBufferPool.getNumberOfAvailableMemorySegments()); + assertThat(networkBufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); // verify the number of buffers taken from the pools - assertEquals( - buffersToTakeFromPool1 - 1, - bufferPool1.bestEffortGetNumOfUsedBuffers() - + bufferPool1.getNumberOfAvailableMemorySegments()); - assertEquals( - buffersToTakeFromPool2 + 1, - bufferPool2.bestEffortGetNumOfUsedBuffers() - + bufferPool2.getNumberOfAvailableMemorySegments()); + assertThat( + bufferPool1.bestEffortGetNumOfUsedBuffers() + + bufferPool1.getNumberOfAvailableMemorySegments()) + .isEqualTo(buffersToTakeFromPool1 - 1); + assertThat( + bufferPool2.bestEffortGetNumOfUsedBuffers() + + bufferPool2.getNumberOfAvailableMemorySegments()) + .isEqualTo(buffersToTakeFromPool2 + 1); } finally { for (Buffer buffer : buffers) { buffer.recycleBuffer(); @@ -173,52 +173,54 @@ public void testOverprovisioned() throws IOException { } @Test - public void testBoundedPools() throws IOException { + void testBoundedPools() throws IOException { BufferPool bufferPool1 = networkBufferPool.createBufferPool(1, 1); - assertEquals(1, bufferPool1.getNumBuffers()); + assertThat(bufferPool1.getNumBuffers()).isEqualTo(1); BufferPool bufferPool2 = networkBufferPool.createBufferPool(1, 2); - assertEquals(2, bufferPool2.getNumBuffers()); + assertThat(bufferPool2.getNumBuffers()).isEqualTo(2); bufferPool1.lazyDestroy(); bufferPool2.lazyDestroy(); } @Test - public void testSingleManagedPoolGetsAll() throws IOException { + void testSingleManagedPoolGetsAll() throws IOException { BufferPool bufferPool = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments(), bufferPool.getNumBuffers()); + assertThat(bufferPool.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); bufferPool.lazyDestroy(); } @Test - public void testSingleManagedPoolGetsAllExceptFixedOnes() throws IOException { + void testSingleManagedPoolGetsAllExceptFixedOnes() throws IOException { BufferPool fixedBufferPool = networkBufferPool.createBufferPool(24, 24); BufferPool flexibleBufferPool = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals(24, fixedBufferPool.getNumBuffers()); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments() - - fixedBufferPool.getNumBuffers(), - flexibleBufferPool.getNumBuffers()); + assertThat(fixedBufferPool.getNumBuffers()).isEqualTo(24); + assertThat(flexibleBufferPool.getNumBuffers()) + .isEqualTo( + networkBufferPool.getTotalNumberOfMemorySegments() + - fixedBufferPool.getNumBuffers()); fixedBufferPool.lazyDestroy(); flexibleBufferPool.lazyDestroy(); } @Test - public void testUniformDistribution() throws IOException { + void testUniformDistribution() throws IOException { BufferPool first = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), first.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); BufferPool second = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, first.getNumBuffers()); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); + assertThat(second.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); first.lazyDestroy(); second.lazyDestroy(); @@ -229,75 +231,81 @@ public void testUniformDistribution() throws IOException { * in case both buffer pools request half of the available buffer count. */ @Test - public void testUniformDistributionAllBuffers() throws IOException { + void testUniformDistributionAllBuffers() throws IOException { BufferPool first = networkBufferPool.createBufferPool( networkBufferPool.getTotalNumberOfMemorySegments() / 2, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), first.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); BufferPool second = networkBufferPool.createBufferPool( networkBufferPool.getTotalNumberOfMemorySegments() / 2, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, first.getNumBuffers()); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); + assertThat(second.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); first.lazyDestroy(); second.lazyDestroy(); } @Test - public void testUniformDistributionBounded1() throws IOException { + void testUniformDistributionBounded1() throws IOException { BufferPool first = networkBufferPool.createBufferPool( 1, networkBufferPool.getTotalNumberOfMemorySegments()); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), first.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); BufferPool second = networkBufferPool.createBufferPool( 1, networkBufferPool.getTotalNumberOfMemorySegments()); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, first.getNumBuffers()); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); + assertThat(second.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); first.lazyDestroy(); second.lazyDestroy(); } @Test - public void testUniformDistributionBounded2() throws IOException { + void testUniformDistributionBounded2() throws IOException { BufferPool first = networkBufferPool.createBufferPool(1, 10); - assertEquals(10, first.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(10); BufferPool second = networkBufferPool.createBufferPool(1, 10); - assertEquals(10, first.getNumBuffers()); - assertEquals(10, second.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(10); + assertThat(second.getNumBuffers()).isEqualTo(10); first.lazyDestroy(); second.lazyDestroy(); } @Test - public void testUniformDistributionBounded3() throws IOException { + void testUniformDistributionBounded3() throws IOException { NetworkBufferPool globalPool = new NetworkBufferPool(3, 128); try { BufferPool first = globalPool.createBufferPool(1, 10); - assertEquals(3, first.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(3); BufferPool second = globalPool.createBufferPool(1, 10); // the order of which buffer pool received 2 or 1 buffer is undefined - assertEquals(3, first.getNumBuffers() + second.getNumBuffers()); - assertNotEquals(3, first.getNumBuffers()); - assertNotEquals(3, second.getNumBuffers()); + assertThat(first.getNumBuffers() + second.getNumBuffers()).isEqualTo(3); + assertThat(first.getNumBuffers()).isNotEqualTo(3); + assertThat(second.getNumBuffers()).isNotEqualTo(3); BufferPool third = globalPool.createBufferPool(1, 10); - assertEquals(1, first.getNumBuffers()); - assertEquals(1, second.getNumBuffers()); - assertEquals(1, third.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(1); + assertThat(second.getNumBuffers()).isEqualTo(1); + assertThat(third.getNumBuffers()).isEqualTo(1); // similar to #verifyAllBuffersReturned() String msg = "Wrong number of available segments after creating buffer pools."; - assertEquals(msg, 0, globalPool.getNumberOfAvailableMemorySegments()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(0); } finally { // in case buffers have actually been requested, we must release them again globalPool.destroyAllBufferPools(); @@ -311,52 +319,62 @@ public void testUniformDistributionBounded3() throws IOException { * requested memory segments or new buffer pools created. */ @Test - public void testUniformDistributionBounded4() throws IOException { + void testUniformDistributionBounded4() throws IOException { NetworkBufferPool globalPool = new NetworkBufferPool(10, 128); try { BufferPool first = globalPool.createBufferPool(1, 10); - assertEquals(10, first.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(10); List segmentList1 = globalPool.requestUnpooledMemorySegments(2); - assertEquals(2, segmentList1.size()); - assertEquals(8, first.getNumBuffers()); + assertThat(segmentList1).hasSize(2); + assertThat(first.getNumBuffers()).isEqualTo(8); BufferPool second = globalPool.createBufferPool(1, 10); - assertEquals(4, first.getNumBuffers()); - assertEquals(4, second.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(4); + assertThat(second.getNumBuffers()).isEqualTo(4); List segmentList2 = globalPool.requestUnpooledMemorySegments(2); - assertEquals(2, segmentList2.size()); - assertEquals(3, first.getNumBuffers()); - assertEquals(3, second.getNumBuffers()); + assertThat(segmentList2).hasSize(2); + assertThat(first.getNumBuffers()).isEqualTo(3); + assertThat(second.getNumBuffers()).isEqualTo(3); List segmentList3 = globalPool.requestUnpooledMemorySegments(2); - assertEquals(2, segmentList3.size()); - assertEquals(2, first.getNumBuffers()); - assertEquals(2, second.getNumBuffers()); + assertThat(segmentList3).hasSize(2); + assertThat(first.getNumBuffers()).isEqualTo(2); + assertThat(second.getNumBuffers()).isEqualTo(2); String msg = "Wrong number of available segments after creating buffer pools and requesting segments."; - assertEquals(msg, 2, globalPool.getNumberOfAvailableMemorySegments()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(2); globalPool.recycleUnpooledMemorySegments(segmentList1); - assertEquals(msg, 4, globalPool.getNumberOfAvailableMemorySegments()); - assertEquals(3, first.getNumBuffers()); - assertEquals(3, second.getNumBuffers()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(4); + assertThat(first.getNumBuffers()).isEqualTo(3); + assertThat(second.getNumBuffers()).isEqualTo(3); globalPool.recycleUnpooledMemorySegments(segmentList2); - assertEquals(msg, 6, globalPool.getNumberOfAvailableMemorySegments()); - assertEquals(4, first.getNumBuffers()); - assertEquals(4, second.getNumBuffers()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(6); + assertThat(first.getNumBuffers()).isEqualTo(4); + assertThat(second.getNumBuffers()).isEqualTo(4); globalPool.recycleUnpooledMemorySegments(segmentList3); - assertEquals(msg, 8, globalPool.getNumberOfAvailableMemorySegments()); - assertEquals(5, first.getNumBuffers()); - assertEquals(5, second.getNumBuffers()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(8); + assertThat(first.getNumBuffers()).isEqualTo(5); + assertThat(second.getNumBuffers()).isEqualTo(5); first.lazyDestroy(); - assertEquals(msg, 9, globalPool.getNumberOfAvailableMemorySegments()); - assertEquals(10, second.getNumBuffers()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .withFailMessage(msg) + .isEqualTo(9); + assertThat(second.getNumBuffers()).isEqualTo(10); } finally { globalPool.destroyAllBufferPools(); globalPool.destroy(); @@ -364,15 +382,15 @@ public void testUniformDistributionBounded4() throws IOException { } @Test - public void testBufferRedistributionMixed1() throws IOException { + void testBufferRedistributionMixed1() throws IOException { // try multiple times for various orders during redistribution for (int i = 0; i < 1_000; ++i) { BufferPool first = networkBufferPool.createBufferPool(1, 10); - assertEquals(10, first.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(10); BufferPool second = networkBufferPool.createBufferPool(1, 10); - assertEquals(10, first.getNumBuffers()); - assertEquals(10, second.getNumBuffers()); + assertThat(first.getNumBuffers()).isEqualTo(10); + assertThat(second.getNumBuffers()).isEqualTo(10); BufferPool third = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); // note: exact buffer distribution depends on the order during the redistribution @@ -383,10 +401,9 @@ public void testBufferRedistributionMixed1() throws IOException { * Math.min(avail, bp.getMaxNumberOfMemorySegments() - 1) / (avail + 20 - 2) + 1; - assertThat( - "Wrong buffer pool size after redistribution", - bp.getNumBuffers(), - isOneOf(size, size + 1)); + assertThat(bp.getNumBuffers()) + .withFailMessage("Wrong buffer pool size after redistribution") + .isIn(size, size + 1); } BufferPool fourth = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); @@ -398,10 +415,9 @@ public void testBufferRedistributionMixed1() throws IOException { * Math.min(avail, bp.getMaxNumberOfMemorySegments() - 1) / (2 * avail + 20 - 2) + 1; - assertThat( - "Wrong buffer pool size after redistribution", - bp.getNumBuffers(), - isOneOf(size, size + 1)); + assertThat(bp.getNumBuffers()) + .withFailMessage("Wrong buffer pool size after redistribution") + .isIn(size, size + 1); } Stream.of(first, second, third, fourth).forEach(BufferPool::lazyDestroy); @@ -411,7 +427,7 @@ public void testBufferRedistributionMixed1() throws IOException { } @Test - public void testAllDistributed() throws IOException { + void testAllDistributed() throws IOException { // try multiple times for various orders during redistribution for (int i = 0; i < 1_000; ++i) { Random random = new Random(); @@ -435,7 +451,7 @@ public void testAllDistributed() throws IOException { for (BufferPool pool : pools) { numDistributedBuffers += pool.getNumBuffers(); } - assertEquals(maxTotalUsed, numDistributedBuffers); + assertThat(numDistributedBuffers).isEqualTo(maxTotalUsed); } pools.forEach(BufferPool::lazyDestroy); @@ -445,21 +461,24 @@ public void testAllDistributed() throws IOException { } @Test - public void testCreateDestroy() throws IOException { + void testCreateDestroy() throws IOException { BufferPool first = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), first.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); BufferPool second = networkBufferPool.createBufferPool(1, Integer.MAX_VALUE); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, first.getNumBuffers()); + assertThat(first.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); - assertEquals( - networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers()); + assertThat(second.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments() / 2); first.lazyDestroy(); - assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), second.getNumBuffers()); + assertThat(second.getNumBuffers()) + .isEqualTo(networkBufferPool.getTotalNumberOfMemorySegments()); second.lazyDestroy(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java index 87f9ebf3abc6e..7d811ea6b579b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java @@ -20,30 +20,24 @@ import org.apache.flink.runtime.execution.CancelTaskException; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the destruction of a {@link LocalBufferPool}. */ public class LocalBufferPoolDestroyTest { @Test - public void testRequestAfterDestroy() throws IOException { + void testRequestAfterDestroy() { NetworkBufferPool networkBufferPool = new NetworkBufferPool(1, 4096); LocalBufferPool localBufferPool = new LocalBufferPool(networkBufferPool, 1); localBufferPool.lazyDestroy(); - try { - localBufferPool.requestBuffer(); - fail("Call should have failed with an IllegalStateException"); - } catch (CancelTaskException e) { - // we expect exactly that - } + assertThatThrownBy(localBufferPool::requestBuffer) + .withFailMessage("Call should have failed with an CancelTaskException") + .isInstanceOf(CancelTaskException.class); } /** @@ -54,7 +48,7 @@ public void testRequestAfterDestroy() throws IOException { * and we check whether the request Thread threw the expected Exception. */ @Test - public void testDestroyWhileBlockingRequest() throws Exception { + void testDestroyWhileBlockingRequest() throws Exception { AtomicReference asyncException = new AtomicReference<>(); NetworkBufferPool networkBufferPool = null; @@ -65,8 +59,8 @@ public void testDestroyWhileBlockingRequest() throws Exception { localBufferPool = new LocalBufferPool(networkBufferPool, 1); // Drain buffer pool - assertNotNull(localBufferPool.requestBuffer()); - assertNull(localBufferPool.requestBuffer()); + assertThat(localBufferPool.requestBuffer()).isNotNull(); + assertThat(localBufferPool.requestBuffer()).isNull(); // Start request Thread Thread thread = new Thread(new BufferRequestTask(localBufferPool, asyncException)); @@ -88,7 +82,9 @@ public void testDestroyWhileBlockingRequest() throws Exception { } // Verify that Thread was in blocking request - assertTrue("Did not trigger blocking buffer request.", success); + assertThat(success) + .withFailMessage("Did not trigger blocking buffer request.") + .isTrue(); // Destroy the buffer pool localBufferPool.lazyDestroy(); @@ -97,8 +93,10 @@ public void testDestroyWhileBlockingRequest() throws Exception { thread.join(); // Verify expected Exception - assertNotNull("Did not throw expected Exception", asyncException.get()); - assertTrue(asyncException.get() instanceof CancelTaskException); + assertThat(asyncException.get()) + .withFailMessage("Did not throw expected Exception") + .isNotNull(); + assertThat(asyncException.get()).isInstanceOf(CancelTaskException.class); } finally { if (localBufferPool != null) { localBufferPool.lazyDestroy(); @@ -137,7 +135,7 @@ private static class BufferRequestTask implements Runnable { private final BufferPool bufferPool; private final AtomicReference asyncException; - public BufferRequestTask(BufferPool bufferPool, AtomicReference asyncException) { + BufferRequestTask(BufferPool bufferPool, AtomicReference asyncException) { this.bufferPool = bufferPool; this.asyncException = asyncException; } @@ -146,7 +144,7 @@ public BufferRequestTask(BufferPool bufferPool, AtomicReference async public void run() { try { String msg = "Test assumption violated: expected no available buffer"; - assertNull(msg, bufferPool.requestBuffer()); + assertThat(bufferPool.requestBuffer()).withFailMessage(msg).isNull(); bufferPool.requestBufferBuilderBlocking(); } catch (Exception t) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 6126d07e4fff2..8aaa2163bdf44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -65,7 +65,7 @@ class LocalBufferPoolTest { private BufferPool localBufferPool; @RegisterExtension - public static final TestExecutorExtension EXECUTOR_RESOURCE = + static final TestExecutorExtension EXECUTOR_EXTENSION = new TestExecutorExtension<>(Executors::newCachedThreadPool); @BeforeEach @@ -587,7 +587,7 @@ void testConcurrentRequestRecycle() throws ExecutionException, InterruptedExcept Future[] taskResults = new Future[numConcurrentTasks]; for (int i = 0; i < numConcurrentTasks; i++) { taskResults[i] = - EXECUTOR_RESOURCE + EXECUTOR_EXTENSION .getExecutor() .submit( new BufferRequesterTask( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 20fd5964b4833..4de1f97fc230a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -23,9 +23,8 @@ import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.execution.CancelTaskException; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.time.Duration; @@ -43,59 +42,37 @@ import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasProperty; -import static org.hamcrest.core.IsCollectionContaining.hasItem; -import static org.hamcrest.core.IsNot.not; -import static org.hamcrest.core.IsNull.notNullValue; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; /** Tests for {@link NetworkBufferPool}. */ -public class NetworkBufferPoolTest extends TestLogger { +class NetworkBufferPoolTest { @Test - public void testCreatePoolAfterDestroy() { + void testCreatePoolAfterDestroy() { try { final int bufferSize = 128; final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize); - assertThat(globalPool.getNumberOfRegisteredBufferPools(), is(0)); + assertThat(globalPool.getNumberOfRegisteredBufferPools()).isEqualTo(0); globalPool.destroy(); - assertTrue(globalPool.isDestroyed()); + assertThat(globalPool.isDestroyed()).isTrue(); - try { - globalPool.createBufferPool(2, 2); - fail("Should throw an IllegalStateException"); - } catch (IllegalStateException e) { - // yippie! - } + assertThatThrownBy(() -> globalPool.createBufferPool(2, 2)) + .withFailMessage("Should throw an IllegalStateException") + .isInstanceOf(IllegalStateException.class); - try { - globalPool.createBufferPool(2, 10); - fail("Should throw an IllegalStateException"); - } catch (IllegalStateException e) { - // yippie! - } + assertThatThrownBy(() -> globalPool.createBufferPool(2, 10)) + .withFailMessage("Should throw an IllegalStateException") + .isInstanceOf(IllegalStateException.class); - try { - globalPool.createBufferPool(2, Integer.MAX_VALUE); - fail("Should throw an IllegalStateException"); - } catch (IllegalStateException e) { - // yippie! - } + assertThatThrownBy(() -> globalPool.createBufferPool(2, Integer.MAX_VALUE)) + .withFailMessage("Should throw an IllegalStateException") + .isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -103,42 +80,42 @@ public void testCreatePoolAfterDestroy() { } @Test - public void testMemoryUsageInTheContextOfMemoryPoolCreation() { + void testMemoryUsageInTheContextOfMemoryPoolCreation() { final int bufferSize = 128; final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize); - assertThat(globalPool.getTotalNumberOfMemorySegments(), is(numBuffers)); - assertThat(globalPool.getNumberOfAvailableMemorySegments(), is(numBuffers)); - assertThat(globalPool.getNumberOfUsedMemorySegments(), is(0)); + assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(numBuffers); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(0); - assertThat(globalPool.getTotalMemory(), is((long) numBuffers * bufferSize)); - assertThat(globalPool.getAvailableMemory(), is((long) numBuffers * bufferSize)); - assertThat(globalPool.getUsedMemory(), is(0L)); + assertThat(globalPool.getTotalMemory()).isEqualTo((long) numBuffers * bufferSize); + assertThat(globalPool.getAvailableMemory()).isEqualTo((long) numBuffers * bufferSize); + assertThat(globalPool.getUsedMemory()).isEqualTo(0L); } @Test - public void testMemoryUsageInTheContextOfMemorySegmentAllocation() { + void testMemoryUsageInTheContextOfMemorySegmentAllocation() { final int bufferSize = 128; final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize); MemorySegment segment = globalPool.requestPooledMemorySegment(); - assertThat(segment, is(notNullValue())); + assertThat(segment).isNotNull(); - assertThat(globalPool.getTotalNumberOfMemorySegments(), is(numBuffers)); - assertThat(globalPool.getNumberOfAvailableMemorySegments(), is(numBuffers - 1)); - assertThat(globalPool.getNumberOfUsedMemorySegments(), is(1)); + assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(numBuffers); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers - 1); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(1); - assertThat(globalPool.getTotalMemory(), is((long) numBuffers * bufferSize)); - assertThat(globalPool.getAvailableMemory(), is((long) (numBuffers - 1) * bufferSize)); - assertThat(globalPool.getUsedMemory(), is((long) bufferSize)); + assertThat(globalPool.getTotalMemory()).isEqualTo((long) numBuffers * bufferSize); + assertThat(globalPool.getAvailableMemory()).isEqualTo((long) (numBuffers - 1) * bufferSize); + assertThat(globalPool.getUsedMemory()).isEqualTo((long) bufferSize); } @Test - public void testMemoryUsageInTheContextOfMemoryPoolDestruction() { + void testMemoryUsageInTheContextOfMemoryPoolDestruction() { final int bufferSize = 128; final int numBuffers = 10; @@ -146,26 +123,26 @@ public void testMemoryUsageInTheContextOfMemoryPoolDestruction() { globalPool.destroy(); - assertThat(globalPool.getTotalNumberOfMemorySegments(), is(0)); - assertThat(globalPool.getNumberOfAvailableMemorySegments(), is(0)); - assertThat(globalPool.getNumberOfUsedMemorySegments(), is(0)); + assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(0); - assertThat(globalPool.getTotalMemory(), is(0L)); - assertThat(globalPool.getAvailableMemory(), is(0L)); - assertThat(globalPool.getUsedMemory(), is(0L)); + assertThat(globalPool.getTotalMemory()).isEqualTo(0L); + assertThat(globalPool.getAvailableMemory()).isEqualTo(0L); + assertThat(globalPool.getUsedMemory()).isEqualTo(0L); } @Test - public void testDestroyAll() throws IOException { + void testDestroyAll() throws IOException { NetworkBufferPool globalPool = new NetworkBufferPool(10, 128); BufferPool fixedPool = globalPool.createBufferPool(2, 2); BufferPool boundedPool = globalPool.createBufferPool(1, 1); BufferPool nonFixedPool = globalPool.createBufferPool(5, Integer.MAX_VALUE); - assertEquals(2, fixedPool.getNumberOfRequiredMemorySegments()); - assertEquals(1, boundedPool.getNumberOfRequiredMemorySegments()); - assertEquals(5, nonFixedPool.getNumberOfRequiredMemorySegments()); + assertThat(fixedPool.getNumberOfRequiredMemorySegments()).isEqualTo(2); + assertThat(boundedPool.getNumberOfRequiredMemorySegments()).isEqualTo(1); + assertThat(nonFixedPool.getNumberOfRequiredMemorySegments()).isEqualTo(5); // actually, the buffer pool sizes may be different due to rounding and based on the // internal order of @@ -179,65 +156,55 @@ public void testDestroyAll() throws IOException { for (BufferPool bp : new BufferPool[] {fixedPool, boundedPool, nonFixedPool}) { Buffer buffer = bp.requestBuffer(); if (buffer != null) { - assertNotNull(buffer.getMemorySegment()); + assertThat(buffer.getMemorySegment()).isNotNull(); buffers.add(buffer); continue collectBuffers; } } } - assertEquals(globalPool.getTotalNumberOfMemorySegments(), buffers.size()); + assertThat(buffers.size()).isEqualTo(globalPool.getTotalNumberOfMemorySegments()); - assertNull(fixedPool.requestBuffer()); - assertNull(boundedPool.requestBuffer()); - assertNull(nonFixedPool.requestBuffer()); + assertThat(fixedPool.requestBuffer()).isNull(); + assertThat(boundedPool.requestBuffer()).isNull(); + assertThat(nonFixedPool.requestBuffer()).isNull(); // destroy all allocated ones globalPool.destroyAllBufferPools(); // check the destroyed status - assertFalse(globalPool.isDestroyed()); - assertTrue(fixedPool.isDestroyed()); - assertTrue(boundedPool.isDestroyed()); - assertTrue(nonFixedPool.isDestroyed()); + assertThat(globalPool.isDestroyed()).isFalse(); + assertThat(fixedPool.isDestroyed()).isTrue(); + assertThat(boundedPool.isDestroyed()).isTrue(); + assertThat(nonFixedPool.isDestroyed()).isTrue(); - assertEquals(0, globalPool.getNumberOfRegisteredBufferPools()); + assertThat(globalPool.getNumberOfRegisteredBufferPools()).isEqualTo(0); // buffers are not yet recycled - assertEquals(0, globalPool.getNumberOfAvailableMemorySegments()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); // the recycled buffers should go to the global pool for (Buffer b : buffers) { b.recycleBuffer(); } - assertEquals( - globalPool.getTotalNumberOfMemorySegments(), - globalPool.getNumberOfAvailableMemorySegments()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()) + .isEqualTo(globalPool.getTotalNumberOfMemorySegments()); // can request no more buffers - try { - fixedPool.requestBuffer(); - fail("Should fail with an CancelTaskException"); - } catch (CancelTaskException e) { - // yippie! - } + assertThatThrownBy(fixedPool::requestBuffer) + .withFailMessage("Should fail with an CancelTaskException") + .isInstanceOf(CancelTaskException.class); - try { - boundedPool.requestBuffer(); - fail("Should fail with an CancelTaskException"); - } catch (CancelTaskException e) { - // that's the way we like it, aha, aha - } + assertThatThrownBy(boundedPool::requestBuffer) + .withFailMessage("Should fail with an CancelTaskException") + .isInstanceOf(CancelTaskException.class); - try { - nonFixedPool.requestBuffer(); - fail("Should fail with an CancelTaskException"); - } catch (CancelTaskException e) { - // stayin' alive - } + assertThatThrownBy(nonFixedPool::requestBuffer) + .withFailMessage("Should fail with an CancelTaskException") + .isInstanceOf(CancelTaskException.class); // can create a new pool now - assertNotNull(globalPool.createBufferPool(10, Integer.MAX_VALUE)); + assertThat(globalPool.createBufferPool(10, Integer.MAX_VALUE)).isNotNull(); } /** @@ -245,7 +212,7 @@ public void testDestroyAll() throws IOException { * NetworkBufferPool} currently containing the number of required free segments. */ @Test - public void testRequestMemorySegmentsLessThanTotalBuffers() throws IOException { + void testRequestMemorySegmentsLessThanTotalBuffers() throws IOException { final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); @@ -253,11 +220,11 @@ public void testRequestMemorySegmentsLessThanTotalBuffers() throws IOException { List memorySegments = Collections.emptyList(); try { memorySegments = globalPool.requestUnpooledMemorySegments(numBuffers / 2); - assertEquals(memorySegments.size(), numBuffers / 2); + assertThat(memorySegments.size()).isEqualTo(numBuffers / 2); globalPool.recycleUnpooledMemorySegments(memorySegments); memorySegments.clear(); - assertEquals(globalPool.getNumberOfAvailableMemorySegments(), numBuffers); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers); } finally { globalPool.recycleUnpooledMemorySegments(memorySegments); // just in case globalPool.destroy(); @@ -269,7 +236,7 @@ public void testRequestMemorySegmentsLessThanTotalBuffers() throws IOException { * required buffers exceeding the capacity of {@link NetworkBufferPool}. */ @Test - public void testRequestMemorySegmentsMoreThanTotalBuffers() { + void testRequestMemorySegmentsMoreThanTotalBuffers() { final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); @@ -278,7 +245,7 @@ public void testRequestMemorySegmentsMoreThanTotalBuffers() { globalPool.requestUnpooledMemorySegments(numBuffers + 1); fail("Should throw an IOException"); } catch (IOException e) { - assertEquals(globalPool.getNumberOfAvailableMemorySegments(), numBuffers); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers); } finally { globalPool.destroy(); } @@ -289,77 +256,74 @@ public void testRequestMemorySegmentsMoreThanTotalBuffers() { * allocated buffers for several requests exceeding the capacity of {@link NetworkBufferPool}. */ @Test - public void testInsufficientNumberOfBuffers() throws Exception { + void testInsufficientNumberOfBuffers() throws Exception { final int numberOfSegmentsToRequest = 5; final NetworkBufferPool globalPool = new NetworkBufferPool(numberOfSegmentsToRequest, 128); try { // the global pool should be in available state initially - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // request 5 segments List segments1 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertFalse(globalPool.getAvailableFuture().isDone()); - assertEquals(numberOfSegmentsToRequest, segments1.size()); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(segments1).hasSize(numberOfSegmentsToRequest); // request only 1 segment - IOException ioException = - assertThrows( - IOException.class, () -> globalPool.requestUnpooledMemorySegments(1)); - - assertTrue(ioException.getMessage().contains("Insufficient number of network buffers")); + assertThatThrownBy(() -> globalPool.requestUnpooledMemorySegments(1)) + .hasMessageContaining("Insufficient number of network buffers") + .isInstanceOf(IOException.class); // recycle 5 segments CompletableFuture availableFuture = globalPool.getAvailableFuture(); globalPool.recycleUnpooledMemorySegments(segments1); - assertTrue(availableFuture.isDone()); + assertThat(availableFuture.isDone()).isTrue(); List segments2 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertFalse(globalPool.getAvailableFuture().isDone()); - assertEquals(numberOfSegmentsToRequest, segments2.size()); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(segments2).hasSize(numberOfSegmentsToRequest); } finally { globalPool.destroy(); } } @Test - public void testEmptyPoolSegmentsUsage() throws IOException { + void testEmptyPoolSegmentsUsage() throws IOException { try (CloseableRegistry closeableRegistry = new CloseableRegistry()) { NetworkBufferPool globalPool = new NetworkBufferPool(0, 128); closeableRegistry.registerCloseable(globalPool::destroy); - assertEquals(0, globalPool.getEstimatedRequestedSegmentsUsage()); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(0); } } @Test - public void testSegmentsUsage() throws IOException { + void testSegmentsUsage() throws IOException { try (CloseableRegistry closeableRegistry = new CloseableRegistry()) { NetworkBufferPool globalPool = new NetworkBufferPool(50, 128); closeableRegistry.registerCloseable(globalPool::destroy); BufferPool bufferPool1 = globalPool.createBufferPool(10, 20); - assertEquals(20, globalPool.getEstimatedNumberOfRequestedMemorySegments()); - assertEquals(40, globalPool.getEstimatedRequestedSegmentsUsage()); - assertThat(globalPool.getUsageWarning(), equalTo(Optional.empty())); + assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(20); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(40); + assertThat(globalPool.getUsageWarning()).isEmpty(); closeableRegistry.registerCloseable( (globalPool.createBufferPool(5, Integer.MAX_VALUE))::lazyDestroy); - assertEquals(30, globalPool.getEstimatedNumberOfRequestedMemorySegments()); - assertEquals(60, globalPool.getEstimatedRequestedSegmentsUsage()); - assertThat(globalPool.getUsageWarning(), equalTo(Optional.empty())); + assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(30); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(60); + assertThat(globalPool.getUsageWarning()).isEmpty(); closeableRegistry.registerCloseable((globalPool.createBufferPool(10, 30))::lazyDestroy); - assertEquals(60, globalPool.getEstimatedNumberOfRequestedMemorySegments()); - assertEquals(120, globalPool.getEstimatedRequestedSegmentsUsage()); - assertThat( - globalPool.getUsageWarning(), - equalTo( + assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(60); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(120); + assertThat(globalPool.getUsageWarning()) + .isEqualTo( Optional.of( "Memory usage [120%] is too high to satisfy all of the requests. " + "This can severely impact network throughput. " @@ -367,16 +331,15 @@ public void testSegmentsUsage() throws IOException { + "or decreasing configured size of network buffer pools. (" + "totalMemory=6.250kb (6400 bytes), " + "requestedMemory=7.500kb (7680 bytes), " - + "missingMemory=1.250kb (1280 bytes))"))); - assertThat(globalPool.getUsageWarning(), equalTo(Optional.empty())); + + "missingMemory=1.250kb (1280 bytes))")); + assertThat(globalPool.getUsageWarning()).isEmpty(); BufferPool bufferPool2 = globalPool.createBufferPool(10, 20); - assertEquals(80, globalPool.getEstimatedNumberOfRequestedMemorySegments()); - assertEquals(160, globalPool.getEstimatedRequestedSegmentsUsage()); - assertThat( - globalPool.getUsageWarning(), - equalTo( + assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(80); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(160); + assertThat(globalPool.getUsageWarning()) + .isEqualTo( Optional.of( "Memory usage [160%] is too high to satisfy all of the requests. " + "This can severely impact network throughput. " @@ -384,18 +347,17 @@ public void testSegmentsUsage() throws IOException { + "or decreasing configured size of network buffer pools. (" + "totalMemory=6.250kb (6400 bytes), " + "requestedMemory=10.000kb (10240 bytes), " - + "missingMemory=3.750kb (3840 bytes))"))); + + "missingMemory=3.750kb (3840 bytes))")); bufferPool2.lazyDestroy(); bufferPool1.lazyDestroy(); - assertEquals(40, globalPool.getEstimatedNumberOfRequestedMemorySegments()); - assertEquals(40 * 128, globalPool.getEstimatedRequestedMemory()); - assertEquals(80, globalPool.getEstimatedRequestedSegmentsUsage()); - assertThat( - globalPool.getUsageWarning(), - equalTo(Optional.of("Memory usage [80%] went back to normal"))); - assertThat(globalPool.getUsageWarning(), equalTo(Optional.empty())); + assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(40); + assertThat(globalPool.getEstimatedRequestedMemory()).isEqualTo(40 * 128); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(80); + assertThat(globalPool.getUsageWarning()) + .isEqualTo(Optional.of("Memory usage [80%] went back to normal")); + assertThat(globalPool.getUsageWarning()).isEmpty(); } } @@ -403,13 +365,13 @@ public void testSegmentsUsage() throws IOException { * Tests {@link NetworkBufferPool#requestUnpooledMemorySegments(int)} with the invalid argument * to cause exception. */ - @Test(expected = IllegalArgumentException.class) - public void testRequestMemorySegmentsWithInvalidArgument() throws IOException { + @Test + void testRequestMemorySegmentsWithInvalidArgument() { NetworkBufferPool globalPool = new NetworkBufferPool(10, 128); // the number of requested buffers should be non-negative - globalPool.requestUnpooledMemorySegments(-1); + assertThatThrownBy(() -> globalPool.requestUnpooledMemorySegments(-1)) + .isInstanceOf(IllegalArgumentException.class); globalPool.destroy(); - fail("Should throw an IllegalArgumentException"); } /** @@ -418,8 +380,7 @@ public void testRequestMemorySegmentsWithInvalidArgument() throws IOException { * occupied by a buffer pool). */ @Test - public void testRequestMemorySegmentsWithBuffersTaken() - throws IOException, InterruptedException { + void testRequestMemorySegmentsWithBuffersTaken() throws IOException, InterruptedException { final int numBuffers = 10; NetworkBufferPool networkBufferPool = new NetworkBufferPool(numBuffers, 128); @@ -435,7 +396,7 @@ public void testRequestMemorySegmentsWithBuffersTaken() for (int i = 0; i < numBuffers; ++i) { Buffer buffer = lbp1.requestBuffer(); buffers.add(buffer); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); } // requestMemorySegments() below will wait for buffers @@ -459,7 +420,7 @@ public void testRequestMemorySegmentsWithBuffersTaken() // take more buffers than are freely available at the moment via requestMemorySegments() isRunning.await(); memorySegments = networkBufferPool.requestUnpooledMemorySegments(numBuffers / 2); - assertThat(memorySegments, not(hasItem(nullValue()))); + assertThat(memorySegments).doesNotContainNull(); } finally { if (bufferRecycler != null) { bufferRecycler.join(); @@ -477,12 +438,12 @@ public void testRequestMemorySegmentsWithBuffersTaken() * aborted in case of a concurrent {@link NetworkBufferPool#destroy()} call. */ @Test - public void testRequestMemorySegmentsInterruptable() throws Exception { + void testRequestMemorySegmentsInterruptable() throws Exception { final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); MemorySegment segment = globalPool.requestPooledMemorySegment(); - assertNotNull(segment); + assertThat(segment).isNotNull(); final OneShotLatch isRunning = new OneShotLatch(); CheckedThread asyncRequest = @@ -505,8 +466,9 @@ public void go() throws IOException { segment.free(); try { - Exception ex = assertThrows(IllegalStateException.class, asyncRequest::sync); - assertTrue(ex.getMessage().contains("destroyed")); + assertThatThrownBy(asyncRequest::sync) + .hasMessageContaining("destroyed") + .isInstanceOf(IllegalStateException.class); } finally { globalPool.destroy(); } @@ -517,12 +479,12 @@ public void go() throws IOException { * aborted and remains in a defined state even if the waiting is interrupted. */ @Test - public void testRequestMemorySegmentsInterruptable2() throws Exception { + void testRequestMemorySegmentsInterruptable2() throws Exception { final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); MemorySegment segment = globalPool.requestPooledMemorySegment(); - assertNotNull(segment); + assertThat(segment).isNotNull(); final OneShotLatch isRunning = new OneShotLatch(); CheckedThread asyncRequest = @@ -547,7 +509,7 @@ public void go() throws IOException { try { asyncRequest.sync(); } catch (IOException e) { - assertThat(e, hasProperty("cause", instanceOf(InterruptedException.class))); + assertThat(e).hasCauseInstanceOf(InterruptedException.class); // test indirectly for NetworkBufferPool#numTotalRequiredBuffers being correct: // -> creating a new buffer pool should not fail @@ -562,7 +524,7 @@ public void go() throws IOException { * exceptionally when failing to acquire all the segments in the specific timeout. */ @Test - public void testRequestMemorySegmentsTimeout() throws Exception { + void testRequestMemorySegmentsTimeout() throws Exception { final int numBuffers = 10; final int numberOfSegmentsToRequest = 2; final Duration requestSegmentsTimeout = Duration.ofMillis(50L); @@ -575,7 +537,7 @@ public void testRequestMemorySegmentsTimeout() throws Exception { localBufferPool.requestBuffer(); } - assertEquals(0, globalPool.getNumberOfAvailableMemorySegments()); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); CheckedThread asyncRequest = new CheckedThread() { @@ -588,8 +550,9 @@ public void go() throws Exception { asyncRequest.start(); try { - Exception ex = assertThrows(IOException.class, asyncRequest::sync); - assertTrue(ex.getMessage().contains("Timeout")); + assertThatThrownBy(asyncRequest::sync) + .hasMessageContaining("Timeout") + .isInstanceOf(IOException.class); } finally { globalPool.destroy(); } @@ -602,33 +565,33 @@ public void go() throws Exception { * NetworkBufferPool#recyclePooledMemorySegment(MemorySegment)}. */ @Test - public void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() { + void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() { final int numBuffers = 2; final NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); try { // the global pool should be in available state initially - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // request the first segment final MemorySegment segment1 = checkNotNull(globalPool.requestPooledMemorySegment()); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // request the second segment final MemorySegment segment2 = checkNotNull(globalPool.requestPooledMemorySegment()); - assertFalse(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); final CompletableFuture availableFuture = globalPool.getAvailableFuture(); // recycle the first segment globalPool.recyclePooledMemorySegment(segment1); - assertTrue(availableFuture.isDone()); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(availableFuture.isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // recycle the second segment globalPool.recyclePooledMemorySegment(segment2); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); } finally { globalPool.destroy(); @@ -642,7 +605,7 @@ public void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() { * NetworkBufferPool#recycleUnpooledMemorySegments(Collection)}. */ @Test - public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception { + void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception { final int numberOfSegmentsToRequest = 5; final int numBuffers = 2 * numberOfSegmentsToRequest; @@ -650,38 +613,38 @@ public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exc try { // the global pool should be in available state initially - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // request 5 segments List segments1 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertTrue(globalPool.getAvailableFuture().isDone()); - assertEquals(numberOfSegmentsToRequest, segments1.size()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(segments1).hasSize(numberOfSegmentsToRequest); // request another 5 segments List segments2 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertFalse(globalPool.getAvailableFuture().isDone()); - assertEquals(numberOfSegmentsToRequest, segments2.size()); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(segments2).hasSize(numberOfSegmentsToRequest); // recycle 5 segments CompletableFuture availableFuture = globalPool.getAvailableFuture(); globalPool.recycleUnpooledMemorySegments(segments1); - assertTrue(availableFuture.isDone()); + assertThat(availableFuture.isDone()).isTrue(); // request another 5 segments final List segments3 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertFalse(globalPool.getAvailableFuture().isDone()); - assertEquals(numberOfSegmentsToRequest, segments3.size()); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(segments3).hasSize(numberOfSegmentsToRequest); // recycle another 5 segments globalPool.recycleUnpooledMemorySegments(segments2); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); // recycle the last 5 segments globalPool.recycleUnpooledMemorySegments(segments3); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); } finally { globalPool.destroy(); @@ -693,8 +656,7 @@ public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exc * to the global network buffer pool. */ @Test - public void testBlockingRequestFromMultiLocalBufferPool() - throws IOException, InterruptedException { + void testBlockingRequestFromMultiLocalBufferPool() throws IOException, InterruptedException { final int localPoolRequiredSize = 5; final int localPoolMaxSize = 10; final int numLocalBufferPool = 2; @@ -711,7 +673,7 @@ public void testBlockingRequestFromMultiLocalBufferPool() final BufferPool localPool = globalPool.createBufferPool(localPoolRequiredSize, localPoolMaxSize); localBufferPools.add(localPool); - assertTrue(localPool.getAvailableFuture().isDone()); + assertThat(localPool.getAvailableFuture().isDone()).isTrue(); } // request some segments from the global pool in two different ways @@ -722,9 +684,9 @@ public void testBlockingRequestFromMultiLocalBufferPool() final List exclusiveSegments = globalPool.requestUnpooledMemorySegments( globalPool.getNumberOfAvailableMemorySegments() - 1); - assertTrue(globalPool.getAvailableFuture().isDone()); + assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); for (final BufferPool localPool : localBufferPools) { - assertTrue(localPool.getAvailableFuture().isDone()); + assertThat(localPool.getAvailableFuture().isDone()).isTrue(); } // blocking request buffers form local buffer pools @@ -751,18 +713,18 @@ public void testBlockingRequestFromMultiLocalBufferPool() while (segmentsRequested.size() + segments.size() + exclusiveSegments.size() < numBuffers) { Thread.sleep(10); - assertNull(cause.get()); + assertThat(cause.get()).isNull(); } final CompletableFuture globalPoolAvailableFuture = globalPool.getAvailableFuture(); - assertFalse(globalPoolAvailableFuture.isDone()); + assertThat(globalPoolAvailableFuture.isDone()).isFalse(); final List> localPoolAvailableFutures = new ArrayList<>(numLocalBufferPool); for (BufferPool localPool : localBufferPools) { CompletableFuture localPoolAvailableFuture = localPool.getAvailableFuture(); localPoolAvailableFutures.add(localPoolAvailableFuture); - assertFalse(localPoolAvailableFuture.isDone()); + assertThat(localPoolAvailableFuture.isDone()).isFalse(); } // recycle the previously requested segments @@ -771,20 +733,20 @@ public void testBlockingRequestFromMultiLocalBufferPool() } globalPool.recycleUnpooledMemorySegments(exclusiveSegments); - assertTrue(globalPoolAvailableFuture.isDone()); + assertThat(globalPoolAvailableFuture.isDone()).isTrue(); for (CompletableFuture localPoolAvailableFuture : localPoolAvailableFutures) { - assertTrue(localPoolAvailableFuture.isDone()); + assertThat(localPoolAvailableFuture.isDone()).isTrue(); } // wait until all blocking buffer requests finish latch.await(); - assertNull(cause.get()); - assertEquals(0, globalPool.getNumberOfAvailableMemorySegments()); - assertFalse(globalPool.getAvailableFuture().isDone()); + assertThat(cause.get()).isNull(); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); for (BufferPool localPool : localBufferPools) { - assertFalse(localPool.getAvailableFuture().isDone()); - assertEquals(localPoolMaxSize, localPool.bestEffortGetNumOfUsedBuffers()); + assertThat(localPool.getAvailableFuture().isDone()).isFalse(); + assertThat(localPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(localPoolMaxSize); } // recycle all the requested buffers diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java index 8e796fe0722df..36b3dbb1f5511 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java @@ -22,20 +22,15 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.io.network.netty.NettyBufferPool; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link NetworkBuffer} class. */ -public class NetworkBufferTest extends AbstractByteBufTest { +class NetworkBufferTest extends AbstractByteBufTest { /** Upper limit for the max size that is sufficient for all the tests. */ private static final int MAX_CAPACITY_UPPER_BOUND = 64 * 1024 * 1024; @@ -84,45 +79,45 @@ private static NetworkBuffer newBuffer( buffer.capacity(length); buffer.setAllocator(NETTY_BUFFER_POOL); - assertSame(ByteOrder.BIG_ENDIAN, buffer.order()); - assertEquals(0, buffer.readerIndex()); - assertEquals(0, buffer.writerIndex()); + assertThat(buffer.order()).isSameAs(ByteOrder.BIG_ENDIAN); + assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.writerIndex()).isEqualTo(0); return buffer; } @Test - public void testDataBufferIsBuffer() { - assertTrue(newBuffer(1024, 1024, true).isBuffer()); + void testDataBufferIsBuffer() { + assertThat(newBuffer(1024, 1024, true).isBuffer()).isTrue(); } @Test - public void testEventBufferIsBuffer() { - assertFalse(newBuffer(1024, 1024, false).isBuffer()); + void testEventBufferIsBuffer() { + assertThat(newBuffer(1024, 1024, false).isBuffer()).isFalse(); } @Test - public void testDataBufferTagAsEvent() { + void testDataBufferTagAsEvent() { testTagAsEvent(true); } @Test - public void testEventBufferTagAsEvent() { + void testEventBufferTagAsEvent() { testTagAsEvent(false); } private static void testTagAsEvent(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); buffer.setDataType(Buffer.DataType.EVENT_BUFFER); - assertFalse(buffer.isBuffer()); + assertThat(buffer.isBuffer()).isFalse(); } @Test - public void testDataBufferGetMemorySegment() { + void testDataBufferGetMemorySegment() { testGetMemorySegment(true); } @Test - public void testEventBufferGetMemorySegment() { + void testEventBufferGetMemorySegment() { testGetMemorySegment(false); } @@ -131,16 +126,16 @@ private static void testGetMemorySegment(boolean isBuffer) { Buffer.DataType dataType = isBuffer ? Buffer.DataType.DATA_BUFFER : Buffer.DataType.EVENT_BUFFER; NetworkBuffer buffer = new NetworkBuffer(segment, FreeingBufferRecycler.INSTANCE, dataType); - assertSame(segment, buffer.getMemorySegment()); + assertThat(buffer.getMemorySegment()).isSameAs(segment); } @Test - public void testDataBufferGetRecycler() { + void testDataBufferGetRecycler() { testGetRecycler(true); } @Test - public void testEventBufferGetRecycler() { + void testEventBufferGetRecycler() { testGetRecycler(false); } @@ -148,16 +143,16 @@ private static void testGetRecycler(boolean isBuffer) { BufferRecycler recycler = MemorySegment::free; NetworkBuffer dataBuffer = newBuffer(1024, 1024, isBuffer, recycler); - assertSame(recycler, dataBuffer.getRecycler()); + assertThat(dataBuffer.getRecycler()).isSameAs(recycler); } @Test - public void testDataBufferRecycleBuffer() { + void testDataBufferRecycleBuffer() { testRecycleBuffer(true); } @Test - public void testEventBufferRecycleBuffer() { + void testEventBufferRecycleBuffer() { testRecycleBuffer(false); } @@ -167,19 +162,19 @@ public void testEventBufferRecycleBuffer() { */ private static void testRecycleBuffer(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); buffer.recycleBuffer(); - assertTrue(buffer.isRecycled()); - assertEquals(0, buffer.refCnt()); + assertThat(buffer.isRecycled()).isTrue(); + assertThat(buffer.refCnt()).isEqualTo(0); } @Test - public void testDataBufferRetainBuffer() { + void testDataBufferRetainBuffer() { testRetainBuffer(true); } @Test - public void testEventBufferRetainBuffer() { + void testEventBufferRetainBuffer() { testRetainBuffer(false); } @@ -189,19 +184,19 @@ public void testEventBufferRetainBuffer() { */ private static void testRetainBuffer(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); buffer.retainBuffer(); - assertFalse(buffer.isRecycled()); - assertEquals(2, buffer.refCnt()); + assertThat(buffer.isRecycled()).isFalse(); + assertThat(buffer.refCnt()).isEqualTo(2); } @Test - public void testDataBufferCreateSlice1() { + void testDataBufferCreateSlice1() { testCreateSlice1(true); } @Test - public void testEventBufferCreateSlice1() { + void testEventBufferCreateSlice1() { testCreateSlice1(false); } @@ -210,24 +205,24 @@ private static void testCreateSlice1(boolean isBuffer) { buffer.setSize(10); // fake some data ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(); - assertEquals(0, slice.getReaderIndex()); - assertEquals(10, slice.getSize()); - assertSame(buffer, slice.unwrap().unwrap()); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getSize()).isEqualTo(10); + assertThat(slice.unwrap().unwrap()).isSameAs(buffer); // slice indices should be independent: buffer.setSize(8); buffer.setReaderIndex(2); - assertEquals(0, slice.getReaderIndex()); - assertEquals(10, slice.getSize()); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getSize()).isEqualTo(10); } @Test - public void testDataBufferCreateSlice2() { + void testDataBufferCreateSlice2() { testCreateSlice2(true); } @Test - public void testEventBufferCreateSlice2() { + void testEventBufferCreateSlice2() { testCreateSlice2(false); } @@ -236,42 +231,42 @@ private static void testCreateSlice2(boolean isBuffer) { buffer.setSize(2); // fake some data ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(1, 10); - assertEquals(0, slice.getReaderIndex()); - assertEquals(10, slice.getSize()); - assertSame(buffer, slice.unwrap().unwrap()); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getSize()).isEqualTo(10); + assertThat(slice.unwrap().unwrap()).isSameAs(buffer); // slice indices should be independent: buffer.setSize(8); buffer.setReaderIndex(2); - assertEquals(0, slice.getReaderIndex()); - assertEquals(10, slice.getSize()); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getSize()).isEqualTo(10); } @Test - public void testDataBufferGetMaxCapacity() { + void testDataBufferGetMaxCapacity() { testGetMaxCapacity(true); } @Test - public void testEventBufferGetMaxCapacity() { + void testEventBufferGetMaxCapacity() { testGetMaxCapacity(false); } private static void testGetMaxCapacity(boolean isBuffer) { NetworkBuffer buffer = newBuffer(100, 1024, isBuffer); - assertEquals(1024, buffer.getMaxCapacity()); + assertThat(buffer.getMaxCapacity()).isEqualTo(1024); MemorySegment segment = buffer.getMemorySegment(); - Assert.assertEquals(segment.size(), buffer.getMaxCapacity()); - Assert.assertEquals(segment.size(), buffer.maxCapacity()); + assertThat(segment.size()).isEqualTo(buffer.getMaxCapacity()); + assertThat(segment.size()).isEqualTo(buffer.maxCapacity()); } @Test - public void testDataBufferGetSetReaderIndex() { + void testDataBufferGetSetReaderIndex() { testGetSetReaderIndex(true); } @Test - public void testEventBufferGetSetReaderIndex() { + void testEventBufferGetSetReaderIndex() { testGetSetReaderIndex(false); } @@ -281,67 +276,67 @@ public void testEventBufferGetSetReaderIndex() { */ private static void testGetSetReaderIndex(boolean isBuffer) { NetworkBuffer buffer = newBuffer(100, 1024, isBuffer); - assertEquals(0, buffer.getReaderIndex()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); // fake some data buffer.setSize(100); - assertEquals(0, buffer.getReaderIndex()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); buffer.setReaderIndex(1); - assertEquals(1, buffer.getReaderIndex()); + assertThat(buffer.getReaderIndex()).isEqualTo(1); } @Test - public void testDataBufferSetGetSize() { + void testDataBufferSetGetSize() { testSetGetSize(true); } @Test - public void testEventBufferSetGetSize() { + void testEventBufferSetGetSize() { testSetGetSize(false); } private static void testSetGetSize(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertEquals(0, buffer.getSize()); // initially 0 - assertEquals(buffer.writerIndex(), buffer.getSize()); - assertEquals(0, buffer.readerIndex()); // initially 0 + assertThat(buffer.getSize()).isEqualTo(0); // initially 0 + assertThat(buffer.writerIndex()).isEqualTo(buffer.getSize()); + assertThat(buffer.readerIndex()).isEqualTo(0); // initially 0 buffer.setSize(10); - assertEquals(10, buffer.getSize()); - assertEquals(buffer.writerIndex(), buffer.getSize()); - assertEquals(0, buffer.readerIndex()); // independent + assertThat(buffer.getSize()).isEqualTo(10); + assertThat(buffer.writerIndex()).isEqualTo(buffer.getSize()); + assertThat(buffer.readerIndex()).isEqualTo(0); // independent } @Test - public void testDataBufferReadableBytes() { + void testDataBufferReadableBytes() { testReadableBytes(true); } @Test - public void testEventBufferReadableBytes() { + void testEventBufferReadableBytes() { testReadableBytes(false); } private static void testReadableBytes(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertEquals(0, buffer.readableBytes()); + assertThat(buffer.readableBytes()).isEqualTo(0); buffer.setSize(10); - assertEquals(10, buffer.readableBytes()); + assertThat(buffer.readableBytes()).isEqualTo(10); buffer.setReaderIndex(2); - assertEquals(8, buffer.readableBytes()); + assertThat(buffer.readableBytes()).isEqualTo(8); buffer.setReaderIndex(10); - assertEquals(0, buffer.readableBytes()); + assertThat(buffer.readableBytes()).isEqualTo(0); } @Test - public void testDataBufferGetNioBufferReadable() { + void testDataBufferGetNioBufferReadable() { testGetNioBufferReadable(true); } @Test - public void testEventBufferGetNioBufferReadable() { + void testEventBufferGetNioBufferReadable() { testGetNioBufferReadable(false); } @@ -349,32 +344,32 @@ private void testGetNioBufferReadable(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); ByteBuffer byteBuffer = buffer.getNioBufferReadable(); - assertFalse(byteBuffer.isReadOnly()); - assertEquals(0, byteBuffer.remaining()); - assertEquals(0, byteBuffer.limit()); - assertEquals(0, byteBuffer.capacity()); + assertThat(byteBuffer.isReadOnly()).isFalse(); + assertThat(byteBuffer.remaining()).isEqualTo(0); + assertThat(byteBuffer.limit()).isEqualTo(0); + assertThat(byteBuffer.capacity()).isEqualTo(0); // add some data buffer.setSize(10); // nothing changes in the byteBuffer - assertEquals(0, byteBuffer.remaining()); - assertEquals(0, byteBuffer.limit()); - assertEquals(0, byteBuffer.capacity()); + assertThat(byteBuffer.remaining()).isEqualTo(0); + assertThat(byteBuffer.limit()).isEqualTo(0); + assertThat(byteBuffer.capacity()).isEqualTo(0); // get a new byteBuffer (should have updated indices) byteBuffer = buffer.getNioBufferReadable(); - assertFalse(byteBuffer.isReadOnly()); - assertEquals(10, byteBuffer.remaining()); - assertEquals(10, byteBuffer.limit()); - assertEquals(10, byteBuffer.capacity()); + assertThat(byteBuffer.isReadOnly()).isFalse(); + assertThat(byteBuffer.remaining()).isEqualTo(10); + assertThat(byteBuffer.limit()).isEqualTo(10); + assertThat(byteBuffer.capacity()).isEqualTo(10); // modify byteBuffer position and verify nothing has changed in the original buffer byteBuffer.position(1); - assertEquals(0, buffer.getReaderIndex()); - assertEquals(10, buffer.getSize()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getSize()).isEqualTo(10); } @Test - public void testGetNioBufferReadableThreadSafe() { + void testGetNioBufferReadableThreadSafe() { NetworkBuffer buffer = newBuffer(1024, 1024); testGetNioBufferReadableThreadSafe(buffer); } @@ -383,19 +378,21 @@ static void testGetNioBufferReadableThreadSafe(Buffer buffer) { ByteBuffer buf1 = buffer.getNioBufferReadable(); ByteBuffer buf2 = buffer.getNioBufferReadable(); - assertNotNull(buf1); - assertNotNull(buf2); + assertThat(buf1).isNotNull(); + assertThat(buf2).isNotNull(); - assertTrue("Repeated call to getNioBuffer() returns the same nio buffer", buf1 != buf2); + assertThat(buf1) + .withFailMessage("Repeated call to getNioBuffer() returns the same nio buffer") + .isNotSameAs(buf2); } @Test - public void testDataBufferGetNioBuffer() { + void testDataBufferGetNioBuffer() { testGetNioBuffer(true); } @Test - public void testEventBufferGetNioBuffer() { + void testEventBufferGetNioBuffer() { testGetNioBuffer(false); } @@ -403,32 +400,32 @@ private void testGetNioBuffer(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); ByteBuffer byteBuffer = buffer.getNioBuffer(1, 1); - assertFalse(byteBuffer.isReadOnly()); - assertEquals(1, byteBuffer.remaining()); - assertEquals(1, byteBuffer.limit()); - assertEquals(1, byteBuffer.capacity()); + assertThat(byteBuffer.isReadOnly()).isFalse(); + assertThat(byteBuffer.remaining()).isEqualTo(1); + assertThat(byteBuffer.limit()).isEqualTo(1); + assertThat(byteBuffer.capacity()).isEqualTo(1); // add some data buffer.setSize(10); // nothing changes in the byteBuffer - assertEquals(1, byteBuffer.remaining()); - assertEquals(1, byteBuffer.limit()); - assertEquals(1, byteBuffer.capacity()); + assertThat(byteBuffer.remaining()).isEqualTo(1); + assertThat(byteBuffer.limit()).isEqualTo(1); + assertThat(byteBuffer.capacity()).isEqualTo(1); // get a new byteBuffer (should have updated indices) byteBuffer = buffer.getNioBuffer(1, 2); - assertFalse(byteBuffer.isReadOnly()); - assertEquals(2, byteBuffer.remaining()); - assertEquals(2, byteBuffer.limit()); - assertEquals(2, byteBuffer.capacity()); + assertThat(byteBuffer.isReadOnly()).isFalse(); + assertThat(byteBuffer.remaining()).isEqualTo(2); + assertThat(byteBuffer.limit()).isEqualTo(2); + assertThat(byteBuffer.capacity()).isEqualTo(2); // modify byteBuffer position and verify nothing has changed in the original buffer byteBuffer.position(1); - assertEquals(0, buffer.getReaderIndex()); - assertEquals(10, buffer.getSize()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getSize()).isEqualTo(10); } @Test - public void testGetNioBufferThreadSafe() { + void testGetNioBufferThreadSafe() { NetworkBuffer buffer = newBuffer(1024, 1024); testGetNioBufferThreadSafe(buffer, 10); } @@ -437,21 +434,22 @@ static void testGetNioBufferThreadSafe(Buffer buffer, int length) { ByteBuffer buf1 = buffer.getNioBuffer(0, length); ByteBuffer buf2 = buffer.getNioBuffer(0, length); - assertNotNull(buf1); - assertNotNull(buf2); + assertThat(buf1).isNotNull(); + assertThat(buf2).isNotNull(); - assertTrue( - "Repeated call to getNioBuffer(int, int) returns the same nio buffer", - buf1 != buf2); + assertThat(buf1) + .withFailMessage( + "Repeated call to getNioBuffer(int, int) returns the same nio buffer") + .isNotSameAs(buf2); } @Test - public void testDataBufferSetAllocator() { + void testDataBufferSetAllocator() { testSetAllocator(true); } @Test - public void testEventBufferSetAllocator() { + void testEventBufferSetAllocator() { testSetAllocator(false); } @@ -460,6 +458,6 @@ private void testSetAllocator(boolean isBuffer) { NettyBufferPool allocator = new NettyBufferPool(1); buffer.setAllocator(allocator); - assertSame(allocator, buffer.alloc()); + assertThat(buffer.alloc()).isSameAs(allocator); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java index 3b92d17761838..bcb21492ae6ff 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java @@ -26,28 +26,23 @@ import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; -import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link ReadOnlySlicedNetworkBuffer}. */ -public class ReadOnlySlicedBufferTest { +class ReadOnlySlicedBufferTest { private static final int BUFFER_SIZE = 1024; private static final int DATA_SIZE = 10; private NetworkBuffer buffer; - @Before - public void setUp() throws Exception { + @BeforeEach + void setUp() throws Exception { final MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE); buffer = new NetworkBuffer( @@ -58,40 +53,41 @@ public void setUp() throws Exception { } @Test - public void testForwardsIsBuffer() throws IOException { - assertEquals(buffer.isBuffer(), buffer.readOnlySlice().isBuffer()); - assertEquals(buffer.isBuffer(), buffer.readOnlySlice(1, 2).isBuffer()); + void testForwardsIsBuffer() throws IOException { + assertThat(buffer.readOnlySlice().isBuffer()).isEqualTo(buffer.isBuffer()); + assertThat(buffer.readOnlySlice(1, 2).isBuffer()).isEqualTo(buffer.isBuffer()); Buffer eventBuffer = EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE, false); - assertEquals(eventBuffer.isBuffer(), eventBuffer.readOnlySlice().isBuffer()); - assertEquals(eventBuffer.isBuffer(), eventBuffer.readOnlySlice(1, 2).isBuffer()); + assertThat(eventBuffer.readOnlySlice().isBuffer()).isEqualTo(eventBuffer.isBuffer()); + assertThat(eventBuffer.readOnlySlice(1, 2).isBuffer()).isEqualTo(eventBuffer.isBuffer()); } @Test - public void testSetDataType1() { + void testSetDataType1() { ReadOnlySlicedNetworkBuffer readOnlyBuffer = buffer.readOnlySlice(); readOnlyBuffer.setDataType(Buffer.DataType.EVENT_BUFFER); - Assertions.assertThat(readOnlyBuffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER); + assertThat(readOnlyBuffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER); } @Test - public void testSetDataType2() { + void testSetDataType2() { ReadOnlySlicedNetworkBuffer readOnlyBuffer = buffer.readOnlySlice(1, 2); readOnlyBuffer.setDataType(Buffer.DataType.EVENT_BUFFER); - Assertions.assertThat(readOnlyBuffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER); - Assertions.assertThat(buffer.readOnlySlice(1, 2).getDataType()) + assertThat(readOnlyBuffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER); + assertThat(buffer.readOnlySlice(1, 2).getDataType()) .isNotEqualTo(Buffer.DataType.EVENT_BUFFER); } @Test - public void testForwardsGetMemorySegment() { - assertSame(buffer.getMemorySegment(), buffer.readOnlySlice().getMemorySegment()); - assertSame(buffer.getMemorySegment(), buffer.readOnlySlice(1, 2).getMemorySegment()); + void testForwardsGetMemorySegment() { + assertThat(buffer.readOnlySlice().getMemorySegment()).isSameAs(buffer.getMemorySegment()); + assertThat(buffer.readOnlySlice(1, 2).getMemorySegment()) + .isSameAs(buffer.getMemorySegment()); } @Test - public void testForwardsGetRecycler() { - assertSame(buffer.getRecycler(), buffer.readOnlySlice().getRecycler()); - assertSame(buffer.getRecycler(), buffer.readOnlySlice(1, 2).getRecycler()); + void testForwardsGetRecycler() { + assertThat(buffer.readOnlySlice().getRecycler()).isSameAs(buffer.getRecycler()); + assertThat(buffer.readOnlySlice(1, 2).getRecycler()).isSameAs(buffer.getRecycler()); } /** @@ -99,12 +95,12 @@ public void testForwardsGetRecycler() { * ReadOnlySlicedNetworkBuffer#isRecycled()}. */ @Test - public void testForwardsRecycleBuffer1() { + void testForwardsRecycleBuffer1() { ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(); - assertFalse(slice.isRecycled()); + assertThat(slice.isRecycled()).isFalse(); slice.recycleBuffer(); - assertTrue(slice.isRecycled()); - assertTrue(buffer.isRecycled()); + assertThat(slice.isRecycled()).isTrue(); + assertThat(buffer.isRecycled()).isTrue(); } /** @@ -112,12 +108,12 @@ public void testForwardsRecycleBuffer1() { * ReadOnlySlicedNetworkBuffer#isRecycled()}. */ @Test - public void testForwardsRecycleBuffer2() { + void testForwardsRecycleBuffer2() { ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(1, 2); - assertFalse(slice.isRecycled()); + assertThat(slice.isRecycled()).isFalse(); slice.recycleBuffer(); - assertTrue(slice.isRecycled()); - assertTrue(buffer.isRecycled()); + assertThat(slice.isRecycled()).isTrue(); + assertThat(buffer.isRecycled()).isTrue(); } /** @@ -125,11 +121,11 @@ public void testForwardsRecycleBuffer2() { * ReadOnlySlicedNetworkBuffer#isRecycled()}. */ @Test - public void testForwardsRetainBuffer1() { + void testForwardsRetainBuffer1() { ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(); - assertEquals(buffer.refCnt(), slice.refCnt()); + assertThat(slice.refCnt()).isEqualTo(buffer.refCnt()); slice.retainBuffer(); - assertEquals(buffer.refCnt(), slice.refCnt()); + assertThat(slice.refCnt()).isEqualTo(buffer.refCnt()); } /** @@ -137,75 +133,75 @@ public void testForwardsRetainBuffer1() { * ReadOnlySlicedNetworkBuffer#isRecycled()}. */ @Test - public void testForwardsRetainBuffer2() { + void testForwardsRetainBuffer2() { ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(1, 2); - assertEquals(buffer.refCnt(), slice.refCnt()); + assertThat(slice.refCnt()).isEqualTo(buffer.refCnt()); slice.retainBuffer(); - assertEquals(buffer.refCnt(), slice.refCnt()); + assertThat(slice.refCnt()).isEqualTo(buffer.refCnt()); } @Test - public void testCreateSlice1() { + void testCreateSlice1() { buffer.readByte(); // so that we do not start at position 0 ReadOnlySlicedNetworkBuffer slice1 = buffer.readOnlySlice(); buffer.readByte(); // should not influence the second slice at all ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(); - assertSame(buffer, slice2.unwrap().unwrap()); - assertSame(slice1.getMemorySegment(), slice2.getMemorySegment()); - assertEquals(1, slice1.getMemorySegmentOffset()); - assertEquals(slice1.getMemorySegmentOffset(), slice2.getMemorySegmentOffset()); + assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); + assertThat(slice2.getMemorySegment()).isEqualTo(slice1.getMemorySegment()); + assertThat(slice2.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice2.getMemorySegmentOffset()).isEqualTo(slice1.getMemorySegmentOffset()); assertReadableBytes(slice1, 1, 2, 3, 4, 5, 6, 7, 8, 9); assertReadableBytes(slice2, 1, 2, 3, 4, 5, 6, 7, 8, 9); } @Test - public void testCreateSlice2() { + void testCreateSlice2() { buffer.readByte(); // so that we do not start at position 0 ReadOnlySlicedNetworkBuffer slice1 = buffer.readOnlySlice(); buffer.readByte(); // should not influence the second slice at all ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(1, 2); - assertSame(buffer, slice2.unwrap().unwrap()); - assertSame(slice1.getMemorySegment(), slice2.getMemorySegment()); - assertEquals(1, slice1.getMemorySegmentOffset()); - assertEquals(2, slice2.getMemorySegmentOffset()); + assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); + assertThat(slice2.getMemorySegment()).isEqualTo(slice1.getMemorySegment()); + assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice2.getMemorySegmentOffset()).isEqualTo(2); assertReadableBytes(slice1, 1, 2, 3, 4, 5, 6, 7, 8, 9); assertReadableBytes(slice2, 2, 3); } @Test - public void testCreateSlice3() { + void testCreateSlice3() { ReadOnlySlicedNetworkBuffer slice1 = buffer.readOnlySlice(1, 2); buffer.readByte(); // should not influence the second slice at all ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(); - assertSame(buffer, slice2.unwrap().unwrap()); - assertSame(slice1.getMemorySegment(), slice2.getMemorySegment()); - assertEquals(1, slice1.getMemorySegmentOffset()); - assertEquals(1, slice2.getMemorySegmentOffset()); + assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); + assertThat(slice2.getMemorySegment()).isSameAs(slice1.getMemorySegment()); + assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice2.getMemorySegmentOffset()).isEqualTo(1); assertReadableBytes(slice1, 1, 2); assertReadableBytes(slice2, 1, 2); } @Test - public void testCreateSlice4() { + void testCreateSlice4() { ReadOnlySlicedNetworkBuffer slice1 = buffer.readOnlySlice(1, 5); buffer.readByte(); // should not influence the second slice at all ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(1, 2); - assertSame(buffer, slice2.unwrap().unwrap()); - assertSame(slice1.getMemorySegment(), slice2.getMemorySegment()); - assertEquals(1, slice1.getMemorySegmentOffset()); - assertEquals(2, slice2.getMemorySegmentOffset()); + assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); + assertThat(slice2.getMemorySegment()).isSameAs(slice1.getMemorySegment()); + assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice2.getMemorySegmentOffset()).isEqualTo(2); assertReadableBytes(slice1, 1, 2, 3, 4, 5); assertReadableBytes(slice2, 2, 3); } @Test - public void testGetMaxCapacity() { - assertEquals(DATA_SIZE, buffer.readOnlySlice().getMaxCapacity()); - assertEquals(2, buffer.readOnlySlice(1, 2).getMaxCapacity()); + void testGetMaxCapacity() { + assertThat(buffer.readOnlySlice().getMaxCapacity()).isEqualTo(DATA_SIZE); + assertThat(buffer.readOnlySlice(1, 2).getMaxCapacity()).isEqualTo(2); } /** @@ -214,7 +210,7 @@ public void testGetMaxCapacity() { * ReadOnlySlicedNetworkBuffer#getReaderIndex()}. */ @Test - public void testGetSetReaderIndex1() { + void testGetSetReaderIndex1() { testGetSetReaderIndex(buffer.readOnlySlice()); } @@ -224,16 +220,16 @@ public void testGetSetReaderIndex1() { * ReadOnlySlicedNetworkBuffer#getReaderIndex()}. */ @Test - public void testGetSetReaderIndex2() { + void testGetSetReaderIndex2() { testGetSetReaderIndex(buffer.readOnlySlice(1, 2)); } private void testGetSetReaderIndex(ReadOnlySlicedNetworkBuffer slice) { - assertEquals(0, buffer.getReaderIndex()); - assertEquals(0, slice.getReaderIndex()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isEqualTo(0); slice.setReaderIndex(1); - assertEquals(0, buffer.getReaderIndex()); - assertEquals(1, slice.getReaderIndex()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isEqualTo(1); } /** @@ -241,7 +237,7 @@ private void testGetSetReaderIndex(ReadOnlySlicedNetworkBuffer slice) { * ReadOnlySlicedNetworkBuffer#setSize(int)}, {@link ReadOnlySlicedNetworkBuffer#getSize()}. */ @Test - public void testGetSetSize1() { + void testGetSetSize1() { testGetSetSize(buffer.readOnlySlice(), DATA_SIZE); } @@ -250,96 +246,96 @@ public void testGetSetSize1() { * ReadOnlySlicedNetworkBuffer#setSize(int)}, {@link ReadOnlySlicedNetworkBuffer#getSize()}. */ @Test - public void testGetSetSize2() { + void testGetSetSize2() { testGetSetSize(buffer.readOnlySlice(1, 2), 2); } private void testGetSetSize(ReadOnlySlicedNetworkBuffer slice, int sliceSize) { - assertEquals(DATA_SIZE, buffer.getSize()); - assertEquals(sliceSize, slice.getSize()); + assertThat(buffer.getSize()).isEqualTo(DATA_SIZE); + assertThat(slice.getSize()).isEqualTo(sliceSize); buffer.setSize(DATA_SIZE + 1); - assertEquals(DATA_SIZE + 1, buffer.getSize()); - assertEquals(sliceSize, slice.getSize()); + assertThat(buffer.getSize()).isEqualTo(DATA_SIZE + 1); + assertThat(slice.getSize()).isEqualTo(sliceSize); } @Test - public void testReadableBytes() { - assertEquals(buffer.readableBytes(), buffer.readOnlySlice().readableBytes()); - assertEquals(2, buffer.readOnlySlice(1, 2).readableBytes()); + void testReadableBytes() { + assertThat(buffer.readOnlySlice().readableBytes()).isEqualTo(buffer.readableBytes()); + assertThat(buffer.readOnlySlice(1, 2).readableBytes()).isEqualTo(2); } @Test - public void testGetNioBufferReadable1() { + void testGetNioBufferReadable1() { testGetNioBufferReadable(buffer.readOnlySlice(), DATA_SIZE); } @Test - public void testGetNioBufferReadable2() { + void testGetNioBufferReadable2() { testGetNioBufferReadable(buffer.readOnlySlice(1, 2), 2); } private void testGetNioBufferReadable(ReadOnlySlicedNetworkBuffer slice, int sliceSize) { ByteBuffer sliceByteBuffer = slice.getNioBufferReadable(); - assertTrue(sliceByteBuffer.isReadOnly()); - assertEquals(sliceSize, sliceByteBuffer.remaining()); - assertEquals(sliceSize, sliceByteBuffer.limit()); - assertEquals(sliceSize, sliceByteBuffer.capacity()); + assertThat(sliceByteBuffer.isReadOnly()).isTrue(); + assertThat(sliceByteBuffer.remaining()).isEqualTo(sliceSize); + assertThat(sliceByteBuffer.limit()).isEqualTo(sliceSize); + assertThat(sliceByteBuffer.capacity()).isEqualTo(sliceSize); // modify sliceByteBuffer position and verify nothing has changed in the original buffer sliceByteBuffer.position(1); - assertEquals(0, buffer.getReaderIndex()); - assertEquals(0, slice.getReaderIndex()); - assertEquals(DATA_SIZE, buffer.getSize()); - assertEquals(sliceSize, slice.getSize()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getSize()).isEqualTo(DATA_SIZE); + assertThat(slice.getSize()).isEqualTo(sliceSize); } @Test - public void testGetNioBuffer1() { + void testGetNioBuffer1() { testGetNioBuffer(buffer.readOnlySlice(), DATA_SIZE); } @Test - public void testGetNioBuffer2() { + void testGetNioBuffer2() { testGetNioBuffer(buffer.readOnlySlice(1, 2), 2); } private void testGetNioBuffer(ReadOnlySlicedNetworkBuffer slice, int sliceSize) { ByteBuffer sliceByteBuffer = slice.getNioBuffer(1, 1); - assertTrue(sliceByteBuffer.isReadOnly()); - assertEquals(1, sliceByteBuffer.remaining()); - assertEquals(1, sliceByteBuffer.limit()); - assertEquals(1, sliceByteBuffer.capacity()); + assertThat(sliceByteBuffer.isReadOnly()).isTrue(); + assertThat(sliceByteBuffer.remaining()).isEqualTo(1); + assertThat(sliceByteBuffer.limit()).isEqualTo(1); + assertThat(sliceByteBuffer.capacity()).isEqualTo(1); // modify sliceByteBuffer position and verify nothing has changed in the original buffer sliceByteBuffer.position(1); - assertEquals(0, buffer.getReaderIndex()); - assertEquals(0, slice.getReaderIndex()); - assertEquals(DATA_SIZE, buffer.getSize()); - assertEquals(sliceSize, slice.getSize()); + assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getSize()).isEqualTo(DATA_SIZE); + assertThat(slice.getSize()).isEqualTo(sliceSize); } @Test - public void testGetNioBufferReadableThreadSafe1() { + void testGetNioBufferReadableThreadSafe1() { NetworkBufferTest.testGetNioBufferReadableThreadSafe(buffer.readOnlySlice()); } @Test - public void testGetNioBufferReadableThreadSafe2() { + void testGetNioBufferReadableThreadSafe2() { NetworkBufferTest.testGetNioBufferReadableThreadSafe(buffer.readOnlySlice(1, 2)); } @Test - public void testGetNioBufferThreadSafe1() { + void testGetNioBufferThreadSafe1() { NetworkBufferTest.testGetNioBufferThreadSafe(buffer.readOnlySlice(), DATA_SIZE); } @Test - public void testGetNioBufferThreadSafe2() { + void testGetNioBufferThreadSafe2() { NetworkBufferTest.testGetNioBufferThreadSafe(buffer.readOnlySlice(1, 2), 2); } @Test - public void testForwardsSetAllocator() { + void testForwardsSetAllocator() { testForwardsSetAllocator(buffer.readOnlySlice()); testForwardsSetAllocator(buffer.readOnlySlice(1, 2)); } @@ -347,8 +343,8 @@ public void testForwardsSetAllocator() { private void testForwardsSetAllocator(ReadOnlySlicedNetworkBuffer slice) { NettyBufferPool allocator = new NettyBufferPool(1); slice.setAllocator(allocator); - assertSame(buffer.alloc(), slice.alloc()); - assertSame(allocator, slice.alloc()); + assertThat(slice.alloc()).isSameAs(buffer.alloc()); + assertThat(slice.alloc()).isSameAs(allocator); } private static void assertReadableBytes(Buffer actualBuffer, int... expectedBytes) { @@ -357,19 +353,19 @@ private static void assertReadableBytes(Buffer actualBuffer, int... expectedByte for (int i = 0; i < actual.length; ++i) { actual[i] = actualBytesBuffer.get(); } - assertArrayEquals(expectedBytes, actual); + assertThat(actual).isEqualTo(expectedBytes); // verify absolutely positioned read method: ByteBuf buffer = (ByteBuf) actualBuffer; for (int i = 0; i < buffer.readableBytes(); ++i) { actual[i] = buffer.getByte(buffer.readerIndex() + i); } - assertArrayEquals(expectedBytes, actual); + assertThat(actual).isEqualTo(expectedBytes); // verify relatively positioned read method: for (int i = 0; i < buffer.readableBytes(); ++i) { actual[i] = buffer.readByte(); } - assertArrayEquals(expectedBytes, actual); + assertThat(actual).isEqualTo(expectedBytes); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java index 4fcf083f3db2e..0960ef928f068 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ByteBufUtilsTest.java @@ -18,24 +18,20 @@ package org.apache.flink.runtime.io.network.netty; -import org.apache.flink.util.TestLogger; - import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; +import static org.assertj.core.api.Assertions.assertThat; /** Tests the methods in {@link ByteBufUtils}. */ -public class ByteBufUtilsTest extends TestLogger { +class ByteBufUtilsTest { private static final byte ACCUMULATION_BYTE = 0x7d; private static final byte NON_ACCUMULATION_BYTE = 0x23; @Test - public void testAccumulateWithoutCopy() { + void testAccumulateWithoutCopy() { int sourceLength = 128; int sourceReaderIndex = 32; int expectedAccumulationSize = 16; @@ -49,13 +45,13 @@ public void testAccumulateWithoutCopy() { ByteBufUtils.accumulate( target, src, expectedAccumulationSize, target.readableBytes()); - assertSame(src, accumulated); - assertEquals(sourceReaderIndex, src.readerIndex()); + assertThat(accumulated).isSameAs(src); + assertThat(src.readerIndex()).isEqualTo(sourceReaderIndex); verifyBufferContent(src, sourceReaderIndex, expectedAccumulationSize); } @Test - public void testAccumulateWithCopy() { + void testAccumulateWithCopy() { int sourceLength = 128; int firstSourceReaderIndex = 32; int secondSourceReaderIndex = 0; @@ -76,9 +72,9 @@ public void testAccumulateWithCopy() { ByteBuf accumulated = ByteBufUtils.accumulate( target, firstSource, expectedAccumulationSize, target.readableBytes()); - assertNull(accumulated); - assertEquals(sourceLength, firstSource.readerIndex()); - assertEquals(firstAccumulationSize, target.readableBytes()); + assertThat(accumulated).isNull(); + assertThat(firstSource.readerIndex()).isEqualTo(sourceLength); + assertThat(target.readableBytes()).isEqualTo(firstAccumulationSize); // The remaining data will be copied from the second buffer, and the target buffer will be // returned @@ -86,9 +82,10 @@ public void testAccumulateWithCopy() { accumulated = ByteBufUtils.accumulate( target, secondSource, expectedAccumulationSize, target.readableBytes()); - assertSame(target, accumulated); - assertEquals(secondSourceReaderIndex + secondAccumulationSize, secondSource.readerIndex()); - assertEquals(expectedAccumulationSize, target.readableBytes()); + assertThat(accumulated).isSameAs(target); + assertThat(secondSource.readerIndex()) + .isEqualTo(secondSourceReaderIndex + secondAccumulationSize); + assertThat(target.readableBytes()).isEqualTo(expectedAccumulationSize); verifyBufferContent(accumulated, 0, expectedAccumulationSize); } @@ -125,10 +122,9 @@ private ByteBuf createSourceBuffer(int size, int readerIndex, int accumulationSi private void verifyBufferContent(ByteBuf buf, int start, int length) { for (int i = 0; i < length; ++i) { byte b = buf.getByte(start + i); - assertEquals( - String.format("The byte at position %d is not right.", start + i), - ACCUMULATION_BYTE, - b); + assertThat(b) + .withFailMessage("The byte at position %d is not right.", start + i) + .isEqualTo(ACCUMULATION_BYTE); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java index 7bc7e9914caa2..bbe237efa3178 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java @@ -33,7 +33,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.Channel; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -49,7 +49,7 @@ import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.initServerAndClient; import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.shutdown; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; @@ -58,7 +58,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -public class CancelPartitionRequestTest { +class CancelPartitionRequestTest { /** * Verifies that requests for non-existing (failed/cancelled) input channels are properly @@ -66,7 +66,7 @@ public class CancelPartitionRequestTest { * This should cancel the request. */ @Test - public void testCancelPartitionRequest() throws Exception { + void testCancelPartitionRequest() throws Exception { NettyServerAndClient serverAndClient = null; @@ -123,7 +123,7 @@ public ResultSubpartitionView answer( } @Test - public void testDuplicateCancel() throws Exception { + void testDuplicateCancel() throws Exception { NettyServerAndClient serverAndClient = null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java index 33b903ff68899..86b6539ee052e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java @@ -24,55 +24,53 @@ import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link CreditBasedSequenceNumberingViewReader}. */ -public class CreditBasedSequenceNumberingViewReaderTest { +class CreditBasedSequenceNumberingViewReaderTest { @Test - public void testResumeConsumption() throws Exception { + void testResumeConsumption() throws Exception { int numCredits = 2; CreditBasedSequenceNumberingViewReader reader1 = createNetworkSequenceViewReader(numCredits); reader1.resumeConsumption(); - assertEquals(numCredits, reader1.getNumCreditsAvailable()); + assertThat(reader1.getNumCreditsAvailable()).isEqualTo(numCredits); reader1.addCredit(numCredits); reader1.resumeConsumption(); - assertEquals(2 * numCredits, reader1.getNumCreditsAvailable()); + assertThat(reader1.getNumCreditsAvailable()).isEqualTo(2 * numCredits); CreditBasedSequenceNumberingViewReader reader2 = createNetworkSequenceViewReader(0); reader2.addCredit(numCredits); - assertEquals(numCredits, reader2.getNumCreditsAvailable()); + assertThat(reader2.getNumCreditsAvailable()).isEqualTo(numCredits); reader2.resumeConsumption(); - assertEquals(0, reader2.getNumCreditsAvailable()); + assertThat(reader2.getNumCreditsAvailable()).isEqualTo(0); } @Test - public void testNeedAnnounceBacklog() throws Exception { + void testNeedAnnounceBacklog() throws Exception { int numCredits = 2; CreditBasedSequenceNumberingViewReader reader1 = createNetworkSequenceViewReader(numCredits); - assertFalse(reader1.needAnnounceBacklog()); + assertThat(reader1.needAnnounceBacklog()).isFalse(); reader1.addCredit(-numCredits); - assertFalse(reader1.needAnnounceBacklog()); + assertThat(reader1.needAnnounceBacklog()).isFalse(); CreditBasedSequenceNumberingViewReader reader2 = createNetworkSequenceViewReader(0); - assertTrue(reader2.needAnnounceBacklog()); + assertThat(reader2.needAnnounceBacklog()).isTrue(); reader2.addCredit(numCredits); - assertFalse(reader2.needAnnounceBacklog()); + assertThat(reader2.needAnnounceBacklog()).isFalse(); reader2.addCredit(-numCredits); - assertTrue(reader2.needAnnounceBacklog()); + assertThat(reader2.needAnnounceBacklog()).isTrue(); } private CreditBasedSequenceNumberingViewReader createNetworkSequenceViewReader( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java index 0aaa11ab64ccb..2b0a1d5dcfd2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java @@ -20,22 +20,21 @@ import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link NettyBufferPool} wrapper. */ -public class NettyBufferPoolTest { +class NettyBufferPoolTest { private final List needReleasing = new ArrayList<>(); - @After - public void tearDown() { + @AfterEach + void tearDown() { try { // Release all of the buffers. for (ByteBuf buf : needReleasing) { @@ -44,7 +43,7 @@ public void tearDown() { // Checks in a separate loop in case we have sliced buffers. for (ByteBuf buf : needReleasing) { - assertEquals(0, buf.refCnt()); + assertThat(buf.refCnt()).isEqualTo(0); } } finally { needReleasing.clear(); @@ -52,34 +51,36 @@ public void tearDown() { } @Test - public void testNoHeapAllocations() throws Exception { + void testNoHeapAllocations() { final NettyBufferPool nettyBufferPool = new NettyBufferPool(1); // Buffers should prefer to be direct - assertTrue(releaseLater(nettyBufferPool.buffer()).isDirect()); - assertTrue(releaseLater(nettyBufferPool.buffer(128)).isDirect()); - assertTrue(releaseLater(nettyBufferPool.buffer(128, 256)).isDirect()); + assertThat(releaseLater(nettyBufferPool.buffer()).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.buffer(128)).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.buffer(128, 256)).isDirect()).isTrue(); // IO buffers should prefer to be direct - assertTrue(releaseLater(nettyBufferPool.ioBuffer()).isDirect()); - assertTrue(releaseLater(nettyBufferPool.ioBuffer(128)).isDirect()); - assertTrue(releaseLater(nettyBufferPool.ioBuffer(128, 256)).isDirect()); + assertThat(releaseLater(nettyBufferPool.ioBuffer()).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.ioBuffer(128)).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.ioBuffer(128, 256)).isDirect()).isTrue(); // Currently we fakes the heap buffer allocation with direct buffers - assertTrue(releaseLater(nettyBufferPool.heapBuffer()).isDirect()); - assertTrue(releaseLater(nettyBufferPool.heapBuffer(128)).isDirect()); - assertTrue(releaseLater(nettyBufferPool.heapBuffer(128, 256)).isDirect()); + assertThat(releaseLater(nettyBufferPool.heapBuffer()).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.heapBuffer(128)).isDirect()).isTrue(); + assertThat(releaseLater(nettyBufferPool.heapBuffer(128, 256)).isDirect()).isTrue(); // Composite buffers allocates the corresponding type of buffers when extending its capacity - assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer()).capacity(1024).isDirect()); - assertTrue(releaseLater(nettyBufferPool.compositeHeapBuffer(10)).capacity(1024).isDirect()); + assertThat(releaseLater(nettyBufferPool.compositeHeapBuffer()).capacity(1024).isDirect()) + .isTrue(); + assertThat(releaseLater(nettyBufferPool.compositeHeapBuffer(10)).capacity(1024).isDirect()) + .isTrue(); // Is direct buffer pooled! - assertTrue(nettyBufferPool.isDirectBufferPooled()); + assertThat(nettyBufferPool.isDirectBufferPooled()).isTrue(); } @Test - public void testAllocationsStatistics() throws Exception { + void testAllocationsStatistics() throws Exception { NettyBufferPool nettyBufferPool = new NettyBufferPool(1); int chunkSize = nettyBufferPool.getChunkSize(); @@ -87,14 +88,14 @@ public void testAllocationsStatistics() throws Exception { // Single large buffer allocates one chunk releaseLater(nettyBufferPool.directBuffer(chunkSize - 64)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); - assertEquals(chunkSize, allocated); + assertThat(allocated).isEqualTo(chunkSize); } { // Allocate a little more (one more chunk required) releaseLater(nettyBufferPool.directBuffer(128)); long allocated = nettyBufferPool.getNumberOfAllocatedBytes().get(); - assertEquals(2 * chunkSize, allocated); + assertThat(allocated).isEqualTo(2 * chunkSize); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java index 58ba3e4785a52..4210aa7e41e32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java @@ -24,8 +24,10 @@ import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.io.network.netty.NettyTestUtil.NettyServerAndClient; import org.apache.flink.runtime.net.SSLUtilsTest; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.util.NetUtils; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel; @@ -33,10 +35,8 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.string.StringEncoder; import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import javax.net.ssl.SSLSessionContext; @@ -48,31 +48,29 @@ import static org.apache.flink.configuration.SecurityOptions.SSL_INTERNAL_HANDSHAKE_TIMEOUT; import static org.apache.flink.configuration.SecurityOptions.SSL_INTERNAL_SESSION_CACHE_SIZE; import static org.apache.flink.configuration.SecurityOptions.SSL_INTERNAL_SESSION_TIMEOUT; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** Tests for the SSL connection between Netty Server and Client used for the data plane. */ -@RunWith(Parameterized.class) -public class NettyClientServerSslTest extends TestLogger { +@ExtendWith(ParameterizedTestExtension.class) +public class NettyClientServerSslTest { - @Parameterized.Parameter public String sslProvider; + @Parameter public String sslProvider; - @Parameterized.Parameters(name = "SSL provider = {0}") + @Parameters(name = "SSL provider = {0}") public static List parameters() { return SSLUtilsTest.AVAILABLE_SSL_PROVIDERS; } /** Verify valid ssl configuration and connection. */ - @Test - public void testValidSslConnection() throws Exception { + @TestTemplate + void testValidSslConnection() throws Exception { testValidSslConnection(createSslConfig()); } /** Verify valid (advanced) ssl configuration and connection. */ - @Test - public void testValidSslConnectionAdvanced() throws Exception { + @TestTemplate + void testValidSslConnectionAdvanced() throws Exception { Configuration sslConfig = createSslConfig(); sslConfig.setInteger(SSL_INTERNAL_SESSION_CACHE_SIZE, 1); @@ -110,8 +108,9 @@ private void testValidSslConnection(Configuration sslConfig) throws Exception { final NettyClient client = NettyTestUtil.initClient(nettyConfig, protocol, bufferPool); serverAndClient = new NettyServerAndClient(server, client); } - Assert.assertNotNull( - "serverAndClient is null due to fail to get a free port", serverAndClient); + assertThat(serverAndClient) + .withFailMessage("serverAndClient is null due to fail to get a free port") + .isNotNull(); Channel ch = NettyTestUtil.connect(serverAndClient); @@ -132,7 +131,7 @@ private void testValidSslConnection(Configuration sslConfig) throws Exception { // session context is only be available after a session was setup -> this should be true // after data was sent serverChannelInitComplete.await(); - assertNotNull(serverSslHandler[0]); + assertThat(serverSslHandler[0]).isNotNull(); // verify server parameters assertEqualsOrDefault( @@ -145,7 +144,9 @@ private void testValidSslConnection(Configuration sslConfig) throws Exception { serverSslHandler[0].getCloseNotifyFlushTimeoutMillis()); SSLSessionContext sessionContext = serverSslHandler[0].engine().getSession().getSessionContext(); - assertNotNull("bug in unit test setup: session context not available", sessionContext); + assertThat(sessionContext) + .withFailMessage("bug in unit test setup: session context not available") + .isNotNull(); // note: can't verify session cache setting at the client - delegate to server instead (with // our own channel initializer) assertEqualsOrDefault( @@ -153,11 +154,11 @@ private void testValidSslConnection(Configuration sslConfig) throws Exception { int sessionTimeout = sslConfig.getInteger(SSL_INTERNAL_SESSION_TIMEOUT); if (sessionTimeout != -1) { // session timeout config is in milliseconds but the context returns it in seconds - assertEquals(sessionTimeout / 1000, sessionContext.getSessionTimeout()); + assertThat(sessionContext.getSessionTimeout()).isEqualTo(sessionTimeout / 1000); } else { - assertTrue( - "default value (-1) should not be propagated", - sessionContext.getSessionTimeout() >= 0); + assertThat(sessionContext.getSessionTimeout()) + .withFailMessage("default value (-1) should not be propagated") + .isGreaterThanOrEqualTo(0); } NettyTestUtil.shutdown(serverAndClient); @@ -167,16 +168,17 @@ private static void assertEqualsOrDefault( Configuration sslConfig, ConfigOption option, long actual) { long expected = sslConfig.getInteger(option); if (expected != option.defaultValue()) { - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } else { - assertTrue( - "default value (" + option.defaultValue() + ") should not be propagated", - actual >= 0); + assertThat(actual) + .withFailMessage( + "default value (%d) should not be propagated", option.defaultValue()) + .isGreaterThanOrEqualTo(0); } } /** Verify failure on invalid ssl configuration. */ - @Test + @TestTemplate public void testInvalidSslConfiguration() throws Exception { NettyProtocol protocol = new NettyTestUtil.NoOpProtocol(); @@ -189,7 +191,7 @@ public void testInvalidSslConfiguration() throws Exception { NettyConfig nettyConfig = createNettyConfig(config, port); serverAndClient = NettyTestUtil.initServerAndClient(protocol, nettyConfig); - Assert.fail("Created server and client from invalid configuration"); + fail("Created server and client from invalid configuration"); } catch (Exception e) { // Exception should be thrown as expected } @@ -198,8 +200,8 @@ public void testInvalidSslConfiguration() throws Exception { } /** Verify SSL handshake error when untrusted server certificate is used. */ - @Test - public void testSslHandshakeError() throws Exception { + @TestTemplate + void testSslHandshakeError() throws Exception { NettyProtocol protocol = new NettyTestUtil.NoOpProtocol(); Configuration config = createSslConfig(); @@ -214,19 +216,20 @@ public void testSslHandshakeError() throws Exception { serverAndClient = NettyTestUtil.initServerAndClient(protocol, nettyConfig); } - Assert.assertNotNull( - "serverAndClient is null due to fail to get a free port", serverAndClient); + assertThat(serverAndClient) + .withFailMessage("serverAndClient is null due to fail to get a free port") + .isNotNull(); Channel ch = NettyTestUtil.connect(serverAndClient); ch.pipeline().addLast(new StringDecoder()).addLast(new StringEncoder()); // Attempting to write data over ssl should fail - assertFalse(ch.writeAndFlush("test").await().isSuccess()); + assertThat(ch.writeAndFlush("test").await().isSuccess()).isFalse(); NettyTestUtil.shutdown(serverAndClient); } - @Test - public void testClientUntrustedCertificate() throws Exception { + @TestTemplate + void testClientUntrustedCertificate() throws Exception { final Configuration serverConfig = createSslConfig(); final Configuration clientConfig = createSslConfig(); @@ -249,20 +252,21 @@ public void testClientUntrustedCertificate() throws Exception { NettyTestUtil.initClient(nettyClientConfig, protocol, bufferPool); serverAndClient = new NettyServerAndClient(server, client); } - Assert.assertNotNull( - "serverAndClient is null due to fail to get a free port", serverAndClient); + assertThat(serverAndClient) + .withFailMessage("serverAndClient is null due to fail to get a free port") + .isNotNull(); final Channel ch = NettyTestUtil.connect(serverAndClient); ch.pipeline().addLast(new StringDecoder()).addLast(new StringEncoder()); // Attempting to write data over ssl should fail - assertFalse(ch.writeAndFlush("test").await().isSuccess()); + assertThat(ch.writeAndFlush("test").await().isSuccess()).isFalse(); NettyTestUtil.shutdown(serverAndClient); } - @Test - public void testSslPinningForValidFingerprint() throws Exception { + @TestTemplate + void testSslPinningForValidFingerprint() throws Exception { NettyProtocol protocol = new NettyTestUtil.NoOpProtocol(); Configuration config = createSslConfig(); @@ -277,19 +281,20 @@ public void testSslPinningForValidFingerprint() throws Exception { serverAndClient = NettyTestUtil.initServerAndClient(protocol, nettyConfig); } - Assert.assertNotNull( - "serverAndClient is null due to fail to get a free port", serverAndClient); + assertThat(serverAndClient) + .withFailMessage("serverAndClient is null due to fail to get a free port") + .isNotNull(); Channel ch = NettyTestUtil.connect(serverAndClient); ch.pipeline().addLast(new StringDecoder()).addLast(new StringEncoder()); - assertTrue(ch.writeAndFlush("test").await().isSuccess()); + assertThat(ch.writeAndFlush("test").await().isSuccess()).isTrue(); NettyTestUtil.shutdown(serverAndClient); } - @Test - public void testSslPinningForInvalidFingerprint() throws Exception { + @TestTemplate + void testSslPinningForInvalidFingerprint() throws Exception { NettyProtocol protocol = new NettyTestUtil.NoOpProtocol(); Configuration config = createSslConfig(); @@ -305,13 +310,14 @@ public void testSslPinningForInvalidFingerprint() throws Exception { serverAndClient = NettyTestUtil.initServerAndClient(protocol, nettyConfig); } - Assert.assertNotNull( - "serverAndClient is null due to fail to get a free port", serverAndClient); + assertThat(serverAndClient) + .withFailMessage("serverAndClient is null due to fail to get a free port") + .isNotNull(); Channel ch = NettyTestUtil.connect(serverAndClient); ch.pipeline().addLast(new StringDecoder()).addLast(new StringEncoder()); - assertFalse(ch.writeAndFlush("test").await().isSuccess()); + assertThat(ch.writeAndFlush("test").await().isSuccess()).isFalse(); NettyTestUtil.shutdown(serverAndClient); } @@ -354,7 +360,7 @@ public void initChannel(SocketChannel channel) throws Exception { super.initChannel(channel); SslHandler sslHandler = (SslHandler) channel.pipeline().get("ssl"); - assertNotNull(sslHandler); + assertThat(sslHandler).isNotNull(); serverHandler[0] = sslHandler; latch.trigger(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java index f93381db59dce..85015690d69a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java @@ -27,23 +27,22 @@ import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap; import org.apache.flink.shaded.netty4.io.netty.channel.EventLoopGroup; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.lang.reflect.Field; import java.net.InetAddress; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.assertj.core.api.Assertions.assertThat; /** Simple netty connection manager test. */ -public class NettyConnectionManagerTest { +class NettyConnectionManagerTest { /** * Tests that the number of arenas and number of threads of the client and server are set to the * same number, that is the number of configured task slots. */ @Test - public void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { + void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { // Expected number of arenas and threads int numberOfSlots = 2; NettyConnectionManager connectionManager; @@ -59,10 +58,11 @@ public void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { connectionManager = createNettyConnectionManager(config); connectionManager.start(); } - assertNotNull( - "connectionManager is null due to fail to get a free port", connectionManager); + assertThat(connectionManager) + .withFailMessage("connectionManager is null due to fail to get a free port") + .isNotNull(); - assertEquals(numberOfSlots, connectionManager.getBufferPool().getNumberOfArenas()); + assertThat(connectionManager.getBufferPool().getNumberOfArenas()).isEqualTo(numberOfSlots); { // Client event loop group @@ -73,7 +73,7 @@ public void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfSlots, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfSlots); } { @@ -85,7 +85,7 @@ public void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfSlots, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfSlots); } { @@ -97,13 +97,13 @@ public void testMatchingNumberOfArenasAndThreadsAsDefault() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfSlots, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfSlots); } } /** Tests that the number of arenas and threads can be configured manually. */ @Test - public void testManualConfiguration() throws Exception { + void testManualConfiguration() throws Exception { // Expected numbers int numberOfArenas = 1; int numberOfClientThreads = 3; @@ -123,10 +123,12 @@ public void testManualConfiguration() throws Exception { connectionManager = createNettyConnectionManager(config); connectionManager.start(); - assertEquals(numberOfArenas, connectionManager.getBufferPool().getNumberOfArenas()); + assertThat(connectionManager.getBufferPool().getNumberOfArenas()) + .isEqualTo(numberOfArenas); } - assertNotNull( - "connectionManager is null due to fail to get a free port", connectionManager); + assertThat(connectionManager) + .withFailMessage("connectionManager is null due to fail to get a free port") + .isNotNull(); { // Client event loop group @@ -137,7 +139,7 @@ public void testManualConfiguration() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfClientThreads, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfClientThreads); } { @@ -149,7 +151,7 @@ public void testManualConfiguration() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfServerThreads, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfServerThreads); } { @@ -161,7 +163,7 @@ public void testManualConfiguration() throws Exception { f.setAccessible(true); Object[] eventExecutors = (Object[]) f.get(group); - assertEquals(numberOfServerThreads, eventExecutors.length); + assertThat(eventExecutors).hasSize(numberOfServerThreads); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionExtension.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionExtension.java index 9036f0fb2d427..4fc3d3c272fae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionExtension.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyLeakDetectionExtension.java @@ -29,7 +29,7 @@ import javax.annotation.concurrent.GuardedBy; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.fail; +import static org.assertj.core.api.Assertions.fail; /** * JUnit5 extension to fail with an assertion when Netty detects a resource leak (only with diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java index 801e478b3404c..ceb4b63bfa6bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java @@ -30,14 +30,13 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import javax.annotation.Nullable; @@ -45,16 +44,14 @@ 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.NettyTestUtil.verifyBufferResponseHeader; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; -import static org.junit.Assert.assertNull; +import static org.assertj.core.api.Assertions.assertThat; /** Tests the client side message decoder. */ -public class NettyMessageClientDecoderDelegateTest extends TestLogger { +class NettyMessageClientDecoderDelegateTest { private static final int BUFFER_SIZE = 1024; @@ -72,8 +69,8 @@ public class NettyMessageClientDecoderDelegateTest extends TestLogger { private InputChannelID releasedInputChannelId; - @Before - public void setup() throws IOException, InterruptedException { + @BeforeEach + void setup() throws IOException, InterruptedException { CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); networkBufferPool = new NetworkBufferPool(NUMBER_OF_BUFFER_RESPONSES, BUFFER_SIZE); @@ -97,8 +94,8 @@ public void setup() throws IOException, InterruptedException { releasedInputChannelId = releasedInputChannel.getInputChannelId(); } - @After - public void tearDown() throws IOException { + @AfterEach + void tearDown() throws IOException { if (inputGate != null) { inputGate.close(); } @@ -115,7 +112,7 @@ public void tearDown() throws IOException { /** Verifies that the client side decoder works well for unreleased input channels. */ @Test - public void testClientMessageDecode() throws Exception { + void testClientMessageDecode() throws Exception { testNettyMessageClientDecoding(false, false, false); } @@ -124,7 +121,7 @@ public void testClientMessageDecode() throws Exception { * consume data buffers of the input channels. */ @Test - public void testClientMessageDecodeWithEmptyBuffers() throws Exception { + void testClientMessageDecodeWithEmptyBuffers() throws Exception { testNettyMessageClientDecoding(true, false, false); } @@ -133,7 +130,7 @@ public void testClientMessageDecodeWithEmptyBuffers() throws Exception { * channel. The data buffer part should be discarded before reading the next message. */ @Test - public void testClientMessageDecodeWithReleasedInputChannel() throws Exception { + void testClientMessageDecodeWithReleasedInputChannel() throws Exception { testNettyMessageClientDecoding(false, true, false); } @@ -142,7 +139,7 @@ public void testClientMessageDecodeWithReleasedInputChannel() throws Exception { * channel. The data buffer part should be discarded before reading the next message. */ @Test - public void testClientMessageDecodeWithRemovedInputChannel() throws Exception { + void testClientMessageDecodeWithRemovedInputChannel() throws Exception { testNettyMessageClientDecoding(false, false, true); } @@ -318,7 +315,7 @@ private List decodeMessages(EmbeddedChannel channel, List List decodedMessages = new ArrayList<>(); Object input; while ((input = channel.readInbound()) != null) { - assertTrue(input instanceof NettyMessage); + assertThat(input).isInstanceOf(NettyMessage.class); decodedMessages.add((NettyMessage) input); } @@ -327,17 +324,17 @@ private List decodeMessages(EmbeddedChannel channel, List private void verifyDecodedMessages( List expectedMessages, List decodedMessages) { - assertEquals(expectedMessages.size(), decodedMessages.size()); + assertThat(decodedMessages.size()).isEqualTo(expectedMessages.size()); for (int i = 0; i < expectedMessages.size(); ++i) { - assertEquals(expectedMessages.get(i).getClass(), decodedMessages.get(i).getClass()); + assertThat(decodedMessages.get(i)).isInstanceOf(expectedMessages.get(i).getClass()); BufferResponse expected = expectedMessages.get(i); BufferResponse actual = (BufferResponse) decodedMessages.get(i); verifyBufferResponseHeader(expected, actual); if (expected.bufferSize == 0 || !expected.receiverId.equals(inputChannelId)) { - assertNull(actual.getBuffer()); + assertThat(actual.getBuffer()).isNull(); } else { - assertEquals(expected.getBuffer(), actual.getBuffer()); + assertThat(actual.getBuffer()).isEqualTo(expected.getBuffer()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java index b6927e5c1ece0..153836415b30d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java @@ -30,14 +30,12 @@ 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.util.TestLoggerExtension; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -60,7 +58,6 @@ * Tests for the serialization and deserialization of the various {@link NettyMessage} sub-classes * sent from server side to client side. */ -@ExtendWith(TestLoggerExtension.class) class NettyMessageClientSideSerializationTest { private static final int BUFFER_SIZE = 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java index 3aedfa3bd5173..57f20e4c61f43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageServerSideSerializationTest.java @@ -21,46 +21,45 @@ import org.apache.flink.runtime.event.task.IntegerTaskEvent; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.Random; import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.encodeAndDecode; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for the serialization and deserialization of the various {@link NettyMessage} sub-classes * sent from client side to server side. */ -public class NettyMessageServerSideSerializationTest extends TestLogger { +class NettyMessageServerSideSerializationTest { private final Random random = new Random(); private EmbeddedChannel channel; - @Before - public void setup() { + @BeforeEach + void setup() { channel = new EmbeddedChannel( new NettyMessage.NettyMessageEncoder(), // For outbound messages new NettyMessage.NettyMessageDecoder()); // For inbound messages } - @After - public void tearDown() { + @AfterEach + void tearDown() { if (channel != null) { channel.close(); } } @Test - public void testPartitionRequest() { + void testPartitionRequest() { NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest( new ResultPartitionID(), @@ -70,14 +69,14 @@ public void testPartitionRequest() { NettyMessage.PartitionRequest actual = encodeAndDecode(expected, channel); - assertEquals(expected.partitionId, actual.partitionId); - assertEquals(expected.queueIndex, actual.queueIndex); - assertEquals(expected.receiverId, actual.receiverId); - assertEquals(expected.credit, actual.credit); + assertThat(actual.partitionId).isEqualTo(expected.partitionId); + assertThat(actual.queueIndex).isEqualTo(expected.queueIndex); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); + assertThat(actual.credit).isEqualTo(expected.credit); } @Test - public void testTaskEventRequest() { + void testTaskEventRequest() { NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest( new IntegerTaskEvent(random.nextInt()), @@ -85,65 +84,65 @@ public void testTaskEventRequest() { new InputChannelID()); NettyMessage.TaskEventRequest actual = encodeAndDecode(expected, channel); - assertEquals(expected.event, actual.event); - assertEquals(expected.partitionId, actual.partitionId); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.event).isEqualTo(expected.event); + assertThat(actual.partitionId).isEqualTo(expected.partitionId); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } @Test - public void testCancelPartitionRequest() { + void testCancelPartitionRequest() { NettyMessage.CancelPartitionRequest expected = new NettyMessage.CancelPartitionRequest(new InputChannelID()); NettyMessage.CancelPartitionRequest actual = encodeAndDecode(expected, channel); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } @Test - public void testCloseRequest() { + void testCloseRequest() { NettyMessage.CloseRequest expected = new NettyMessage.CloseRequest(); NettyMessage.CloseRequest actual = encodeAndDecode(expected, channel); - assertEquals(expected.getClass(), actual.getClass()); + assertThat(actual).isExactlyInstanceOf(expected.getClass()); } @Test - public void testAddCredit() { + void testAddCredit() { NettyMessage.AddCredit expected = new NettyMessage.AddCredit( random.nextInt(Integer.MAX_VALUE) + 1, new InputChannelID()); NettyMessage.AddCredit actual = encodeAndDecode(expected, channel); - assertEquals(expected.credit, actual.credit); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.credit).isEqualTo(expected.credit); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } @Test - public void testResumeConsumption() { + void testResumeConsumption() { NettyMessage.ResumeConsumption expected = new NettyMessage.ResumeConsumption(new InputChannelID()); NettyMessage.ResumeConsumption actual = encodeAndDecode(expected, channel); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } @Test - public void testAckAllUserRecordsProcessed() { + void testAckAllUserRecordsProcessed() { NettyMessage.AckAllUserRecordsProcessed expected = new NettyMessage.AckAllUserRecordsProcessed(new InputChannelID()); NettyMessage.AckAllUserRecordsProcessed actual = encodeAndDecode(expected, channel); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } @Test - public void testNewBufferSize() { + void testNewBufferSize() { NettyMessage.NewBufferSize expected = new NettyMessage.NewBufferSize( random.nextInt(Integer.MAX_VALUE), new InputChannelID()); NettyMessage.NewBufferSize actual = encodeAndDecode(expected, channel); - assertEquals(expected.bufferSize, actual.bufferSize); - assertEquals(expected.receiverId, actual.receiverId); + assertThat(actual.bufferSize).isEqualTo(expected.bufferSize); + assertThat(actual.receiverId).isEqualTo(expected.receiverId); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index ad34bdf363e85..fd532cd8996b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -30,13 +30,15 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputChannelBuilder; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -44,25 +46,20 @@ import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.mockConnectionManagerWithPartitionRequestClient; -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link NettyPartitionRequestClient}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class NettyPartitionRequestClientTest { - @Parameterized.Parameter public boolean connectionReuseEnabled; + @Parameter public boolean connectionReuseEnabled; - @Parameterized.Parameters(name = "connection reuse enabled = {0}") + @Parameters(name = "connection reuse enabled = {0}") public static Object[] parameters() { return new Object[][] {new Object[] {true}, new Object[] {false}}; } - @Test - public void testPartitionRequestClientReuse() throws Exception { + @TestTemplate + void testPartitionRequestClientReuse() throws Exception { final CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); final EmbeddedChannel channel = new EmbeddedChannel(handler); @@ -76,11 +73,11 @@ public void testPartitionRequestClientReuse() throws Exception { try { // Client should not be disposed in idle client.close(inputChannel); - assertFalse(client.canBeDisposed()); + assertThat(client.canBeDisposed()).isFalse(); // Client should be disposed in error handler.notifyAllChannelsOfErrorAndClose(new RuntimeException()); - assertTrue(client.canBeDisposed()); + assertThat(client.canBeDisposed()).isTrue(); } finally { // Release all the buffer resources inputGate.close(); @@ -90,8 +87,8 @@ public void testPartitionRequestClientReuse() throws Exception { } } - @Test - public void testRetriggerPartitionRequest() throws Exception { + @TestTemplate + void testRetriggerPartitionRequest() throws Exception { final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs final CreditBasedPartitionRequestClientHandler handler = @@ -120,13 +117,12 @@ public void testRetriggerPartitionRequest() throws Exception { // first subpartition request inputChannel.requestSubpartition(); - assertTrue(channel.isWritable()); + assertThat(channel.isWritable()).isTrue(); Object readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((PartitionRequest) readFromOutbound).receiverId); - assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); + assertThat(((PartitionRequest) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); + assertThat(((PartitionRequest) readFromOutbound).credit).isEqualTo(numExclusiveBuffers); // retrigger subpartition request, e.g. due to failures inputGate.retriggerPartitionRequest( @@ -135,11 +131,10 @@ public void testRetriggerPartitionRequest() throws Exception { runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((PartitionRequest) readFromOutbound).receiverId); - assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); + assertThat(((PartitionRequest) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); + assertThat(((PartitionRequest) readFromOutbound).credit).isEqualTo(numExclusiveBuffers); // retrigger subpartition request once again, e.g. due to failures inputGate.retriggerPartitionRequest( @@ -148,13 +143,12 @@ public void testRetriggerPartitionRequest() throws Exception { runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((PartitionRequest) readFromOutbound).receiverId); - assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); + assertThat(((PartitionRequest) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); + assertThat(((PartitionRequest) readFromOutbound).credit).isEqualTo(numExclusiveBuffers); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); } finally { // Release all the buffer resources inputGate.close(); @@ -164,8 +158,8 @@ public void testRetriggerPartitionRequest() throws Exception { } } - @Test - public void testDoublePartitionRequest() throws Exception { + @TestTemplate + void testDoublePartitionRequest() throws Exception { final CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); final EmbeddedChannel channel = new EmbeddedChannel(handler); @@ -185,15 +179,14 @@ public void testDoublePartitionRequest() throws Exception { inputChannel.requestSubpartition(); // The input channel should only send one partition request - assertTrue(channel.isWritable()); + assertThat(channel.isWritable()).isTrue(); Object readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((PartitionRequest) readFromOutbound).receiverId); - assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); + assertThat(((PartitionRequest) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); + assertThat(((PartitionRequest) readFromOutbound).credit).isEqualTo(numExclusiveBuffers); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); } finally { // Release all the buffer resources inputGate.close(); @@ -203,8 +196,8 @@ public void testDoublePartitionRequest() throws Exception { } } - @Test - public void testResumeConsumption() throws Exception { + @TestTemplate + void testResumeConsumption() throws Exception { final CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); final EmbeddedChannel channel = new EmbeddedChannel(handler); @@ -224,15 +217,14 @@ public void testResumeConsumption() throws Exception { inputChannel.resumeConsumption(); channel.runPendingTasks(); Object readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(ResumeConsumption.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((ResumeConsumption) readFromOutbound).receiverId); + assertThat(readFromOutbound).isInstanceOf(ResumeConsumption.class); + assertThat(((ResumeConsumption) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); } finally { // Release all the buffer resources inputGate.close(); @@ -242,8 +234,8 @@ public void testResumeConsumption() throws Exception { } } - @Test - public void testAcknowledgeAllRecordsProcessed() throws Exception { + @TestTemplate + void testAcknowledgeAllRecordsProcessed() throws Exception { CreditBasedPartitionRequestClientHandler handler = new CreditBasedPartitionRequestClientHandler(); EmbeddedChannel channel = new EmbeddedChannel(handler); @@ -263,15 +255,15 @@ public void testAcknowledgeAllRecordsProcessed() throws Exception { inputChannel.acknowledgeAllRecordsProcessed(); channel.runPendingTasks(); Object readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(PartitionRequest.class)); + assertThat(readFromOutbound).isInstanceOf(PartitionRequest.class); readFromOutbound = channel.readOutbound(); - assertThat(readFromOutbound, instanceOf(NettyMessage.AckAllUserRecordsProcessed.class)); - assertEquals( - inputChannel.getInputChannelId(), - ((NettyMessage.AckAllUserRecordsProcessed) readFromOutbound).receiverId); + assertThat(readFromOutbound) + .isInstanceOf(NettyMessage.AckAllUserRecordsProcessed.class); + assertThat(((NettyMessage.AckAllUserRecordsProcessed) readFromOutbound).receiverId) + .isEqualTo(inputChannel.getInputChannelId()); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); } finally { // Release all the buffer resources inputGate.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java index 42dc38c8e30ce..bbff50ef16b1c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyTestUtil.java @@ -40,7 +40,7 @@ import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Test utility for Netty server and client setup. */ public class NettyTestUtil { @@ -198,7 +198,7 @@ static T encodeAndDecode(T msg, EmbeddedChannel channel while ((encoded = channel.readOutbound()) != null) { msgNotEmpty = channel.writeInbound(encoded); } - assertTrue(msgNotEmpty); + assertThat(msgNotEmpty).isTrue(); return channel.readInbound(); } @@ -213,7 +213,7 @@ static void verifyErrorResponse(ErrorResponse expected, ErrorResponse actual) { assertEquals(expected.cause.getMessage(), actual.cause.getMessage()); if (expected.receiverId == null) { - assertTrue(actual.isFatalError()); + assertThat(actual.isFatalError()).isTrue(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java index d923d76a6c908..0b3e206576ecb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java @@ -25,7 +25,6 @@ import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelException; @@ -55,12 +54,12 @@ import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.shutdown; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.Assertions.fail; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Mockito.mock; /** {@link PartitionRequestClientFactory} test. */ @ExtendWith(ParameterizedTestExtension.class) -public class PartitionRequestClientFactoryTest extends TestLogger { +public class PartitionRequestClientFactoryTest { private static final ResourceID RESOURCE_ID = ResourceID.generate(); @Parameter public boolean connectionReuseEnabled; @@ -317,8 +316,7 @@ void testNettyClientConnectRetryMultipleThread() throws Exception { client = runnableFuture.get(); assertThat(client).isNotNull(); } catch (Exception e) { - System.out.println(e.getMessage()); - fail(); + fail(e.getMessage()); } }); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java index 34b949742db62..718d8af0f0a86 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java @@ -45,48 +45,38 @@ import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEventBufferConsumer; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link PartitionRequestQueue}. */ -public class PartitionRequestQueueTest { - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +class PartitionRequestQueueTest { private static final int BUFFER_SIZE = 1024 * 1024; private static FileChannelManager fileChannelManager; - @BeforeClass - public static void setUp() throws Exception { + @BeforeAll + static void setUp(@TempDir File temporaryFolder) throws Exception { fileChannelManager = new FileChannelManagerImpl( - new String[] {TEMPORARY_FOLDER.newFolder().getAbsolutePath()}, "testing"); + new String[] {temporaryFolder.getAbsolutePath()}, "testing"); } - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { fileChannelManager.close(); } @@ -96,7 +86,7 @@ public static void shutdown() throws Exception { * messages. */ @Test - public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { + void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { final int buffersToWrite = 5; PartitionRequestQueue queue = new PartitionRequestQueue(); EmbeddedChannel channel = new EmbeddedChannel(queue); @@ -112,11 +102,11 @@ public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { new ResultPartitionID(), 0); reader1.notifyDataAvailable(); - assertTrue(reader1.getAvailabilityAndBacklog().isAvailable()); - assertFalse(reader1.isRegisteredAsAvailable()); + assertThat(reader1.getAvailabilityAndBacklog().isAvailable()).isTrue(); + assertThat(reader1.isRegisteredAsAvailable()).isFalse(); channel.unsafe().outboundBuffer().setUserDefinedWritability(1, false); - assertFalse(channel.isWritable()); + assertThat(channel.isWritable()).isFalse(); reader1.notifyDataAvailable(); channel.runPendingTasks(); @@ -127,26 +117,26 @@ public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { new DefaultBufferResultSubpartitionView(buffersToWrite), new ResultPartitionID(), 0); - assertTrue(reader2.getAvailabilityAndBacklog().isAvailable()); - assertFalse(reader2.isRegisteredAsAvailable()); + assertThat(reader2.getAvailabilityAndBacklog().isAvailable()).isTrue(); + assertThat(reader2.isRegisteredAsAvailable()).isFalse(); reader2.notifyDataAvailable(); // changing a channel writability should result in draining both reader1 and reader2 channel.unsafe().outboundBuffer().setUserDefinedWritability(1, true); channel.runPendingTasks(); - assertEquals(buffersToWrite, channel.outboundMessages().size()); + assertThat(channel.outboundMessages()).hasSize(buffersToWrite); } /** Tests {@link PartitionRequestQueue} buffer writing with default buffers. */ @Test - public void testDefaultBufferWriting() throws Exception { + void testDefaultBufferWriting() throws Exception { testBufferWriting(new DefaultBufferResultSubpartitionView(1)); } /** Tests {@link PartitionRequestQueue} buffer writing with read-only buffers. */ @Test - public void testReadOnlyBufferWriting() throws Exception { + void testReadOnlyBufferWriting() throws Exception { testBufferWriting(new ReadOnlyBufferResultSubpartitionView(1)); } @@ -169,13 +159,13 @@ private void testBufferWriting(ResultSubpartitionView view) throws IOException { channel.runPendingTasks(); Object read = channel.readOutbound(); - assertNotNull(read); + assertThat(read).isNotNull(); if (read instanceof NettyMessage.ErrorResponse) { ((NettyMessage.ErrorResponse) read).cause.printStackTrace(); } - assertThat(read, instanceOf(NettyMessage.BufferResponse.class)); + assertThat(read).isInstanceOf(NettyMessage.BufferResponse.class); read = channel.readOutbound(); - assertNull(read); + assertThat(read).isNull(); } private static class DefaultBufferResultSubpartitionView extends NoOpResultSubpartitionView { @@ -242,7 +232,7 @@ public AvailabilityWithBacklog getAvailabilityAndBacklog(int numCreditsAvailable * even though it has no available credits. */ @Test - public void testEnqueueReaderByNotifyingEventBuffer() throws Exception { + void testEnqueueReaderByNotifyingEventBuffer() throws Exception { // setup final ResultSubpartitionView view = new NextIsEventResultSubpartitionView(); @@ -259,7 +249,7 @@ public void testEnqueueReaderByNotifyingEventBuffer() throws Exception { // block the channel so that we see an intermediate state in the test ByteBuf channelBlockingBuffer = blockChannel(channel); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); // Notify an available event buffer to trigger enqueue the reader reader.notifyDataAvailable(); @@ -268,16 +258,16 @@ public void testEnqueueReaderByNotifyingEventBuffer() throws Exception { // The reader is enqueued in the pipeline because the next buffer is an event, even though // no credits are available - assertThat(queue.getAvailableReaders(), contains(reader)); // contains only (this) one! - assertEquals(0, reader.getNumCreditsAvailable()); + assertThat(queue.getAvailableReaders()).contains(reader); // contains only (this) one! + assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); // Flush the buffer to make the channel writable again and see the final results channel.flush(); - assertSame(channelBlockingBuffer, channel.readOutbound()); + assertThat((ByteBuf) channel.readOutbound()).isSameAs(channelBlockingBuffer); - assertEquals(0, queue.getAvailableReaders().size()); - assertEquals(0, reader.getNumCreditsAvailable()); - assertNull(channel.readOutbound()); + assertThat(queue.getAvailableReaders()).isEmpty(); + assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat((Object) channel.readOutbound()).isNull(); } private static class NextIsEventResultSubpartitionView extends NoOpResultSubpartitionView { @@ -293,7 +283,7 @@ public AvailabilityWithBacklog getAvailabilityAndBacklog(int numCreditsAvailable * buffers. */ @Test - public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { + void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { // setup final ResultSubpartitionView view = new DefaultBufferResultSubpartitionView(10); @@ -312,7 +302,7 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { // block the channel so that we see an intermediate state in the test ByteBuf channelBlockingBuffer = blockChannel(channel); - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); // Notify available buffers to trigger enqueue the reader final int notifyNumBuffers = 5; @@ -324,10 +314,10 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { // the reader is not enqueued in the pipeline because no credits are available // -> it should still have the same number of pending buffers - assertEquals(0, queue.getAvailableReaders().size()); - assertTrue(reader.hasBuffersAvailable().isAvailable()); - assertFalse(reader.isRegisteredAsAvailable()); - assertEquals(0, reader.getNumCreditsAvailable()); + assertThat(queue.getAvailableReaders()).isEmpty(); + assertThat(reader.hasBuffersAvailable().isAvailable()).isTrue(); + assertThat(reader.isRegisteredAsAvailable()).isFalse(); + assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); // Notify available credits to trigger enqueue the reader again final int notifyNumCredits = 3; @@ -339,24 +329,25 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { // since the channel is blocked though, we will not process anything and only enqueue // the // reader once - assertTrue(reader.isRegisteredAsAvailable()); - assertThat(queue.getAvailableReaders(), contains(reader)); // contains only (this) one! - assertEquals(i, reader.getNumCreditsAvailable()); - assertTrue(reader.hasBuffersAvailable().isAvailable()); + assertThat(reader.isRegisteredAsAvailable()).isTrue(); + assertThat(queue.getAvailableReaders()).contains(reader); // contains only (this) one! + assertThat(reader.getNumCreditsAvailable()).isEqualTo(i); + assertThat(reader.hasBuffersAvailable().isAvailable()).isTrue(); } // Flush the buffer to make the channel writable again and see the final results channel.flush(); - assertSame(channelBlockingBuffer, channel.readOutbound()); + assertThat((ByteBuf) channel.readOutbound()).isSameAs(channelBlockingBuffer); - assertEquals(0, queue.getAvailableReaders().size()); - assertEquals(0, reader.getNumCreditsAvailable()); - assertTrue(reader.hasBuffersAvailable().isAvailable()); - assertFalse(reader.isRegisteredAsAvailable()); + assertThat(queue.getAvailableReaders()).isEmpty(); + assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader.hasBuffersAvailable().isAvailable()).isTrue(); + assertThat(reader.isRegisteredAsAvailable()).isFalse(); for (int i = 1; i <= notifyNumCredits; i++) { - assertThat(channel.readOutbound(), instanceOf(NettyMessage.BufferResponse.class)); + assertThat((Object) channel.readOutbound()) + .isInstanceOf(NettyMessage.BufferResponse.class); } - assertNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNull(); } /** @@ -365,7 +356,7 @@ public void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { * there are credit and data available. */ @Test - public void testEnqueueReaderByResumingConsumption() throws Exception { + void testEnqueueReaderByResumingConsumption() throws Exception { PipelinedSubpartition subpartition = PipelinedSubpartitionTest.createPipelinedSubpartition(); Buffer.DataType dataType1 = Buffer.DataType.ALIGNED_CHECKPOINT_BARRIER; @@ -386,26 +377,26 @@ public void testEnqueueReaderByResumingConsumption() throws Exception { reader.requestSubpartitionView(partitionProvider, new ResultPartitionID(), 0); queue.notifyReaderCreated(reader); - assertTrue(reader.getAvailabilityAndBacklog().isAvailable()); + assertThat(reader.getAvailabilityAndBacklog().isAvailable()).isTrue(); reader.notifyDataAvailable(); channel.runPendingTasks(); - assertFalse(reader.getAvailabilityAndBacklog().isAvailable()); - assertEquals(1, subpartition.unsynchronizedGetNumberOfQueuedBuffers()); + assertThat(reader.getAvailabilityAndBacklog().isAvailable()).isFalse(); + assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(1); queue.addCreditOrResumeConsumption( receiverId, NetworkSequenceViewReader::resumeConsumption); - assertFalse(reader.getAvailabilityAndBacklog().isAvailable()); - assertEquals(0, subpartition.unsynchronizedGetNumberOfQueuedBuffers()); + assertThat(reader.getAvailabilityAndBacklog().isAvailable()).isFalse(); + assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(0); Object data1 = channel.readOutbound(); - assertEquals(dataType1, ((NettyMessage.BufferResponse) data1).buffer.getDataType()); + assertThat(((NettyMessage.BufferResponse) data1).buffer.getDataType()).isEqualTo(dataType1); Object data2 = channel.readOutbound(); - assertEquals(dataType2, ((NettyMessage.BufferResponse) data2).buffer.getDataType()); + assertThat(((NettyMessage.BufferResponse) data2).buffer.getDataType()).isEqualTo(dataType2); } @Test - public void testAnnounceBacklog() throws Exception { + void testAnnounceBacklog() throws Exception { PipelinedSubpartition subpartition = PipelinedSubpartitionTest.createPipelinedSubpartition(); subpartition.add(createEventBufferConsumer(4096, Buffer.DataType.DATA_BUFFER)); @@ -428,24 +419,24 @@ public void testAnnounceBacklog() throws Exception { reader.notifyDataAvailable(); channel.runPendingTasks(); Object data = channel.readOutbound(); - assertTrue(data instanceof NettyMessage.BacklogAnnouncement); + assertThat(data).isInstanceOf(NettyMessage.BacklogAnnouncement.class); NettyMessage.BacklogAnnouncement announcement = (NettyMessage.BacklogAnnouncement) data; - assertEquals(receiverId, announcement.receiverId); - assertEquals(subpartition.getBuffersInBacklogUnsafe(), announcement.backlog); + assertThat(announcement.receiverId).isEqualTo(receiverId); + assertThat(announcement.backlog).isEqualTo(subpartition.getBuffersInBacklogUnsafe()); subpartition.release(); reader.notifyDataAvailable(); channel.runPendingTasks(); - assertNotNull(channel.readOutbound()); + assertThat((Object) channel.readOutbound()).isNotNull(); } @Test - public void testCancelPartitionRequestForUnavailableView() throws Exception { + void testCancelPartitionRequestForUnavailableView() throws Exception { testCancelPartitionRequest(false); } @Test - public void testCancelPartitionRequestForAvailableView() throws Exception { + void testCancelPartitionRequestForAvailableView() throws Exception { testCancelPartitionRequest(true); } @@ -469,18 +460,18 @@ private void testCancelPartitionRequest(boolean isAvailableView) throws Exceptio // add credit to make this reader available for adding into availableReaders queue if (isAvailableView) { queue.addCreditOrResumeConsumption(receiverId, viewReader -> viewReader.addCredit(1)); - assertTrue(queue.getAvailableReaders().contains(reader)); + assertThat(queue.getAvailableReaders().contains(reader)).isTrue(); } // cancel this subpartition view queue.cancel(receiverId); channel.runPendingTasks(); - assertFalse(queue.getAvailableReaders().contains(reader)); + assertThat(queue.getAvailableReaders().contains(reader)).isFalse(); // the reader view should be released (the partition is not, though, blocking partitions // support multiple successive readers for recovery and caching) - assertTrue(reader.isReleased()); + assertThat(reader.isReleased()).isTrue(); // cleanup partition.release(); @@ -488,7 +479,7 @@ private void testCancelPartitionRequest(boolean isAvailableView) throws Exceptio } @Test - public void testNotifyNewBufferSize() throws Exception { + void testNotifyNewBufferSize() throws Exception { // given: Result partition and the reader for subpartition 0. ResultPartition parent = createResultPartition(); @@ -520,10 +511,10 @@ public void testNotifyNewBufferSize() throws Exception { // then: Buffers of received size will be in outbound channel. Object data1 = channel.readOutbound(); // The size can not be less than the first record in buffer. - assertEquals(128, ((NettyMessage.BufferResponse) data1).buffer.getSize()); + assertThat(((NettyMessage.BufferResponse) data1).buffer.getSize()).isEqualTo(128); Object data2 = channel.readOutbound(); // The size should shrink up to notified buffer size. - assertEquals(65, ((NettyMessage.BufferResponse) data2).buffer.getSize()); + assertThat(((NettyMessage.BufferResponse) data2).buffer.getSize()).isEqualTo(65); } private static ResultPartition createResultPartition() throws IOException { @@ -574,7 +565,7 @@ static ByteBuf blockChannel(EmbeddedChannel channel) { // to the wire although the buffer is "empty". ByteBuf channelBlockingBuffer = Unpooled.buffer(highWaterMark).writerIndex(highWaterMark); channel.write(channelBlockingBuffer); - assertFalse(channel.isWritable()); + assertThat(channel.isWritable()).isFalse(); return channelBlockingBuffer; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java index 67cf02554a759..e1075b9766f5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java @@ -32,31 +32,25 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.concurrent.CompletableFuture; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link PartitionRequestServerHandler}. */ -public class PartitionRequestServerHandlerTest extends TestLogger { +class PartitionRequestServerHandlerTest { /** * Tests that {@link PartitionRequestServerHandler} responds {@link ErrorResponse} with wrapped * {@link PartitionNotFoundException} after receiving invalid {@link PartitionRequest}. */ @Test - public void testResponsePartitionNotFoundException() { + void testResponsePartitionNotFoundException() { final PartitionRequestServerHandler serverHandler = new PartitionRequestServerHandler( new ResultPartitionManager(), @@ -71,18 +65,18 @@ public void testResponsePartitionNotFoundException() { // Read the response message after handling partition request final Object msg = channel.readOutbound(); - assertThat(msg, instanceOf(ErrorResponse.class)); + assertThat(msg).isInstanceOf(ErrorResponse.class); final ErrorResponse err = (ErrorResponse) msg; - assertThat(err.cause, instanceOf(PartitionNotFoundException.class)); + assertThat(err.cause).isInstanceOf(PartitionNotFoundException.class); final ResultPartitionID actualPartitionId = ((PartitionNotFoundException) err.cause).getPartitionId(); - assertThat(partitionId, is(actualPartitionId)); + assertThat(partitionId).isEqualTo(actualPartitionId); } @Test - public void testResumeConsumption() { + void testResumeConsumption() { final InputChannelID inputChannelID = new InputChannelID(); final PartitionRequestQueue partitionRequestQueue = new PartitionRequestQueue(); final TestViewReader testViewReader = @@ -99,11 +93,11 @@ public void testResumeConsumption() { channel.writeInbound(new ResumeConsumption(inputChannelID)); channel.runPendingTasks(); - assertTrue(testViewReader.consumptionResumed); + assertThat(testViewReader.consumptionResumed).isTrue(); } @Test - public void testAcknowledgeAllRecordsProcessed() throws IOException { + void testAcknowledgeAllRecordsProcessed() throws IOException { InputChannelID inputChannelID = new InputChannelID(); ResultPartition resultPartition = @@ -132,16 +126,16 @@ public void testAcknowledgeAllRecordsProcessed() throws IOException { resultPartition.notifyEndOfData(StopMode.DRAIN); CompletableFuture allRecordsProcessedFuture = resultPartition.getAllDataProcessedFuture(); - assertFalse(allRecordsProcessedFuture.isDone()); + assertThat(allRecordsProcessedFuture.isDone()).isFalse(); channel.writeInbound(new NettyMessage.AckAllUserRecordsProcessed(inputChannelID)); channel.runPendingTasks(); - assertTrue(allRecordsProcessedFuture.isDone()); - assertFalse(allRecordsProcessedFuture.isCompletedExceptionally()); + assertThat(allRecordsProcessedFuture.isDone()).isTrue(); + assertThat(allRecordsProcessedFuture.isCompletedExceptionally()).isFalse(); } @Test - public void testNewBufferSize() { + void testNewBufferSize() { final InputChannelID inputChannelID = new InputChannelID(); final PartitionRequestQueue partitionRequestQueue = new PartitionRequestQueue(); final TestViewReader testViewReader = @@ -158,11 +152,11 @@ public void testNewBufferSize() { channel.writeInbound(new NettyMessage.NewBufferSize(666, inputChannelID)); channel.runPendingTasks(); - assertEquals(666, testViewReader.bufferSize); + assertThat(testViewReader.bufferSize).isEqualTo(666); } @Test - public void testReceivingNewBufferSizeBeforeReaderIsCreated() { + void testReceivingNewBufferSizeBeforeReaderIsCreated() { final InputChannelID inputChannelID = new InputChannelID(); final PartitionRequestQueue partitionRequestQueue = new PartitionRequestQueue(); final TestViewReader testViewReader = @@ -179,10 +173,12 @@ public void testReceivingNewBufferSizeBeforeReaderIsCreated() { channel.runPendingTasks(); // If error happens outbound messages would be not empty. - assertTrue(channel.outboundMessages().toString(), channel.outboundMessages().isEmpty()); + assertThat(channel.outboundMessages().isEmpty()) + .withFailMessage(channel.outboundMessages().toString()) + .isTrue(); // New buffer size should be silently ignored because it is possible situation. - assertEquals(-1, testViewReader.bufferSize); + assertThat(testViewReader.bufferSize).isEqualTo(-1); } private static class TestViewReader extends CreditBasedSequenceNumberingViewReader { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java index c8faf422fc088..d0067122b9bb8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ServerTransportErrorHandlingTest.java @@ -33,7 +33,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.stubbing.Answer; import java.util.concurrent.CountDownLatch; @@ -42,17 +42,17 @@ import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.connect; import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.initServerAndClient; import static org.apache.flink.runtime.io.network.netty.NettyTestUtil.shutdown; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class ServerTransportErrorHandlingTest { +class ServerTransportErrorHandlingTest { /** Verifies remote closes trigger the release of all resources. */ @Test - public void testRemoteClose() throws Exception { + void testRemoteClose() throws Exception { final TestPooledBufferProvider outboundBuffers = new TestPooledBufferProvider(16); final CountDownLatch sync = new CountDownLatch(1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java index 2f44048b25a0c..6f90313cea846 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java @@ -21,47 +21,41 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Optional; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link AbstractPartitionTracker}. */ -public class AbstractPartitionTrackerTest extends TestLogger { +public class AbstractPartitionTrackerTest { @Test - public void testStartStopTracking() { + void testStartStopTracking() { final TestPartitionTracker partitionTracker = new TestPartitionTracker(); final ResourceID executorWithTrackedPartition = new ResourceID("tracked"); final ResourceID executorWithoutTrackedPartition = new ResourceID("untracked"); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), - is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition)) + .isFalse(); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition)) + .isFalse(); partitionTracker.startTrackingPartition( executorWithTrackedPartition, new ResultPartitionID()); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(true)); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), - is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition)).isTrue(); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition)) + .isFalse(); partitionTracker.stopTrackingPartitionsFor(executorWithTrackedPartition); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); - assertThat( - partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), - is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition)) + .isFalse(); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition)) + .isFalse(); } public static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java index a071b80d9ead3..fce757a10dfb4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java @@ -24,8 +24,7 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Assertions for availability futures. */ public class AvailabilityUtil { @@ -37,11 +36,11 @@ public static void assertFutureCompletion( ThrowingRunnable runnable) throws E { final CompletableFuture availableFuture = futureSupplier.get(); - assertEquals(before, availableFuture.isDone()); + assertThat(availableFuture.isDone()).isEqualTo(before); runnable.run(); - assertEquals(after, futureSupplier.get().isDone()); + assertThat(futureSupplier.get().isDone()).isEqualTo(after); if (after) { - assertTrue(availableFuture.isDone()); + assertThat(availableFuture.isDone()).isTrue(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java index 33ddb75fc5276..7c6efd94e9909 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java @@ -21,32 +21,31 @@ import org.apache.flink.runtime.io.disk.FileChannelManagerImpl; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; +import org.apache.flink.testutils.junit.utils.TempDirUtils; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.IOException; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createView; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for the availability handling of the BoundedBlockingSubpartitions with not constant * availability. */ -public class BoundedBlockingSubpartitionAvailabilityTest { +class BoundedBlockingSubpartitionAvailabilityTest { - @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); + @TempDir private static Path tmpFolder; private static final int BUFFER_SIZE = 32 * 1024; @Test - public void testInitiallyNotAvailable() throws Exception { + void testInitiallyNotAvailable() throws Exception { final ResultSubpartition subpartition = createPartitionWithData(10); final CountingAvailabilityListener listener = new CountingAvailabilityListener(); @@ -54,7 +53,7 @@ public void testInitiallyNotAvailable() throws Exception { final ResultSubpartitionView subpartitionView = createView(subpartition, listener); // assert - assertEquals(0, listener.numNotifications); + assertThat(listener.numNotifications).isEqualTo(0); // cleanup subpartitionView.releaseAllResources(); @@ -62,7 +61,7 @@ public void testInitiallyNotAvailable() throws Exception { } @Test - public void testUnavailableWhenBuffersExhausted() throws Exception { + void testUnavailableWhenBuffersExhausted() throws Exception { // setup final ResultSubpartition subpartition = createPartitionWithData(100_000); final CountingAvailabilityListener listener = new CountingAvailabilityListener(); @@ -72,8 +71,8 @@ public void testUnavailableWhenBuffersExhausted() throws Exception { final List data = drainAvailableData(reader); // assert - assertFalse(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); - assertFalse(data.get(data.size() - 1).isDataAvailable()); + assertThat(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()).isFalse(); + assertThat(data.get(data.size() - 1).isDataAvailable()).isFalse(); // cleanup reader.releaseAllResources(); @@ -81,7 +80,7 @@ public void testUnavailableWhenBuffersExhausted() throws Exception { } @Test - public void testAvailabilityNotificationWhenBuffersReturn() throws Exception { + void testAvailabilityNotificationWhenBuffersReturn() throws Exception { // setup final ResultSubpartition subpartition = createPartitionWithData(100_000); final CountingAvailabilityListener listener = new CountingAvailabilityListener(); @@ -93,8 +92,8 @@ public void testAvailabilityNotificationWhenBuffersReturn() throws Exception { data.get(1).buffer().recycleBuffer(); // assert - assertTrue(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); - assertEquals(1, listener.numNotifications); + assertThat(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()).isTrue(); + assertThat(listener.numNotifications).isEqualTo(1); // cleanup reader.releaseAllResources(); @@ -102,7 +101,7 @@ public void testAvailabilityNotificationWhenBuffersReturn() throws Exception { } @Test - public void testNotAvailableWhenEmpty() throws Exception { + void testNotAvailableWhenEmpty() throws Exception { // setup final ResultSubpartition subpartition = createPartitionWithData(100_000); final ResultSubpartitionView reader = @@ -112,7 +111,7 @@ public void testNotAvailableWhenEmpty() throws Exception { drainAllData(reader); // assert - assertFalse(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); + assertThat(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()).isFalse(); // cleanup reader.releaseAllResources(); @@ -132,7 +131,10 @@ private static ResultSubpartition createPartitionWithData(int numberOfBuffers) .setSSLEnabled(true) .setFileChannelManager( new FileChannelManagerImpl( - new String[] {TMP_FOLDER.newFolder().toString()}, + new String[] { + TempDirUtils.newFolder(tmpFolder) + .getAbsolutePath() + }, "data")) .setNetworkBufferSize(BUFFER_SIZE) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java index 05afd7e64e790..1d47f80c2badb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java @@ -24,14 +24,14 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; import javax.annotation.Nullable; @@ -39,16 +39,13 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Arrays; -import java.util.Collection; +import java.util.List; import java.util.stream.Collectors; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** * Behavior tests for the {@link BoundedBlockingSubpartition} and the {@link @@ -57,7 +54,7 @@ *

Full read / write tests for the partition and the reader are in {@link * BoundedBlockingSubpartitionWriteReadTest}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class BoundedBlockingSubpartitionTest extends SubpartitionTestBase { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); @@ -68,23 +65,23 @@ public class BoundedBlockingSubpartitionTest extends SubpartitionTestBase { private final boolean sslEnabled; - @Parameterized.Parameters(name = "type = {0}, sslEnabled = {1}") - public static Collection parameters() { + @Parameters(name = "type = {0}, sslEnabled = {1}") + public static List parameters() { return Arrays.stream(BoundedBlockingSubpartitionType.values()) .map((type) -> new Object[][] {{type, true}, {type, false}}) .flatMap(Arrays::stream) .collect(Collectors.toList()); } - @ClassRule public static final TemporaryFolder TMP_DIR = new TemporaryFolder(); + @TempDir private Path tmpFolder; - @BeforeClass - public static void setUp() { + @BeforeAll + static void setUp() { fileChannelManager = new FileChannelManagerImpl(new String[] {tempDir}, "testing"); } - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { fileChannelManager.close(); } @@ -96,8 +93,8 @@ public BoundedBlockingSubpartitionTest( // ------------------------------------------------------------------------ - @Test - public void testCreateReaderBeforeFinished() throws Exception { + @TestTemplate + void testCreateReaderBeforeFinished() throws Exception { final ResultSubpartition partition = createSubpartition(); try { @@ -109,8 +106,8 @@ public void testCreateReaderBeforeFinished() throws Exception { partition.release(); } - @Test - public void testCloseBoundedData() throws Exception { + @TestTemplate + void testCloseBoundedData() throws Exception { final TestingBoundedDataReader reader = new TestingBoundedDataReader(); final TestingBoundedData data = new TestingBoundedData(reader); final BoundedBlockingSubpartitionReader bbspr = @@ -122,11 +119,12 @@ public void testCloseBoundedData() throws Exception { bbspr.releaseAllResources(); - assertTrue(reader.closed); + assertThat(reader.closed).isTrue(); } - @Test - public void testRecycleCurrentBufferOnFailure() throws Exception { + @TestTemplate + void testRecycleCurrentBufferOnFailure(BoundedBlockingSubpartitionType type, boolean sslEnabled) + throws Exception { final ResultPartition resultPartition = createPartition(ResultPartitionType.BLOCKING, fileChannelManager); final BoundedBlockingSubpartition subpartition = @@ -147,16 +145,16 @@ public void testRecycleCurrentBufferOnFailure() throws Exception { // expected } - assertFalse(consumer.isRecycled()); + assertThat(consumer.isRecycled()).isFalse(); - assertNotNull(subpartition.getCurrentBuffer()); - assertFalse(subpartition.getCurrentBuffer().isRecycled()); + assertThat(subpartition.getCurrentBuffer()).isNotNull(); + assertThat(subpartition.getCurrentBuffer().isRecycled()).isFalse(); } finally { subpartition.release(); - assertTrue(consumer.isRecycled()); + assertThat(consumer.isRecycled()).isTrue(); - assertNull(subpartition.getCurrentBuffer()); + assertThat(subpartition.getCurrentBuffer()).isNull(); } } @@ -169,7 +167,7 @@ ResultSubpartition createSubpartition() throws Exception { return type.create( 0, resultPartition, - new File(TMP_DIR.newFolder(), "subpartition"), + new File(tmpFolder.toFile(), "subpartition"), BufferBuilderTestUtils.BUFFER_SIZE, sslEnabled); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java index 4e519f812584b..beea4b40d2428 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java @@ -32,15 +32,14 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.IOException; @@ -49,18 +48,17 @@ import java.util.Collection; import java.util.stream.Collectors; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class BoundedBlockingSubpartitionWriteReadTest { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); private static FileChannelManager fileChannelManager; - @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); + @TempDir private File tmpFolder; private static final int BUFFER_SIZE = 1024 * 1024; @@ -99,18 +97,18 @@ public BoundedBlockingSubpartitionWriteReadTest( // tests // ------------------------------------------------------------------------ - @BeforeClass - public static void setUp() { + @BeforeAll + static void setUp() { fileChannelManager = new FileChannelManagerImpl(new String[] {tempDir}, "testing"); } - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { fileChannelManager.close(); } - @Test - public void testWriteAndReadData() throws Exception { + @TestTemplate + void testWriteAndReadData() throws Exception { final int numLongs = 15_000_000; // roughly 115 MiBytes // setup @@ -130,8 +128,8 @@ public void testWriteAndReadData() throws Exception { subpartition.release(); } - @Test - public void testRead10ConsumersSequential() throws Exception { + @TestTemplate + void testRead10ConsumersSequential() throws Exception { final int numLongs = 10_000_000; // setup @@ -153,8 +151,8 @@ public void testRead10ConsumersSequential() throws Exception { subpartition.release(); } - @Test - public void testRead10ConsumersConcurrent() throws Exception { + @TestTemplate + void testRead10ConsumersConcurrent() throws Exception { final int numLongs = 15_000_000; // setup @@ -197,8 +195,8 @@ private static void readLongs( int nextExpectedBacklog = numBuffers - 1; while ((next = reader.getNextBuffer()) != null && next.buffer().isBuffer()) { - assertTrue(next.isDataAvailable()); - assertEquals(nextExpectedBacklog, next.buffersInBacklog()); + assertThat(next.isDataAvailable()).isTrue(); + assertThat(next.buffersInBacklog()).isEqualTo(nextExpectedBacklog); ByteBuffer buffer = next.buffer().getNioBufferReadable(); if (compressionEnabled && next.buffer().isCompressed()) { @@ -208,15 +206,15 @@ private static void readLongs( uncompressedBuffer.recycleBuffer(); } while (buffer.hasRemaining()) { - assertEquals(expectedNextLong++, buffer.getLong()); + assertThat(buffer.getLong()).isEqualTo(expectedNextLong++); } next.buffer().recycleBuffer(); nextExpectedBacklog--; } - assertEquals(numLongs, expectedNextLong); - assertEquals(-1, nextExpectedBacklog); + assertThat(expectedNextLong).isEqualTo(numLongs); + assertThat(nextExpectedBacklog).isEqualTo(-1); } // ------------------------------------------------------------------------ @@ -272,7 +270,7 @@ private BoundedBlockingSubpartition createSubpartition() throws IOException { fileChannelManager, compressionEnabled, BUFFER_SIZE), - new File(TMP_FOLDER.newFolder(), "partitiondata"), + new File(tmpFolder, "partitiondata"), BUFFER_SIZE, sslEnabled); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java index f1849c6993a48..72ed97cdf31be 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java @@ -22,30 +22,28 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferCompressor; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.hamcrest.Matchers; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; -import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class BoundedDataTestBase { - @ClassRule public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder(); + private Path subpartitionDataPath; /** * Max buffer sized used by the tests that write data. For implementations that need to @@ -61,11 +59,16 @@ public abstract class BoundedDataTestBase { private static final BufferDecompressor DECOMPRESSOR = new BufferDecompressor(BUFFER_SIZE, COMPRESSION_CODEC); - @Parameterized.Parameter public static boolean compressionEnabled; + @Parameter public static boolean compressionEnabled; - @Parameterized.Parameters(name = "compressionEnabled = {0}") - public static Boolean[] compressionEnabled() { - return new Boolean[] {false, true}; + @Parameters(name = "compressionEnabled = {0}") + public static List compressionEnabled() { + return Arrays.asList(false, true); + } + + @BeforeEach + void before(@TempDir Path tempDir) { + this.subpartitionDataPath = tempDir.resolve("subpartitiondata"); } // ------------------------------------------------------------------------ @@ -80,25 +83,25 @@ protected abstract BoundedData createBoundedDataWithRegion(Path tempFilePath, in throws IOException; protected BoundedData createBoundedData() throws IOException { - return createBoundedData(createTempPath()); + return createBoundedData(subpartitionDataPath); } private BoundedData createBoundedDataWithRegion(int regionSize) throws IOException { - return createBoundedDataWithRegion(createTempPath(), regionSize); + return createBoundedDataWithRegion(subpartitionDataPath, regionSize); } // ------------------------------------------------------------------------ // Tests // ------------------------------------------------------------------------ - @Test + @TestTemplate public void testWriteAndReadData() throws Exception { try (BoundedData bd = createBoundedData()) { testWriteAndReadData(bd); } } - @Test + @TestTemplate public void testWriteAndReadDataAcrossRegions() throws Exception { if (!isRegionBased()) { return; @@ -117,7 +120,7 @@ private void testWriteAndReadData(BoundedData bd) throws Exception { readLongs(bd.createReader(), numBuffers, numLongs); } - @Test + @TestTemplate public void returnNullAfterEmpty() throws Exception { try (BoundedData bd = createBoundedData()) { bd.finishWrite(); @@ -125,31 +128,30 @@ public void returnNullAfterEmpty() throws Exception { final BoundedData.Reader reader = bd.createReader(); // check that multiple calls now return empty buffers - assertNull(reader.nextBuffer()); - assertNull(reader.nextBuffer()); - assertNull(reader.nextBuffer()); + assertThat(reader.nextBuffer()).isNull(); + assertThat(reader.nextBuffer()).isNull(); + assertThat(reader.nextBuffer()).isNull(); } } - @Test + @TestTemplate public void testDeleteFileOnClose() throws Exception { - final Path path = createTempPath(); - final BoundedData bd = createBoundedData(path); - assertTrue(Files.exists(path)); + final BoundedData bd = createBoundedData(subpartitionDataPath); + assertThat(Files.exists(subpartitionDataPath)).isTrue(); bd.close(); - assertFalse(Files.exists(path)); + assertThat(Files.exists(subpartitionDataPath)).isFalse(); } - @Test + @TestTemplate public void testGetSizeSingleRegion() throws Exception { try (BoundedData bd = createBoundedData()) { testGetSize(bd, 60_787, 76_687); } } - @Test + @TestTemplate public void testGetSizeMultipleRegions() throws Exception { if (!isRegionBased()) { return; @@ -168,13 +170,13 @@ private static void testGetSize(BoundedData bd, int bufferSize1, int bufferSize2 bufferSize1 + bufferSize2 + 2 * BufferReaderWriterUtil.HEADER_LENGTH; bd.writeBuffer(BufferBuilderTestUtils.buildSomeBuffer(bufferSize1)); - assertEquals(expectedSize1, bd.getSize()); + assertThat(bd.getSize()).isEqualTo(expectedSize1); bd.writeBuffer(BufferBuilderTestUtils.buildSomeBuffer(bufferSize2)); - assertEquals(expectedSizeFinal, bd.getSize()); + assertThat(bd.getSize()).isEqualTo(expectedSizeFinal); bd.finishWrite(); - assertEquals(expectedSizeFinal, bd.getSize()); + assertThat(bd.getSize()).isEqualTo(expectedSizeFinal); } // ------------------------------------------------------------------------ @@ -232,11 +234,7 @@ private static void readLongs(BoundedData.Reader reader, int numBuffersExpected, b.recycleBuffer(); } - assertEquals(numBuffersExpected, numBuffers); - assertThat(nextValue, Matchers.greaterThanOrEqualTo((long) numLongs)); - } - - private static Path createTempPath() throws IOException { - return new File(TMP_FOLDER.newFolder(), "subpartitiondata").toPath(); + assertThat(numBuffers).isEqualTo(numBuffersExpected); + assertThat(nextValue).isGreaterThanOrEqualTo((long) numLongs); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtilTest.java index 5e0fee63c780c..1358b6b02dd86 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtilTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BufferReaderWriterUtilTest.java @@ -23,10 +23,8 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import java.io.IOException; @@ -44,7 +42,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link BufferReaderWriterUtil}. */ -@ExtendWith(TestLoggerExtension.class) class BufferReaderWriterUtilTest { // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java index bd2583e8f32b7..a845d2e5a0aa4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java @@ -24,10 +24,11 @@ import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,28 +41,26 @@ import java.util.Random; import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link SortBasedDataBuffer} and {@link HashBasedDataBuffer}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class DataBufferTest { private final boolean useHashBuffer; - @Parameterized.Parameters(name = "UseHashBuffer = {0}") - public static Object[] parameters() { - return new Object[] {true, false}; + @Parameters(name = "UseHashBuffer = {0}") + public static List parameters() { + return Arrays.asList(true, false); } public DataBufferTest(boolean useHashBuffer) { this.useHashBuffer = useHashBuffer; } - @Test - public void testWriteAndReadDataBuffer() throws Exception { + @TestTemplate + void testWriteAndReadDataBuffer() throws Exception { int numSubpartitions = 10; int bufferSize = 1024; int bufferPoolSize = 512; @@ -131,14 +130,14 @@ public void testWriteAndReadDataBuffer() throws Exception { // read all data from the sort buffer if (dataBuffer.hasRemaining()) { - assertTrue(dataBuffer instanceof HashBasedDataBuffer); + assertThat(dataBuffer).isInstanceOf(HashBasedDataBuffer.class); dataBuffer.finish(); while (dataBuffer.hasRemaining()) { addBufferRead(copyIntoSegment(bufferSize, dataBuffer), buffersRead, numBytesRead); } } - assertEquals(0, dataBuffer.numTotalBytes()); + assertThat(dataBuffer.numTotalBytes()).isEqualTo(0); checkWriteReadResult( numSubpartitions, numBytesWritten, numBytesRead, dataWritten, buffersRead); } @@ -177,7 +176,8 @@ public static void checkWriteReadResult( Queue[] dataWritten, Queue[] buffersRead) { for (int subpartitionIndex = 0; subpartitionIndex < numSubpartitions; ++subpartitionIndex) { - assertEquals(numBytesWritten[subpartitionIndex], numBytesRead[subpartitionIndex]); + assertThat(numBytesRead[subpartitionIndex]) + .isEqualTo(numBytesWritten[subpartitionIndex]); List eventsWritten = new ArrayList<>(); List eventsRead = new ArrayList<>(); @@ -202,18 +202,20 @@ public static void checkWriteReadResult( subpartitionDataWritten.flip(); subpartitionDataRead.flip(); - assertEquals(subpartitionDataWritten, subpartitionDataRead); + assertThat(subpartitionDataRead).isEqualTo(subpartitionDataWritten); - assertEquals(eventsWritten.size(), eventsRead.size()); + assertThat(eventsRead.size()).isEqualTo(eventsWritten.size()); for (int i = 0; i < eventsWritten.size(); ++i) { - assertEquals(eventsWritten.get(i).dataType, eventsRead.get(i).getDataType()); - assertEquals(eventsWritten.get(i).data, eventsRead.get(i).getNioBufferReadable()); + assertThat(eventsRead.get(i).getDataType()) + .isEqualTo(eventsWritten.get(i).dataType); + assertThat(eventsRead.get(i).getNioBufferReadable()) + .isEqualTo(eventsWritten.get(i).data); } } } - @Test - public void testWriteReadWithEmptyChannel() throws Exception { + @TestTemplate + void testWriteReadWithEmptyChannel() throws Exception { int bufferPoolSize = 10; int bufferSize = 1024; int numSubpartitions = 5; @@ -253,12 +255,12 @@ private void checkReadResult( DataBuffer dataBuffer, ByteBuffer expectedBuffer, int expectedChannel, int bufferSize) { MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(bufferSize); BufferWithChannel bufferWithChannel = dataBuffer.getNextBuffer(segment); - assertEquals(expectedChannel, bufferWithChannel.getChannelIndex()); - assertEquals(expectedBuffer, bufferWithChannel.getBuffer().getNioBufferReadable()); + assertThat(bufferWithChannel.getChannelIndex()).isEqualTo(expectedChannel); + assertThat(bufferWithChannel.getBuffer().getNioBufferReadable()).isEqualTo(expectedBuffer); } - @Test(expected = IllegalArgumentException.class) - public void testWriteEmptyData() throws Exception { + @TestTemplate + void testWriteEmptyData() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); @@ -266,31 +268,40 @@ public void testWriteEmptyData() throws Exception { ByteBuffer record = ByteBuffer.allocate(1); record.position(1); - dataBuffer.append(record, 0, Buffer.DataType.DATA_BUFFER); + assertThatThrownBy(() -> dataBuffer.append(record, 0, Buffer.DataType.DATA_BUFFER)) + .isInstanceOf(IllegalArgumentException.class); } - @Test(expected = IllegalStateException.class) - public void testWriteFinishedDataBuffer() throws Exception { + @TestTemplate + void testWriteFinishedDataBuffer() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); dataBuffer.finish(); - dataBuffer.append(ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER); + assertThatThrownBy( + () -> + dataBuffer.append( + ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER)) + .isInstanceOf(IllegalStateException.class); } - @Test(expected = IllegalStateException.class) - public void testWriteReleasedDataBuffer() throws Exception { + @TestTemplate + void testWriteReleasedDataBuffer() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); dataBuffer.release(); - dataBuffer.append(ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER); + assertThatThrownBy( + () -> + dataBuffer.append( + ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER)) + .isInstanceOf(IllegalStateException.class); } - @Test - public void testWriteMoreDataThanCapacity() throws Exception { + @TestTemplate + void testWriteMoreDataThanCapacity() throws Exception { int bufferPoolSize = 10; int bufferSize = 1024; @@ -306,8 +317,8 @@ public void testWriteMoreDataThanCapacity() throws Exception { appendAndCheckResult(dataBuffer, bufferSize + 1, true, numBytes, numRecords, true); } - @Test - public void testWriteLargeRecord() throws Exception { + @TestTemplate + void testWriteLargeRecord() throws Exception { int bufferPoolSize = 10; int bufferSize = 1024; @@ -325,52 +336,63 @@ private void appendAndCheckResult( throws IOException { ByteBuffer largeRecord = ByteBuffer.allocate(recordSize); - assertEquals(isFull, dataBuffer.append(largeRecord, 0, Buffer.DataType.DATA_BUFFER)); - assertEquals(numBytes, dataBuffer.numTotalBytes()); - assertEquals(numRecords, dataBuffer.numTotalRecords()); - assertEquals(hasRemaining, dataBuffer.hasRemaining()); + assertThat(dataBuffer.append(largeRecord, 0, Buffer.DataType.DATA_BUFFER)) + .isEqualTo(isFull); + assertThat(dataBuffer.numTotalBytes()).isEqualTo(numBytes); + assertThat(dataBuffer.numTotalRecords()).isEqualTo(numRecords); + assertThat(dataBuffer.hasRemaining()).isEqualTo(hasRemaining); } - @Test(expected = IllegalStateException.class) - public void testReadUnfinishedDataBuffer() throws Exception { + @TestTemplate + void testReadUnfinishedDataBuffer() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); dataBuffer.append(ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER); - assertTrue(dataBuffer.hasRemaining()); - dataBuffer.getNextBuffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize)); + assertThat(dataBuffer.hasRemaining()).isTrue(); + assertThatThrownBy( + () -> + dataBuffer.getNextBuffer( + MemorySegmentFactory.allocateUnpooledSegment(bufferSize))) + .isInstanceOf(IllegalStateException.class); } - @Test(expected = IllegalStateException.class) - public void testReadReleasedDataBuffer() throws Exception { + @TestTemplate + void testReadReleasedDataBuffer() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); dataBuffer.append(ByteBuffer.allocate(1), 0, Buffer.DataType.DATA_BUFFER); dataBuffer.finish(); - assertTrue(dataBuffer.hasRemaining()); + assertThat(dataBuffer.hasRemaining()).isTrue(); dataBuffer.release(); - assertTrue(dataBuffer.hasRemaining()); + assertThat(dataBuffer.hasRemaining()).isTrue(); - dataBuffer.getNextBuffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize)); + assertThatThrownBy( + () -> + dataBuffer.getNextBuffer( + MemorySegmentFactory.allocateUnpooledSegment(bufferSize))) + .isInstanceOf(IllegalStateException.class); } - @Test - public void testReadEmptyDataBuffer() throws Exception { + @TestTemplate + void testReadEmptyDataBuffer() throws Exception { int bufferSize = 1024; DataBuffer dataBuffer = createDataBuffer(1, bufferSize, 1); dataBuffer.finish(); - assertFalse(dataBuffer.hasRemaining()); - assertNull( - dataBuffer.getNextBuffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize))); + assertThat(dataBuffer.hasRemaining()).isFalse(); + assertThat( + dataBuffer.getNextBuffer( + MemorySegmentFactory.allocateUnpooledSegment(bufferSize))) + .isNull(); } - @Test - public void testReleaseDataBuffer() throws Exception { + @TestTemplate + void testReleaseDataBuffer() throws Exception { int bufferPoolSize = 10; int bufferSize = 1024; int recordSize = (bufferPoolSize - 1) * bufferSize; @@ -386,17 +408,17 @@ public void testReleaseDataBuffer() throws Exception { new SortBasedDataBuffer(segments, bufferPool, 1, bufferSize, bufferPoolSize, null); dataBuffer.append(ByteBuffer.allocate(recordSize), 0, Buffer.DataType.DATA_BUFFER); - assertEquals(bufferPoolSize, bufferPool.bestEffortGetNumOfUsedBuffers()); - assertTrue(dataBuffer.hasRemaining()); - assertEquals(1, dataBuffer.numTotalRecords()); - assertEquals(recordSize, dataBuffer.numTotalBytes()); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(bufferPoolSize); + assertThat(dataBuffer.hasRemaining()).isTrue(); + assertThat(dataBuffer.numTotalRecords()).isEqualTo(1); + assertThat(dataBuffer.numTotalBytes()).isEqualTo(recordSize); // should release all data and resources dataBuffer.release(); - assertEquals(0, bufferPool.bestEffortGetNumOfUsedBuffers()); - assertTrue(dataBuffer.hasRemaining()); - assertEquals(1, dataBuffer.numTotalRecords()); - assertEquals(recordSize, dataBuffer.numTotalBytes()); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(0); + assertThat(dataBuffer.hasRemaining()).isTrue(); + assertThat(dataBuffer.numTotalRecords()).isEqualTo(1); + assertThat(dataBuffer.numTotalBytes()).isEqualTo(recordSize); } private DataBuffer createDataBuffer(int bufferPoolSize, int bufferSize, int numSubpartitions) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java index 137aecf1bab3a..340a6fc3c99af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java @@ -36,22 +36,22 @@ import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.net.SSLUtilsTest; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.testutils.serialization.types.ByteArrayType; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPromise; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.List; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests the bug reported in FLINK-131O0. @@ -67,8 +67,8 @@ * recycled while fetching the second buffer to trigger next read ahead, which breaks the above * assumption. */ -@RunWith(Parameterized.class) -public class FileBufferReaderITCase extends TestLogger { +@ExtendWith(ParameterizedTestExtension.class) +public class FileBufferReaderITCase { private static final int parallelism = 8; @@ -82,22 +82,22 @@ public class FileBufferReaderITCase extends TestLogger { private static final byte[] dataSource = new byte[recordSize]; - @Parameterized.Parameters(name = "SSL Enabled = {0}") + @Parameters(name = "SSL Enabled = {0}") public static List paras() { return Arrays.asList(true, false); } - @Parameterized.Parameter public boolean sslEnabled; + @Parameter public boolean sslEnabled; - @BeforeClass - public static void setup() { + @BeforeAll + static void setup() { for (int i = 0; i < dataSource.length; i++) { dataSource[i] = 0; } } - @Test - public void testSequentialReading() throws Exception { + @TestTemplate + void testSequentialReading() throws Exception { // setup final Configuration configuration; if (sslEnabled) { @@ -213,7 +213,7 @@ public void invoke() throws Exception { numReceived++; } - assertThat(numReceived, is(numRecords)); + assertThat(numReceived).isEqualTo(numRecords); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java index a2fe6b317d9a1..511fff4f975a2 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java @@ -24,9 +24,9 @@ import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; import org.apache.flink.runtime.util.EnvironmentInformation; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.file.Path; @@ -34,25 +34,22 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSomeBuffer; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createView; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ -public class FileChannelBoundedDataTest extends BoundedDataTestBase { +class FileChannelBoundedDataTest extends BoundedDataTestBase { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); private static FileChannelManager fileChannelManager; - @BeforeClass - public static void setUp() { + @BeforeAll + static void setUp() { fileChannelManager = new FileChannelManagerImpl(new String[] {tempDir}, "testing"); } - @AfterClass - public static void shutdown() throws Exception { + @AfterAll + static void shutdown() throws Exception { fileChannelManager.close(); } @@ -73,7 +70,7 @@ protected BoundedData createBoundedDataWithRegion(Path tempFilePath, int regionS } @Test - public void testReadNextBuffer() throws Exception { + void testReadNextBuffer() throws Exception { final int numberOfBuffers = 3; try (final BoundedData data = createBoundedData()) { writeBuffers(data, numberOfBuffers); @@ -82,10 +79,10 @@ public void testReadNextBuffer() throws Exception { final Buffer buffer1 = reader.nextBuffer(); final Buffer buffer2 = reader.nextBuffer(); - assertNotNull(buffer1); - assertNotNull(buffer2); + assertThat(buffer1).isNotNull(); + assertThat(buffer2).isNotNull(); // there are only two available memory segments for reading data - assertNull(reader.nextBuffer()); + assertThat(reader.nextBuffer()).isNull(); // cleanup buffer1.recycleBuffer(); @@ -94,7 +91,7 @@ public void testReadNextBuffer() throws Exception { } @Test - public void testRecycleBufferForNotifyingSubpartitionView() throws Exception { + void testRecycleBufferForNotifyingSubpartitionView() throws Exception { final int numberOfBuffers = 2; try (final BoundedData data = createBoundedData()) { writeBuffers(data, numberOfBuffers); @@ -104,28 +101,28 @@ public void testRecycleBufferForNotifyingSubpartitionView() throws Exception { final BoundedData.Reader reader = data.createReader(subpartitionView); final Buffer buffer1 = reader.nextBuffer(); final Buffer buffer2 = reader.nextBuffer(); - assertNotNull(buffer1); - assertNotNull(buffer2); + assertThat(buffer1).isNotNull(); + assertThat(buffer2).isNotNull(); - assertFalse(subpartitionView.isAvailable); + assertThat(subpartitionView.isAvailable).isFalse(); buffer1.recycleBuffer(); // the view is notified while recycling buffer if reader has not tagged finished - assertTrue(subpartitionView.isAvailable); + assertThat(subpartitionView.isAvailable).isTrue(); subpartitionView.resetAvailable(); - assertFalse(subpartitionView.isAvailable); + assertThat(subpartitionView.isAvailable).isFalse(); // the next buffer is null to make reader tag finished - assertNull(reader.nextBuffer()); + assertThat(reader.nextBuffer()).isNull(); buffer2.recycleBuffer(); // the view is not notified while recycling buffer if reader already finished - assertFalse(subpartitionView.isAvailable); + assertThat(subpartitionView.isAvailable).isFalse(); } } @Test - public void testRecycleBufferForNotifyingBufferAvailabilityListener() throws Exception { + void testRecycleBufferForNotifyingBufferAvailabilityListener() throws Exception { final ResultSubpartition subpartition = createFileBoundedBlockingSubpartition(); final int numberOfBuffers = 2; writeBuffers(subpartition, numberOfBuffers); @@ -133,19 +130,20 @@ public void testRecycleBufferForNotifyingBufferAvailabilityListener() throws Exc final VerifyNotificationBufferAvailabilityListener listener = new VerifyNotificationBufferAvailabilityListener(); final ResultSubpartitionView subpartitionView = createView(subpartition, listener); - assertFalse(listener.isAvailable); + assertThat(listener.isAvailable).isFalse(); final BufferAndBacklog buffer1 = subpartitionView.getNextBuffer(); final BufferAndBacklog buffer2 = subpartitionView.getNextBuffer(); - assertNotNull(buffer1); - assertNotNull(buffer2); + assertThat(buffer1).isNotNull(); + assertThat(buffer2).isNotNull(); // the next buffer is null in view because FileBufferReader has no available buffers for // reading ahead - assertFalse(subpartitionView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); + assertThat(subpartitionView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()) + .isFalse(); // recycle a buffer to trigger notification of data available buffer1.buffer().recycleBuffer(); - assertTrue(listener.isAvailable); + assertThat(listener.isAvailable).isTrue(); // cleanup buffer2.buffer().recycleBuffer(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index 88c957eed9de0..4f3e154f891ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -39,7 +39,7 @@ import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; @@ -52,16 +52,14 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createDummyConnectionManager; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** Tests verifying fairness in input gates. */ public class InputGateFairnessTest { @Test - public void testFairConsumptionLocalChannelsPreFilled() throws Exception { + void testFairConsumptionLocalChannelsPreFilled() throws Exception { final int numberOfChannels = 37; final int buffersPerChannel = 27; @@ -113,7 +111,7 @@ public void testFairConsumptionLocalChannelsPreFilled() throws Exception { // read all the buffers and the EOF event for (int i = numberOfChannels * (buffersPerChannel + 1); i > 0; --i) { - assertNotNull(gate.getNext()); + assertThat(gate.getNext()).isNotNull(); int min = Integer.MAX_VALUE; int max = 0; @@ -124,14 +122,14 @@ public void testFairConsumptionLocalChannelsPreFilled() throws Exception { max = Math.max(max, size); } - assertTrue(max == min || max == (min + 1)); + assertThat(max == min || max == (min + 1)).isTrue(); } - assertFalse(gate.getNext().isPresent()); + assertThat(gate.getNext()).isEmpty(); } @Test - public void testFairConsumptionLocalChannels() throws Exception { + void testFairConsumptionLocalChannels() throws Exception { final int numberOfChannels = 37; final int buffersPerChannel = 27; @@ -178,7 +176,7 @@ public void testFairConsumptionLocalChannels() throws Exception { // read all the buffers and the EOF event for (int i = 0; i < numberOfChannels * buffersPerChannel; i++) { - assertNotNull(gate.getNext()); + assertThat(gate.getNext()).isNotNull(); int min = Integer.MAX_VALUE; int max = 0; @@ -189,7 +187,7 @@ public void testFairConsumptionLocalChannels() throws Exception { max = Math.max(max, size); } - assertTrue(max == min || max == min + 1); + assertThat(max == min || max == min + 1).isTrue(); if (i % (2 * numberOfChannels) == 0) { // add three buffers to each channel, in random order @@ -201,7 +199,7 @@ public void testFairConsumptionLocalChannels() throws Exception { } @Test - public void testFairConsumptionRemoteChannelsPreFilled() throws Exception { + void testFairConsumptionRemoteChannelsPreFilled() throws Exception { final int numberOfChannels = 37; final int buffersPerChannel = 27; @@ -234,7 +232,7 @@ public void testFairConsumptionRemoteChannelsPreFilled() throws Exception { // read all the buffers and the EOF event for (int i = numberOfChannels * (buffersPerChannel + 1); i > 0; --i) { - assertNotNull(gate.getNext()); + assertThat(gate.getNext()).isNotNull(); int min = Integer.MAX_VALUE; int max = 0; @@ -245,14 +243,14 @@ public void testFairConsumptionRemoteChannelsPreFilled() throws Exception { max = Math.max(max, size); } - assertTrue(max == min || max == (min + 1)); + assertThat(max == min || max == (min + 1)).isTrue(); } - assertFalse(gate.getNext().isPresent()); + assertThat(gate.getNext()).isEmpty(); } @Test - public void testFairConsumptionRemoteChannels() throws Exception { + void testFairConsumptionRemoteChannels() throws Exception { final int numberOfChannels = 37; final int buffersPerChannel = 27; @@ -279,7 +277,7 @@ public void testFairConsumptionRemoteChannels() throws Exception { // read all the buffers and the EOF event for (int i = 0; i < numberOfChannels * buffersPerChannel; i++) { - assertNotNull(gate.getNext()); + assertThat(gate.getNext()).isNotNull(); int min = Integer.MAX_VALUE; int max = 0; @@ -290,7 +288,7 @@ public void testFairConsumptionRemoteChannels() throws Exception { max = Math.max(max, size); } - assertTrue(max == min || max == (min + 1)); + assertThat(max == min || max == (min + 1)).isTrue(); if (i % (2 * numberOfChannels) == 0) { // add three buffers to each channel, in random order @@ -395,9 +393,9 @@ public FairnessVerifyingInputGate( @Override public Optional getNext() throws IOException, InterruptedException { synchronized (channelsWithData) { - assertTrue( - "too many input channels", - channelsWithData.size() <= getNumberOfInputChannels()); + assertThat(channelsWithData.size()) + .withFailMessage("too few input channels") + .isLessThanOrEqualTo(getNumberOfInputChannels()); ensureUnique(channelsWithData.asUnmodifiableCollection()); } @@ -413,8 +411,9 @@ private void ensureUnique(Collection channels) { } } - assertTrue( - "found duplicate input channels", uniquenessChecker.size() == channels.size()); + assertThat(uniquenessChecker.size()) + .withFailMessage("found duplicate input channels") + .isEqualTo(channels.size()); uniquenessChecker.clear(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java index 980e02d5b5a16..2140a37b1bf3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java @@ -27,12 +27,10 @@ import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava31.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -45,27 +43,24 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link JobMasterPartitionTrackerImpl}. */ -public class JobMasterPartitionTrackerImplTest extends TestLogger { +class JobMasterPartitionTrackerImplTest { @Test - public void testPipelinedPartitionIsNotTracked() { + void testPipelinedPartitionIsNotTracked() { testReleaseOnConsumptionHandling(ResultPartitionType.PIPELINED); } @Test - public void testBlockingPartitionIsTracked() { + void testBlockingPartitionIsTracked() { testReleaseOnConsumptionHandling(ResultPartitionType.BLOCKING); } @Test - public void testPipelinedApproximatePartitionIsTracked() { + void testPipelinedApproximatePartitionIsTracked() { testReleaseOnConsumptionHandling(ResultPartitionType.PIPELINED_APPROXIMATE); } @@ -81,13 +76,12 @@ private static void testReleaseOnConsumptionHandling(ResultPartitionType resultP AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId, resultPartitionType, true)); - assertThat( - partitionTracker.isTrackingPartitionsFor(resourceId), - is(resultPartitionType.isReleaseByScheduler())); + assertThat(partitionTracker.isTrackingPartitionsFor(resourceId)) + .isEqualTo(resultPartitionType.isReleaseByScheduler()); } @Test - public void testReleaseCallsWithLocalResources() { + void testReleaseCallsWithLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final JobID jobId = new JobID(); @@ -109,22 +103,23 @@ public void testReleaseCallsWithLocalResources() { AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId, true)); - assertThat(partitionTracker.isTrackingPartitionsFor(tmId), is(true)); + assertThat(partitionTracker.isTrackingPartitionsFor(tmId)).isTrue(); partitionTracker.stopTrackingAndReleasePartitions(Arrays.asList(resultPartitionId)); - assertEquals(1, releaseCall.size()); + assertThat(releaseCall).hasSize(1); ReleaseCall releaseOrPromoteCall = releaseCall.remove(); - assertEquals(tmId, releaseOrPromoteCall.getTaskExecutorId()); - assertEquals(jobId, releaseOrPromoteCall.getJobId()); - assertThat(releaseOrPromoteCall.getReleasedPartitions(), contains(resultPartitionId)); - assertEquals(1, shuffleMaster.externallyReleasedPartitions.size()); - assertEquals(resultPartitionId, shuffleMaster.externallyReleasedPartitions.remove()); - assertThat(partitionTracker.isTrackingPartitionsFor(tmId), is(false)); + assertThat(releaseOrPromoteCall.getTaskExecutorId()).isEqualTo(tmId); + assertThat(releaseOrPromoteCall.getJobId()).isEqualTo(jobId); + assertThat(releaseOrPromoteCall.getReleasedPartitions()).contains(resultPartitionId); + assertThat(shuffleMaster.externallyReleasedPartitions).hasSize(1); + assertThat(shuffleMaster.externallyReleasedPartitions.remove()) + .isEqualTo(resultPartitionId); + assertThat(partitionTracker.isTrackingPartitionsFor(tmId)).isFalse(); } @Test - public void testReleaseCallsWithoutLocalResources() { + void testReleaseCallsWithoutLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue releaseCalls = new ArrayBlockingQueue<>(4); @@ -145,18 +140,18 @@ public void testReleaseCallsWithoutLocalResources() { tmId, AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId, false)); - assertThat(partitionTracker.isTrackingPartitionsFor(tmId), is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(tmId)).isFalse(); partitionTracker.stopTrackingAndReleasePartitions(Arrays.asList(resultPartitionId)); - assertEquals(0, releaseCalls.size()); - assertEquals(0, promoteCalls.size()); - assertEquals(1, shuffleMaster.externallyReleasedPartitions.size()); - assertThat(shuffleMaster.externallyReleasedPartitions, contains(resultPartitionId)); + assertThat(releaseCalls).isEmpty(); + assertThat(promoteCalls).isEmpty(); + assertThat(shuffleMaster.externallyReleasedPartitions).hasSize(1); + assertThat(shuffleMaster.externallyReleasedPartitions).contains(resultPartitionId); } @Test - public void testStopTrackingIssuesNoReleaseCalls() { + void testStopTrackingIssuesNoReleaseCalls() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue releaseCalls = new ArrayBlockingQueue<>(4); @@ -180,13 +175,13 @@ public void testStopTrackingIssuesNoReleaseCalls() { partitionTracker.stopTrackingPartitionsFor(taskExecutorId1); - assertEquals(0, releaseCalls.size()); - assertEquals(0, promoteCalls.size()); - assertEquals(0, shuffleMaster.externallyReleasedPartitions.size()); + assertThat(releaseCalls).isEmpty(); + assertThat(promoteCalls).isEmpty(); + assertThat(shuffleMaster.externallyReleasedPartitions).isEmpty(); } @Test - public void testTrackingInternalAndExternalPartitionsByTmId() { + void testTrackingInternalAndExternalPartitionsByTmId() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue releaseCalls = new ArrayBlockingQueue<>(4); @@ -209,35 +204,35 @@ public void testTrackingInternalAndExternalPartitionsByTmId() { AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId2, false)); // No local resource is occupied - assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId), is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId)).isFalse(); partitionTracker.startTrackingPartition( taskExecutorId, AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId1, true)); // Local resource is occupied - assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId), is(true)); + assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId)).isTrue(); assertThat( - partitionTracker.getAllTrackedPartitions().stream() - .map(desc -> desc.getShuffleDescriptor().getResultPartitionID()) - .collect(Collectors.toList()), - containsInAnyOrder(resultPartitionId1, resultPartitionId2)); + partitionTracker.getAllTrackedPartitions().stream() + .map(desc -> desc.getShuffleDescriptor().getResultPartitionID()) + .collect(Collectors.toList())) + .containsExactlyInAnyOrder(resultPartitionId1, resultPartitionId2); partitionTracker.stopTrackingPartitionsFor(taskExecutorId); - assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId), is(false)); - assertThat(partitionTracker.isPartitionTracked(resultPartitionId1), is(false)); - assertThat(partitionTracker.isPartitionTracked(resultPartitionId2), is(true)); + assertThat(partitionTracker.isTrackingPartitionsFor(taskExecutorId)).isFalse(); + assertThat(partitionTracker.isPartitionTracked(resultPartitionId1)).isFalse(); + assertThat(partitionTracker.isPartitionTracked(resultPartitionId2)).isTrue(); assertThat( - Iterables.getOnlyElement(partitionTracker.getAllTrackedPartitions()) - .getShuffleDescriptor() - .getResultPartitionID(), - is(resultPartitionId2)); + Iterables.getOnlyElement(partitionTracker.getAllTrackedPartitions()) + .getShuffleDescriptor() + .getResultPartitionID()) + .isEqualTo(resultPartitionId2); } @Test - public void testGetJobPartitionClusterPartition() { + void testGetJobPartitionClusterPartition() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue releaseCalls = new ArrayBlockingQueue<>(4); @@ -264,14 +259,14 @@ public void testGetJobPartitionClusterPartition() { partitionTracker.startTrackingPartition(taskExecutorId, clusterPartition); partitionTracker.startTrackingPartition(taskExecutorId, jobPartition); - Assertions.assertThat(partitionTracker.getAllTrackedNonClusterPartitions()) + assertThat(partitionTracker.getAllTrackedNonClusterPartitions()) .containsExactly(jobPartition); - Assertions.assertThat(partitionTracker.getAllTrackedClusterPartitions()) + assertThat(partitionTracker.getAllTrackedClusterPartitions()) .containsExactly(clusterPartition); } @Test - public void testGetShuffleDescriptors() { + void testGetShuffleDescriptors() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); IntermediateDataSetID intermediateDataSetId = new IntermediateDataSetID(); @@ -290,13 +285,12 @@ public void testGetShuffleDescriptors() { partitionTracker.connectToResourceManager(resourceManagerGateway); partitionTracker.getClusterPartitionShuffleDescriptors(intermediateDataSetId); - assertThat( - resourceManagerGateway.requestedIntermediateDataSetIds, - contains(intermediateDataSetId)); + assertThat(resourceManagerGateway.requestedIntermediateDataSetIds) + .contains(intermediateDataSetId); } - @Test(expected = NullPointerException.class) - public void testGetShuffleDescriptorsBeforeConnectToResourceManager() { + @Test + void testGetShuffleDescriptorsBeforeConnectToResourceManager() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); IntermediateDataSetID intermediateDataSetId = new IntermediateDataSetID(); @@ -310,11 +304,16 @@ public void testGetShuffleDescriptorsBeforeConnectToResourceManager() { Optional.of( createTaskExecutorGateway( resourceId, releaseCalls, promoteCalls))); - partitionTracker.getClusterPartitionShuffleDescriptors(intermediateDataSetId); + + assertThatThrownBy( + () -> + partitionTracker.getClusterPartitionShuffleDescriptors( + intermediateDataSetId)) + .isInstanceOf(NullPointerException.class); } @Test - public void testReleaseJobPartitionPromoteClusterPartition() { + void testReleaseJobPartitionPromoteClusterPartition() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); @@ -368,24 +367,24 @@ public void testReleaseJobPartitionPromoteClusterPartition() { Arrays.asList(clusterPartitionId0, clusterPartitionId1)); // Exactly one call should have been made to the hosting task executor - assertEquals(1, taskExecutorReleaseCalls.size()); - assertEquals(1, taskExecutorPromoteCalls.size()); + assertThat(taskExecutorReleaseCalls).hasSize(1); + assertThat(taskExecutorPromoteCalls).hasSize(1); final ReleaseCall releaseCall = taskExecutorReleaseCalls.remove(); final PromoteCall promoteCall = taskExecutorPromoteCalls.remove(); // One local partition released and one local partition promoted. - assertEquals( - jobPartitionId0, Iterables.getOnlyElement(releaseCall.getReleasedPartitions())); - assertEquals( - clusterPartitionId0, Iterables.getOnlyElement(promoteCall.getPromotedPartitions())); + assertThat(Iterables.getOnlyElement(releaseCall.getReleasedPartitions())) + .isEqualTo(jobPartitionId0); + assertThat(Iterables.getOnlyElement(promoteCall.getPromotedPartitions())) + .isEqualTo(clusterPartitionId0); // Both internal and external partitions will be fed into shuffle-master for releasing. Collection externallyReleasedPartitions = new ArrayList<>(shuffleMaster.externallyReleasedPartitions); - assertThat( - externallyReleasedPartitions, containsInAnyOrder(jobPartitionId0, jobPartitionId1)); + assertThat(externallyReleasedPartitions) + .containsExactlyInAnyOrder(jobPartitionId0, jobPartitionId1); } private static TaskExecutorGateway createTaskExecutorGateway( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 4281a7f344d66..368fd37c1a677 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -32,18 +32,17 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.testutils.MiniClusterResource; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.util.TestLogger; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import java.nio.ByteBuffer; /** Test for consuming a pipelined result only partially. */ -public class PartialConsumePipelinedResultTest extends TestLogger { +class PartialConsumePipelinedResultTest { // Test configuration private static final int NUMBER_OF_TMS = 1; @@ -52,9 +51,9 @@ public class PartialConsumePipelinedResultTest extends TestLogger { private static final int NUMBER_OF_NETWORK_BUFFERS = 128; - @ClassRule - public static final MiniClusterResource MINI_CLUSTER_RESOURCE = - new MiniClusterResource( + @RegisterExtension + private static final InternalMiniClusterExtension MINI_CLUSTER_RESOURCE = + new InternalMiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setConfiguration(getFlinkConfiguration()) .setNumberTaskManagers(NUMBER_OF_TMS) @@ -81,7 +80,7 @@ private static Configuration getFlinkConfiguration() { * @see FLINK-1930 */ @Test - public void testPartialConsumePipelinedResultReceiver() throws Exception { + void testPartialConsumePipelinedResultReceiver() throws Exception { final JobVertex sender = new JobVertex("Sender"); sender.setInvokableClass(SlowBufferSender.class); sender.setParallelism(PARALLELISM); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java index 16b3e6df770ab..2f5ac36ce556b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java @@ -21,8 +21,10 @@ import org.apache.flink.runtime.io.disk.NoOpFileChannelManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.testutils.junit.extensions.parameterized.NoOpTestExtension; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; @@ -30,13 +32,11 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest.assertContent; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderAndConsumerTest.toByteBuffer; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link PipelinedApproximateSubpartition}. */ -public class PipelinedApproximateSubpartitionTest extends PipelinedSubpartitionTest { +@ExtendWith(NoOpTestExtension.class) +class PipelinedApproximateSubpartitionTest extends PipelinedSubpartitionTest { private static final int BUFFER_SIZE = 4 * Integer.BYTES; @Override @@ -44,30 +44,30 @@ PipelinedSubpartition createSubpartition() throws Exception { return createPipelinedApproximateSubpartition(); } - @Test + @TestTemplate @Override - public void testIllegalReadViewRequest() { + void testIllegalReadViewRequest() { // This is one of the main differences between PipelinedApproximateSubpartition and // PipelinedSubpartition // PipelinedApproximateSubpartition allows to recreate a view (release the old view first) } - @Test - public void testRecreateReadView() throws Exception { + @TestTemplate + void testRecreateReadView() throws Exception { final PipelinedApproximateSubpartition subpartition = createPipelinedApproximateSubpartition(); // first request - assertNotNull(subpartition.createReadView(() -> {})); - assertFalse(subpartition.isPartialBufferCleanupRequired()); + assertThat(subpartition.createReadView(() -> {})).isNotNull(); + assertThat(subpartition.isPartialBufferCleanupRequired()).isFalse(); // reconnecting request - assertNotNull(subpartition.createReadView(() -> {})); - assertTrue(subpartition.isPartialBufferCleanupRequired()); + assertThat(subpartition.createReadView(() -> {})).isNotNull(); + assertThat(subpartition.isPartialBufferCleanupRequired()).isTrue(); } - @Test - public void testSkipPartialDataEndsInBufferWithNoMoreData() throws Exception { + @TestTemplate + void testSkipPartialDataEndsInBufferWithNoMoreData() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -76,14 +76,14 @@ public void testSkipPartialDataEndsInBufferWithNoMoreData() throws Exception { assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 0, 1, 2, 3); subpartition.setIsPartialBufferCleanupRequired(); - assertNull(subpartition.pollBuffer()); + assertThat(subpartition.pollBuffer()).isNull(); writer.emitRecord(toByteBuffer(8, 9), 0); assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 8, 9); } - @Test - public void testSkipPartialDataEndsInBufferWithMoreData() throws Exception { + @TestTemplate + void testSkipPartialDataEndsInBufferWithMoreData() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -97,8 +97,8 @@ public void testSkipPartialDataEndsInBufferWithMoreData() throws Exception { assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 8, 9); } - @Test - public void testSkipPartialDataStartWithFullRecord() throws Exception { + @TestTemplate + void testSkipPartialDataStartWithFullRecord() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -112,8 +112,8 @@ public void testSkipPartialDataStartWithFullRecord() throws Exception { assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 42, 8, 9); } - @Test - public void testSkipPartialDataStartWithinBuffer() throws Exception { + @TestTemplate + void testSkipPartialDataStartWithinBuffer() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -130,8 +130,8 @@ public void testSkipPartialDataStartWithinBuffer() throws Exception { assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 11); } - @Test - public void testSkipPartialDataLongRecordOccupyEntireBuffer() throws Exception { + @TestTemplate + void testSkipPartialDataLongRecordOccupyEntireBuffer() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -141,10 +141,10 @@ public void testSkipPartialDataLongRecordOccupyEntireBuffer() throws Exception { assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 0, 1, 2, 3); subpartition.setIsPartialBufferCleanupRequired(); - assertNull(subpartition.pollBuffer()); + assertThat(subpartition.pollBuffer()).isNull(); } - @Test + @TestTemplate public void testSkipPartialDataLongRecordOccupyEntireBufferWithMoreData() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = @@ -161,14 +161,14 @@ public void testSkipPartialDataLongRecordOccupyEntireBufferWithMoreData() throws // release again subpartition.setIsPartialBufferCleanupRequired(); // 102 is cleaned up - assertNull(subpartition.pollBuffer()); + assertThat(subpartition.pollBuffer()).isNull(); writer.emitRecord(toByteBuffer(200, 201, 202, 203), 0); assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 200, 201, 202); assertContent(requireNonNull(subpartition.pollBuffer()).buffer(), null, 203); } - @Test + @TestTemplate public void testSkipPartialDataLongRecordEndWithBuffer() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java index e10e980b073f3..34b7139c4b3b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java @@ -18,12 +18,12 @@ package org.apache.flink.runtime.io.network.partition; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Additional tests for {@link PipelinedApproximateSubpartitionView} which require an availability @@ -31,12 +31,12 @@ * * @see PipelinedSubpartitionTest */ -public class PipelinedApproximateSubpartitionWithReadViewTest +class PipelinedApproximateSubpartitionWithReadViewTest extends PipelinedSubpartitionWithReadViewTest { - @Before + @BeforeEach @Override - public void before() throws IOException { + void before() throws IOException { setup(ResultPartitionType.PIPELINED_APPROXIMATE); subpartition = new PipelinedApproximateSubpartition(0, 2, resultPartition); availablityListener = new AwaitableBufferAvailablityListener(); @@ -45,12 +45,13 @@ public void before() throws IOException { @Test @Override - public void testRelease() { + void testRelease() { readView.releaseAllResources(); - assertTrue( - resultPartition - .getPartitionManager() - .getUnreleasedPartitions() - .contains(resultPartition.getPartitionId())); + assertThat( + resultPartition + .getPartitionManager() + .getUnreleasedPartitions() + .contains(resultPartition.getPartitionId())) + .isTrue(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java index ed169f9322a75..1631894057016 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedResultPartitionReleaseOnConsumptionTest.java @@ -18,21 +18,19 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.io.network.buffer.UnpooledBufferPool; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.ByteBuffer; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link PipelinedResultPartitionReleaseOnConsumptionTest}. */ -public class PipelinedResultPartitionReleaseOnConsumptionTest extends TestLogger { +class PipelinedResultPartitionReleaseOnConsumptionTest { @Test - public void testConsumptionBasedPartitionRelease() { + void testConsumptionBasedPartitionRelease() { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() @@ -44,15 +42,15 @@ public void testConsumptionBasedPartitionRelease() { manager.registerResultPartition(partition); partition.onConsumedSubpartition(0); - assertFalse(partition.isReleased()); + assertThat(partition.isReleased()).isFalse(); partition.onConsumedSubpartition(1); partition.close(); - assertTrue(partition.isReleased()); + assertThat(partition.isReleased()).isTrue(); } @Test - public void testConsumptionBeforePartitionClose() throws IOException { + void testConsumptionBeforePartitionClose() throws IOException { final ResultPartition partition = new ResultPartitionBuilder() .setResultPartitionType(ResultPartitionType.PIPELINED) @@ -63,14 +61,14 @@ public void testConsumptionBeforePartitionClose() throws IOException { partition.setup(); partition.emitRecord(ByteBuffer.allocate(16), 0); partition.onConsumedSubpartition(0); - assertFalse(partition.isReleased()); + assertThat(partition.isReleased()).isFalse(); partition.emitRecord(ByteBuffer.allocate(16), 0); partition.close(); - assertTrue(partition.isReleased()); + assertThat(partition.isReleased()).isTrue(); } @Test - public void testMultipleReleaseCallsAreIdempotent() { + void testMultipleReleaseCallsAreIdempotent() { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() @@ -83,11 +81,11 @@ public void testMultipleReleaseCallsAreIdempotent() { partition.onConsumedSubpartition(0); partition.onConsumedSubpartition(0); - assertFalse(partition.isReleased()); + assertThat(partition.isReleased()).isFalse(); } @Test - public void testReleaseAfterIdempotentCalls() { + void testReleaseAfterIdempotentCalls() { final ResultPartitionManager manager = new ResultPartitionManager(); final ResultPartition partition = new ResultPartitionBuilder() @@ -102,6 +100,6 @@ public void testReleaseAfterIdempotentCalls() { partition.onConsumedSubpartition(1); partition.close(); - assertTrue(partition.isReleased()); + assertThat(partition.isReleased()).isTrue(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 295b49d75d5b3..e0b4f3fd5a1a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -35,15 +35,15 @@ import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer; import org.apache.flink.runtime.io.network.util.TestSubpartitionProducer; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; -import org.apache.flink.testutils.executor.TestExecutorResource; +import org.apache.flink.testutils.executor.TestExecutorExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.NoOpTestExtension; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; import java.io.IOException; import java.util.ArrayList; @@ -55,11 +55,10 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.assertj.core.api.Fail.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -70,12 +69,13 @@ * * @see PipelinedSubpartitionWithReadViewTest */ +@ExtendWith(NoOpTestExtension.class) public class PipelinedSubpartitionTest extends SubpartitionTestBase { /** Executor service for concurrent produce/consume tests. */ - @ClassRule - public static final TestExecutorResource EXECUTOR_RESOURCE = - new TestExecutorResource<>(() -> Executors.newCachedThreadPool()); + @RegisterExtension + static final TestExecutorExtension EXECUTOR_EXTENSION = + new TestExecutorExtension<>(Executors::newCachedThreadPool); @Override PipelinedSubpartition createSubpartition() throws Exception { @@ -85,16 +85,16 @@ PipelinedSubpartition createSubpartition() throws Exception { @Override ResultSubpartition createFailingWritesSubpartition() throws Exception { // the tests relating to this are currently not supported by the PipelinedSubpartition - Assume.assumeTrue(false); + assumeThat(false).isTrue(); return null; } - @Test - public void testIllegalReadViewRequest() throws Exception { + @TestTemplate + void testIllegalReadViewRequest() throws Exception { final PipelinedSubpartition subpartition = createSubpartition(); // Successful request - assertNotNull(subpartition.createReadView(new NoOpBufferAvailablityListener())); + assertThat(subpartition.createReadView(new NoOpBufferAvailablityListener())).isNotNull(); try { subpartition.createReadView(new NoOpBufferAvailablityListener()); @@ -105,38 +105,38 @@ public void testIllegalReadViewRequest() throws Exception { } /** Verifies that the isReleased() check of the view checks the parent subpartition. */ - @Test - public void testIsReleasedChecksParent() { + @TestTemplate + void testIsReleasedChecksParent() { PipelinedSubpartition subpartition = mock(PipelinedSubpartition.class); PipelinedSubpartitionView reader = new PipelinedSubpartitionView(subpartition, mock(BufferAvailabilityListener.class)); - assertFalse(reader.isReleased()); + assertThat(reader.isReleased()).isFalse(); verify(subpartition, times(1)).isReleased(); when(subpartition.isReleased()).thenReturn(true); - assertTrue(reader.isReleased()); + assertThat(reader.isReleased()).isTrue(); verify(subpartition, times(2)).isReleased(); } - @Test - public void testConcurrentFastProduceAndFastConsume() throws Exception { + @TestTemplate + void testConcurrentFastProduceAndFastConsume() throws Exception { testProduceConsume(false, false); } - @Test - public void testConcurrentFastProduceAndSlowConsume() throws Exception { + @TestTemplate + void testConcurrentFastProduceAndSlowConsume() throws Exception { testProduceConsume(false, true); } - @Test - public void testConcurrentSlowProduceAndFastConsume() throws Exception { + @TestTemplate + void testConcurrentSlowProduceAndFastConsume() throws Exception { testProduceConsume(true, false); } - @Test - public void testConcurrentSlowProduceAndSlowConsume() throws Exception { + @TestTemplate + void testConcurrentSlowProduceAndSlowConsume() throws Exception { testProduceConsume(true, true); } @@ -183,12 +183,12 @@ public BufferAndChannel getNextBuffer() throws Exception { @Override public void onBuffer(Buffer buffer) { final MemorySegment segment = buffer.getMemorySegment(); - assertEquals(segment.size(), buffer.getSize()); + assertThat(buffer.getSize()).isEqualTo(segment.size()); int expected = numberOfBuffers * (segment.size() / 4); for (int i = 0; i < segment.size(); i += 4) { - assertEquals(expected, segment.getInt(i)); + assertThat(segment.getInt(i)).isEqualTo(expected); expected++; } @@ -215,24 +215,26 @@ public void onEvent(AbstractEvent event) { CompletableFuture producerResult = CompletableFuture.supplyAsync( - CheckedSupplier.unchecked(producer::call), EXECUTOR_RESOURCE.getExecutor()); + CheckedSupplier.unchecked(producer::call), + EXECUTOR_EXTENSION.getExecutor()); CompletableFuture consumerResult = CompletableFuture.supplyAsync( - CheckedSupplier.unchecked(consumer::call), EXECUTOR_RESOURCE.getExecutor()); + CheckedSupplier.unchecked(consumer::call), + EXECUTOR_EXTENSION.getExecutor()); FutureUtils.waitForAll(Arrays.asList(producerResult, consumerResult)) .get(60_000L, TimeUnit.MILLISECONDS); } /** Tests cleanup of {@link PipelinedSubpartition#release()} with no read view attached. */ - @Test - public void testCleanupReleasedPartitionNoView() throws Exception { + @TestTemplate + void testCleanupReleasedPartitionNoView() throws Exception { testCleanupReleasedPartition(false); } /** Tests cleanup of {@link PipelinedSubpartition#release()} with a read view attached. */ - @Test - public void testCleanupReleasedPartitionWithView() throws Exception { + @TestTemplate + void testCleanupReleasedPartitionWithView() throws Exception { testCleanupReleasedPartition(true); } @@ -252,7 +254,7 @@ private void testCleanupReleasedPartition(boolean createView) throws Exception { try { partition.add(buffer1); partition.add(buffer2); - assertEquals(2, partition.getNumberOfQueuedBuffers()); + assertThat(partition.getNumberOfQueuedBuffers()).isEqualTo(2); // create the read view first ResultSubpartitionView view = null; @@ -261,13 +263,13 @@ private void testCleanupReleasedPartition(boolean createView) throws Exception { } partition.release(); - assertEquals(0, partition.getNumberOfQueuedBuffers()); + assertThat(partition.getNumberOfQueuedBuffers()).isEqualTo(0); - assertTrue(partition.isReleased()); + assertThat(partition.isReleased()).isTrue(); if (createView) { - assertTrue(view.isReleased()); + assertThat(view.isReleased()).isTrue(); } - assertTrue(buffer1.isRecycled()); + assertThat(buffer1.isRecycled()).isTrue(); } finally { buffer1Recycled = buffer1.isRecycled(); if (!buffer1Recycled) { @@ -279,86 +281,91 @@ private void testCleanupReleasedPartition(boolean createView) throws Exception { } } if (!buffer1Recycled) { - Assert.fail("buffer 1 not recycled"); + fail("buffer 1 not recycled"); } if (!buffer2Recycled) { - Assert.fail("buffer 2 not recycled"); + fail("buffer 2 not recycled"); } - assertEquals(2, partition.getTotalNumberOfBuffersUnsafe()); - assertEquals(0, partition.getTotalNumberOfBytesUnsafe()); // buffer data is never consumed + assertThat(partition.getTotalNumberOfBuffersUnsafe()).isEqualTo(2); + assertThat(partition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(0); // buffer data is never consumed } - @Test - public void testReleaseParent() throws Exception { + @TestTemplate + void testReleaseParent() throws Exception { final ResultSubpartition partition = createSubpartition(); verifyViewReleasedAfterParentRelease(partition); } - @Test - public void testNumberOfQueueBuffers() throws Exception { + @TestTemplate + void testNumberOfQueueBuffers() throws Exception { final PipelinedSubpartition subpartition = createSubpartition(); subpartition.add(createFilledFinishedBufferConsumer(4096)); - assertEquals(1, subpartition.getNumberOfQueuedBuffers()); + assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(1); subpartition.add(createFilledFinishedBufferConsumer(4096)); - assertEquals(2, subpartition.getNumberOfQueuedBuffers()); + assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(2); subpartition.getNextBuffer(); - assertEquals(1, subpartition.getNumberOfQueuedBuffers()); + assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(1); } - @Test - public void testNewBufferSize() throws Exception { + @TestTemplate + void testNewBufferSize() throws Exception { // given: Buffer size equal to integer max value by default. final PipelinedSubpartition subpartition = createSubpartition(); - assertEquals(Integer.MAX_VALUE, subpartition.add(createFilledFinishedBufferConsumer(4))); + assertThat(subpartition.add(createFilledFinishedBufferConsumer(4))) + .isEqualTo(Integer.MAX_VALUE); // when: Changing buffer size. subpartition.bufferSize(42); // then: Changes successfully applied. - assertEquals(42, subpartition.add(createFilledFinishedBufferConsumer(4))); + assertThat(subpartition.add(createFilledFinishedBufferConsumer(4))).isEqualTo(42); } - @Test(expected = IllegalArgumentException.class) - public void testNegativeNewBufferSize() throws Exception { + @TestTemplate + void testNegativeNewBufferSize() throws Exception { // given: Buffer size equal to integer max value by default. final PipelinedSubpartition subpartition = createSubpartition(); - assertEquals(Integer.MAX_VALUE, subpartition.add(createFilledFinishedBufferConsumer(4))); + assertThat(subpartition.add(createFilledFinishedBufferConsumer(4))) + .isEqualTo(Integer.MAX_VALUE); // when: Changing buffer size to the negative value. - subpartition.bufferSize(-1); + assertThatThrownBy(() -> subpartition.bufferSize(-1)) + .isInstanceOf(IllegalArgumentException.class); } - @Test - public void testNegativeBufferSizeAsSignOfAddingFail() throws Exception { + @TestTemplate + void testNegativeBufferSizeAsSignOfAddingFail() throws Exception { // given: Buffer size equal to integer max value by default. final PipelinedSubpartition subpartition = createSubpartition(); - assertEquals(Integer.MAX_VALUE, subpartition.add(createFilledFinishedBufferConsumer(4))); + assertThat(subpartition.add(createFilledFinishedBufferConsumer(4))) + .isEqualTo(Integer.MAX_VALUE); // when: Finishing the subpartition which make following adding impossible. subpartition.finish(); // then: -1 should be return because the add operation fails. - assertEquals(-1, subpartition.add(createFilledFinishedBufferConsumer(4))); + assertThat(subpartition.add(createFilledFinishedBufferConsumer(4))).isEqualTo(-1); } - @Test - public void testProducerFailedException() { + @TestTemplate + void testProducerFailedException() { PipelinedSubpartition subpartition = new FailurePipelinedSubpartition(0, 2, PartitionTestUtils.createPartition()); ResultSubpartitionView view = subpartition.createReadView(new NoOpBufferAvailablityListener()); - assertNotNull(view.getFailureCause()); - assertTrue(view.getFailureCause() instanceof CancelTaskException); + assertThat(view.getFailureCause()).isNotNull(); + assertThat(view.getFailureCause()).isInstanceOf(CancelTaskException.class); } - @Test - public void testConsumeTimeoutableCheckpointBarrierQuickly() throws Exception { + @TestTemplate + void testConsumeTimeoutableCheckpointBarrierQuickly() throws Exception { PipelinedSubpartition subpartition = createSubpartition(); subpartition.setChannelStateWriter(ChannelStateWriter.NO_OP); assertSubpartitionChannelStateFuturesAndQueuedBuffers(subpartition, null, true, 0, false); @@ -379,7 +386,7 @@ private void testConsumeQuicklyWithNDataBuffers( subpartition.add(createFilledFinishedBufferConsumer(4096)); } subpartition.add(getTimeoutableBarrierBuffer(checkpointId)); - assertEquals(checkpointId, subpartition.getChannelStateCheckpointId()); + assertThat(subpartition.getChannelStateCheckpointId()).isEqualTo(checkpointId); CompletableFuture> channelStateFuture = subpartition.getChannelStateFuture(); assertSubpartitionChannelStateFuturesAndQueuedBuffers( subpartition, channelStateFuture, false, numberOfDataBuffers + 1, false); @@ -393,12 +400,12 @@ private void testConsumeQuicklyWithNDataBuffers( assertSubpartitionChannelStateFuturesAndQueuedBuffers( subpartition, channelStateFuture, true, 0, true); - assertTrue(channelStateFuture.get().isEmpty()); + assertThat(channelStateFuture.get().isEmpty()).isTrue(); subpartition.resumeConsumption(); } - @Test - public void testTimeoutAlignedToUnalignedBarrier() throws Exception { + @TestTemplate + void testTimeoutAlignedToUnalignedBarrier() throws Exception { PipelinedSubpartition subpartition = createSubpartition(); subpartition.setChannelStateWriter(ChannelStateWriter.NO_OP); assertSubpartitionChannelStateFuturesAndQueuedBuffers(subpartition, null, true, 0, false); @@ -422,7 +429,7 @@ private void testTimeoutWithNDataBuffers( } subpartition.add(getTimeoutableBarrierBuffer(checkpointId)); - assertEquals(checkpointId, subpartition.getChannelStateCheckpointId()); + assertThat(subpartition.getChannelStateCheckpointId()).isEqualTo(checkpointId); CompletableFuture> channelStateFuture = subpartition.getChannelStateFuture(); assertSubpartitionChannelStateFuturesAndQueuedBuffers( subpartition, channelStateFuture, false, numberOfDataBuffers + 1, false); @@ -436,25 +443,25 @@ private void testTimeoutWithNDataBuffers( pollBufferAndCheckType(subpartition, Buffer.DataType.DATA_BUFFER); } - assertEquals(expectedBuffers, channelStateFuture.get()); + assertThat(channelStateFuture.get()).isEqualTo(expectedBuffers); } private void pollBufferAndCheckType( PipelinedSubpartition subpartition, Buffer.DataType dataType) { ResultSubpartition.BufferAndBacklog barrierBuffer = subpartition.pollBuffer(); - assertNotNull(barrierBuffer); - assertEquals(dataType, barrierBuffer.buffer().getDataType()); + assertThat(barrierBuffer).isNotNull(); + assertThat(barrierBuffer.buffer().getDataType()).isEqualTo(dataType); } - @Test - public void testConcurrentTimeoutableCheckpointBarrier() throws Exception { + @TestTemplate + void testConcurrentTimeoutableCheckpointBarrier() throws Exception { PipelinedSubpartition subpartition = createSubpartition(); subpartition.setChannelStateWriter(ChannelStateWriter.NO_OP); subpartition.add(getTimeoutableBarrierBuffer(10L)); - assertEquals(10L, subpartition.getChannelStateCheckpointId()); + assertThat(subpartition.getChannelStateCheckpointId()).isEqualTo(10L); CompletableFuture> checkpointFuture10 = subpartition.getChannelStateFuture(); - assertNotNull(checkpointFuture10); + assertThat(checkpointFuture10).isNotNull(); try { // It should fail due to currently does not support concurrent unaligned checkpoints. @@ -483,10 +490,11 @@ private void assertSubpartitionChannelStateFuturesAndQueuedBuffers( boolean channelStateFutureIsNull, long numberOfQueuedBuffers, boolean expectedFutureIsDone) { - assertEquals(channelStateFutureIsNull, subpartition.getChannelStateFuture() == null); - assertEquals(numberOfQueuedBuffers, subpartition.getNumberOfQueuedBuffers()); + assertThat(subpartition.getChannelStateFuture() == null) + .isEqualTo(channelStateFutureIsNull); + assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(numberOfQueuedBuffers); if (channelStateFuture != null) { - assertEquals(expectedFutureIsDone, channelStateFuture.isDone()); + assertThat(channelStateFuture.isDone()).isEqualTo(expectedFutureIsDone); } } @@ -503,15 +511,15 @@ private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) ResultSubpartitionView view = partition.createReadView(listener); // The added bufferConsumer and end-of-partition event - assertNotNull(view.getNextBuffer()); - assertNotNull(view.getNextBuffer()); + assertThat(view.getNextBuffer()).isNotNull(); + assertThat(view.getNextBuffer()).isNotNull(); // Release the parent - assertFalse(view.isReleased()); + assertThat(view.isReleased()).isFalse(); partition.release(); // Verify that parent release is reflected at partition view - assertTrue(view.isReleased()); + assertThat(view.isReleased()).isTrue(); } public static PipelinedSubpartition createPipelinedSubpartition() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index 3d15bb135f170..f80035435c8e7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -32,13 +32,14 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import javax.annotation.Nullable; @@ -53,14 +54,8 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledUnfinishedBufferConsumer; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.BUFFER_SIZE; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.core.IsInstanceOf.instanceOf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Additional tests for {@link PipelinedSubpartition} which require an availability listener and a @@ -68,7 +63,7 @@ * * @see PipelinedSubpartitionTest */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class PipelinedSubpartitionWithReadViewTest { ResultPartition resultPartition; @@ -76,76 +71,76 @@ public class PipelinedSubpartitionWithReadViewTest { AwaitableBufferAvailablityListener availablityListener; PipelinedSubpartitionView readView; - @Parameterized.Parameter public boolean compressionEnabled; + @Parameter public boolean compressionEnabled; - @Parameterized.Parameters(name = "compressionEnabled = {0}") - public static Boolean[] parameters() { - return new Boolean[] {false, true}; + @Parameters(name = "compressionEnabled = {0}") + public static List parameters() { + return Arrays.asList(false, true); } - @Before - public void before() throws IOException { + @BeforeEach + void before() throws IOException { setup(ResultPartitionType.PIPELINED); subpartition = new PipelinedSubpartition(0, 2, resultPartition); availablityListener = new AwaitableBufferAvailablityListener(); readView = subpartition.createReadView(availablityListener); } - @After - public void tearDown() { + @AfterEach + void tearDown() { readView.releaseAllResources(); subpartition.release(); } - @Test(expected = IllegalStateException.class) - public void testAddTwoNonFinishedBuffer() throws IOException { + @TestTemplate + void testAddTwoNonFinishedBuffer() throws IOException { subpartition.add(createBufferBuilder().createBufferConsumer()); subpartition.add(createBufferBuilder().createBufferConsumer()); - assertNull(readView.getNextBuffer()); + assertThatThrownBy(readView::getNextBuffer).isInstanceOf(IllegalStateException.class); } - @Test - public void testRelease() { + @TestTemplate + void testRelease() { readView.releaseAllResources(); resultPartition.close(); - assertFalse( - resultPartition - .getPartitionManager() - .getUnreleasedPartitions() - .contains(resultPartition.getPartitionId())); + assertThat( + resultPartition + .getPartitionManager() + .getUnreleasedPartitions() + .contains(resultPartition.getPartitionId())) + .isFalse(); } - @Test - public void testAddEmptyNonFinishedBuffer() throws IOException { - assertEquals(0, availablityListener.getNumNotifications()); + @TestTemplate + void testAddEmptyNonFinishedBuffer() throws IOException { + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); BufferBuilder bufferBuilder = createBufferBuilder(); subpartition.add(bufferBuilder.createBufferConsumer()); - assertEquals(0, availablityListener.getNumNotifications()); - assertNull(readView.getNextBuffer()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(readView.getNextBuffer()).isNull(); bufferBuilder.finish(); bufferBuilder = createBufferBuilder(); subpartition.add(bufferBuilder.createBufferConsumer()); - assertEquals(1, subpartition.getBuffersInBacklogUnsafe()); - assertEquals( - 1, - availablityListener - .getNumNotifications()); // notification from finishing previous buffer. - assertNull(readView.getNextBuffer()); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + // notification from finishing previous buffer. + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + + assertThat(readView.getNextBuffer()).isNull(); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); } - @Test - public void testAddNonEmptyNotFinishedBuffer() throws Exception { - assertEquals(0, availablityListener.getNumNotifications()); + @TestTemplate + void testAddNonEmptyNotFinishedBuffer() throws Exception { + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // note that since the buffer builder is not finished, there is still a retained instance! - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); assertNextBuffer(readView, 1024, false, 0, false, false); } @@ -153,13 +148,13 @@ public void testAddNonEmptyNotFinishedBuffer() throws Exception { * Normally moreAvailable flag from InputChannel should ignore non finished BufferConsumers, * otherwise we would busy loop on the unfinished BufferConsumers. */ - @Test - public void testUnfinishedBufferBehindFinished() throws Exception { + @TestTemplate + void testUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished - assertEquals(1, subpartition.getBuffersInBacklogUnsafe()); - assertThat(availablityListener.getNumNotifications(), greaterThan(0L)); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isGreaterThan(0L); assertNextBuffer(readView, 1025, false, 0, false, true); // not notified, but we could still access the unfinished buffer assertNextBuffer(readView, 1024, false, 0, false, false); @@ -170,20 +165,20 @@ public void testUnfinishedBufferBehindFinished() throws Exception { * After flush call unfinished BufferConsumers should be reported as available, otherwise we * might not flush some of the data. */ - @Test - public void testFlushWithUnfinishedBufferBehindFinished() throws Exception { + @TestTemplate + void testFlushWithUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished long oldNumNotifications = availablityListener.getNumNotifications(); - assertEquals(1, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); subpartition.flush(); // buffer queue is > 1, should already be notified, no further notification necessary - assertThat(oldNumNotifications, greaterThan(0L)); - assertEquals(oldNumNotifications, availablityListener.getNumNotifications()); + assertThat(oldNumNotifications).isGreaterThan(0L); + assertThat(availablityListener.getNumNotifications()).isEqualTo(oldNumNotifications); - assertEquals(2, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(2); assertNextBuffer(readView, 1025, true, 1, false, true); assertNextBuffer(readView, 1024, false, 0, false, false); assertNoNextBuffer(readView); @@ -192,176 +187,169 @@ public void testFlushWithUnfinishedBufferBehindFinished() throws Exception { /** * A flush call with a buffer size of 1 should always notify consumers (unless already flushed). */ - @Test - public void testFlushWithUnfinishedBufferBehindFinished2() throws Exception { + @TestTemplate + void testFlushWithUnfinishedBufferBehindFinished2() throws Exception { // no buffers -> no notification or any other effects subpartition.flush(); - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished - assertEquals(1, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); assertNextBuffer(readView, 1025, false, 0, false, true); long oldNumNotifications = availablityListener.getNumNotifications(); subpartition.flush(); // buffer queue is 1 again -> need to flush - assertEquals(oldNumNotifications + 1, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(oldNumNotifications + 1); subpartition.flush(); // calling again should not flush again - assertEquals(oldNumNotifications + 1, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(oldNumNotifications + 1); - assertEquals(1, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); assertNextBuffer(readView, 1024, false, 0, false, false); assertNoNextBuffer(readView); } - @Test - public void testMultipleEmptyBuffers() throws Exception { - assertEquals(0, availablityListener.getNumNotifications()); + @TestTemplate + void testMultipleEmptyBuffers() throws Exception { + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertEquals(1, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(2, subpartition.getBuffersInBacklogUnsafe()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(2); subpartition.add(createFilledFinishedBufferConsumer(1024)); - assertEquals(1, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); assertNextBuffer(readView, 1024, false, 0, false, true); } - @Test - public void testEmptyFlush() { + @TestTemplate + void testEmptyFlush() { subpartition.flush(); - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); } - @Test - public void testBasicPipelinedProduceConsumeLogic() throws Exception { + @TestTemplate + void testBasicPipelinedProduceConsumeLogic() throws Exception { // Empty => should return null - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); assertNoNextBuffer(readView); - assertFalse( - readView.getAvailabilityAndBacklog(0).isAvailable()); // also after getNextBuffer() - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()) + .isFalse(); // also after getNextBuffer() + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); // Add data to the queue... subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); - assertEquals(1, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); - assertEquals( - 0, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(0); // only updated when getting the buffer - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); // ...and one available result assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); - assertEquals( - BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(BUFFER_SIZE); // only updated when getting the buffer + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); assertNoNextBuffer(readView); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); // Add data to the queue... subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); - assertEquals(2, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); - assertEquals( - BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, availablityListener.getNumNotifications()); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(2); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(BUFFER_SIZE); // only updated when getting the buffer + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); - assertEquals( - 2 * BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(2 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); assertNoNextBuffer(readView); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); // some tests with events // fill with: buffer, event, and buffer subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); subpartition.add(createEventBufferConsumer(BUFFER_SIZE, Buffer.DataType.EVENT_BUFFER)); - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); - assertFalse(readView.getAvailabilityAndBacklog(0).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); - assertEquals(5, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals( - 1, subpartition.getBuffersInBacklogUnsafe()); // two buffers (events don't count) - assertEquals( - 2 * BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(1, availablityListener.getNumNotifications()); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(5); + assertThat(subpartition.getBuffersInBacklogUnsafe()) + .isEqualTo(1); // two buffers (events don't count) + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(2 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); // the first buffer assertNextBuffer(readView, BUFFER_SIZE, true, 0, true, true); - assertEquals( - 3 * BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(3 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); // the event assertNextEvent(readView, BUFFER_SIZE, null, false, 0, false, true); - assertEquals( - 4 * BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(4 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); // the remaining buffer assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); - assertEquals( - 5 * BUFFER_SIZE, - subpartition.getTotalNumberOfBytesUnsafe()); // only updated when getting the buffer - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(5 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); // nothing more assertNoNextBuffer(readView); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); - assertEquals(5, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals(5 * BUFFER_SIZE, subpartition.getTotalNumberOfBytesUnsafe()); - assertEquals(1, availablityListener.getNumNotifications()); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(5); + assertThat(subpartition.getTotalNumberOfBytesUnsafe()) + .isEqualTo(5 * BUFFER_SIZE); // only updated when getting the buffer + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); } - @Test - public void testBarrierOvertaking() throws Exception { + @TestTemplate + void testBarrierOvertaking() throws Exception { final RecordingChannelStateWriter channelStateWriter = new RecordingChannelStateWriter(); subpartition.setChannelStateWriter(channelStateWriter); subpartition.add(createFilledFinishedBufferConsumer(1)); - assertEquals(0, availablityListener.getNumNotifications()); - assertEquals(0, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); subpartition.add(createFilledFinishedBufferConsumer(2)); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(0, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); BufferConsumer eventBuffer = EventSerializer.toBufferConsumer(EndOfSuperstepEvent.INSTANCE, false); subpartition.add(eventBuffer); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(0, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); subpartition.add(createFilledFinishedBufferConsumer(4)); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(0, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); CheckpointOptions options = CheckpointOptions.unaligned( @@ -371,14 +359,13 @@ public void testBarrierOvertaking() throws Exception { BufferConsumer barrierBuffer = EventSerializer.toBufferConsumer(new CheckpointBarrier(0, 0, options), true); subpartition.add(barrierBuffer); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(1, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); final List inflight = channelStateWriter.getAddedOutput().get(subpartition.getSubpartitionInfo()); - assertEquals( - Arrays.asList(1, 2, 4), - inflight.stream().map(Buffer::getSize).collect(Collectors.toList())); + assertThat(inflight.stream().map(Buffer::getSize).collect(Collectors.toList())) + .containsExactly(1, 2, 4); inflight.forEach(Buffer::recycleBuffer); assertNextEvent( @@ -403,8 +390,8 @@ public void testBarrierOvertaking() throws Exception { assertNoNextBuffer(readView); } - @Test - public void testAvailabilityAfterPriority() throws Exception { + @TestTemplate + void testAvailabilityAfterPriority() throws Exception { subpartition.setChannelStateWriter(ChannelStateWriter.NO_OP); CheckpointOptions options = @@ -414,16 +401,16 @@ public void testAvailabilityAfterPriority() throws Exception { BufferConsumer barrierBuffer = EventSerializer.toBufferConsumer(new CheckpointBarrier(0, 0, options), true); subpartition.add(barrierBuffer); - assertEquals(1, availablityListener.getNumNotifications()); - assertEquals(1, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); subpartition.add(createFilledFinishedBufferConsumer(1)); - assertEquals(2, availablityListener.getNumNotifications()); - assertEquals(1, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(2); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); subpartition.add(createFilledFinishedBufferConsumer(2)); - assertEquals(2, availablityListener.getNumNotifications()); - assertEquals(1, availablityListener.getNumPriorityEvents()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(2); + assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); assertNextEvent( readView, @@ -438,18 +425,18 @@ public void testAvailabilityAfterPriority() throws Exception { assertNoNextBuffer(readView); } - @Test - public void testBacklogConsistentWithNumberOfConsumableBuffers() throws Exception { + @TestTemplate + void testBacklogConsistentWithNumberOfConsumableBuffers() throws Exception { testBacklogConsistentWithNumberOfConsumableBuffers(false, false); } - @Test - public void testBacklogConsistentWithConsumableBuffersForFlushedPartition() throws Exception { + @TestTemplate + void testBacklogConsistentWithConsumableBuffersForFlushedPartition() throws Exception { testBacklogConsistentWithNumberOfConsumableBuffers(true, false); } - @Test - public void testBacklogConsistentWithConsumableBuffersForFinishedPartition() throws Exception { + @TestTemplate + void testBacklogConsistentWithConsumableBuffersForFinishedPartition() throws Exception { testBacklogConsistentWithNumberOfConsumableBuffers(false, true); } @@ -479,7 +466,7 @@ private void testBacklogConsistentWithNumberOfConsumableBuffers( try (final CloseableRegistry closeableRegistry = new CloseableRegistry()) { while (readView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()) { ResultSubpartition.BufferAndBacklog bufferAndBacklog = readView.getNextBuffer(); - assertNotNull(bufferAndBacklog); + assertThat(bufferAndBacklog).isNotNull(); if (bufferAndBacklog.buffer().isBuffer()) { ++numberOfConsumableBuffers; @@ -488,12 +475,12 @@ private void testBacklogConsistentWithNumberOfConsumableBuffers( closeableRegistry.registerCloseable(bufferAndBacklog.buffer()::recycleBuffer); } - assertThat(backlog, is(numberOfConsumableBuffers)); + assertThat(backlog).isEqualTo(numberOfConsumableBuffers); } } - @Test - public void testResumeBlockedSubpartitionWithEvents() throws IOException, InterruptedException { + @TestTemplate + void testResumeBlockedSubpartitionWithEvents() throws IOException, InterruptedException { blockSubpartitionByCheckpoint(1); // add an event after subpartition blocked @@ -506,8 +493,8 @@ public void testResumeBlockedSubpartitionWithEvents() throws IOException, Interr assertNextEvent(readView, BUFFER_SIZE, null, false, 0, false, true); } - @Test - public void testResumeBlockedSubpartitionWithUnfinishedBufferFlushed() + @TestTemplate + void testResumeBlockedSubpartitionWithUnfinishedBufferFlushed() throws IOException, InterruptedException { blockSubpartitionByCheckpoint(1); @@ -522,8 +509,8 @@ public void testResumeBlockedSubpartitionWithUnfinishedBufferFlushed() assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); } - @Test - public void testResumeBlockedSubpartitionWithUnfinishedBufferNotFlushed() + @TestTemplate + void testResumeBlockedSubpartitionWithUnfinishedBufferNotFlushed() throws IOException, InterruptedException { blockSubpartitionByCheckpoint(1); @@ -536,8 +523,8 @@ public void testResumeBlockedSubpartitionWithUnfinishedBufferNotFlushed() resumeConsumptionAndCheckAvailability(Integer.MAX_VALUE, false); } - @Test - public void testResumeBlockedSubpartitionWithFinishedBuffers() + @TestTemplate + void testResumeBlockedSubpartitionWithFinishedBuffers() throws IOException, InterruptedException { blockSubpartitionByCheckpoint(1); @@ -553,8 +540,8 @@ public void testResumeBlockedSubpartitionWithFinishedBuffers() assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); } - @Test - public void testResumeBlockedEmptySubpartition() throws IOException, InterruptedException { + @TestTemplate + void testResumeBlockedEmptySubpartition() throws IOException, InterruptedException { blockSubpartitionByCheckpoint(1); // Resumption will not make the subpartition available since it is empty. @@ -569,24 +556,24 @@ private void blockSubpartitionByCheckpoint(int numNotifications) subpartition.add( createEventBufferConsumer(BUFFER_SIZE, Buffer.DataType.ALIGNED_CHECKPOINT_BARRIER)); - assertEquals(numNotifications, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(numNotifications); assertNextEvent(readView, BUFFER_SIZE, null, false, 0, false, true); } private void checkNumNotificationsAndAvailability(int numNotifications) throws IOException, InterruptedException { - assertEquals(numNotifications, availablityListener.getNumNotifications()); + assertThat(availablityListener.getNumNotifications()).isEqualTo(numNotifications); // view not available and no buffer can be read - assertFalse(readView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()).isFalse(); assertNoNextBuffer(readView); } private void resumeConsumptionAndCheckAvailability(int availableCredit, boolean dataAvailable) { readView.resumeConsumption(); - assertEquals( - dataAvailable, readView.getAvailabilityAndBacklog(availableCredit).isAvailable()); + assertThat(readView.getAvailabilityAndBacklog(availableCredit).isAvailable()) + .isEqualTo(dataAvailable); } static void assertNextBuffer( @@ -641,46 +628,49 @@ private static void assertNextBufferOrEvent( checkArgument(expectedEventClass == null || !expectedIsBuffer); ResultSubpartition.BufferAndBacklog bufferAndBacklog = readView.getNextBuffer(); - assertNotNull(bufferAndBacklog); + assertThat(bufferAndBacklog).isNotNull(); try { - assertEquals( - "buffer size", - expectedReadableBufferSize, - bufferAndBacklog.buffer().readableBytes()); - assertEquals("buffer or event", expectedIsBuffer, bufferAndBacklog.buffer().isBuffer()); + assertThat(bufferAndBacklog.buffer().readableBytes()) + .as("buffer size") + .isEqualTo(expectedReadableBufferSize); + assertThat(bufferAndBacklog.buffer().isBuffer()) + .as("buffer or event") + .isEqualTo(expectedIsBuffer); if (expectedEventClass != null) { - Assert.assertThat( - EventSerializer.fromBuffer( - bufferAndBacklog.buffer(), ClassLoader.getSystemClassLoader()), - instanceOf(expectedEventClass)); + assertThat( + EventSerializer.fromBuffer( + bufferAndBacklog.buffer(), + ClassLoader.getSystemClassLoader())) + .isInstanceOf(expectedEventClass); } - assertEquals( - "data available", expectedIsDataAvailable, bufferAndBacklog.isDataAvailable()); - assertEquals( - "data available", - expectedIsDataAvailable, - readView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()); - assertEquals("backlog", expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog()); - assertEquals( - "event available", - expectedIsEventAvailable, - bufferAndBacklog.isEventAvailable()); - assertEquals( - "event available", - expectedIsEventAvailable, - readView.getAvailabilityAndBacklog(0).isAvailable()); - - assertFalse("not recycled", bufferAndBacklog.buffer().isRecycled()); + assertThat(bufferAndBacklog.isDataAvailable()) + .as("data available") + .isEqualTo(expectedIsDataAvailable); + assertThat(readView.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()) + .as("data available") + .isEqualTo(expectedIsDataAvailable); + assertThat(bufferAndBacklog.buffersInBacklog()) + .as("backlog") + .isEqualTo(expectedBuffersInBacklog); + assertThat(bufferAndBacklog.isEventAvailable()) + .as("event available") + .isEqualTo(expectedIsEventAvailable); + assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()) + .as("event available") + .isEqualTo(expectedIsEventAvailable); + + assertThat(bufferAndBacklog.buffer().isRecycled()).as("not recycled").isFalse(); } finally { bufferAndBacklog.buffer().recycleBuffer(); } - assertEquals( - "recycled", expectedRecycledAfterRecycle, bufferAndBacklog.buffer().isRecycled()); + assertThat(bufferAndBacklog.buffer().isRecycled()) + .as("recycled") + .isEqualTo(expectedRecycledAfterRecycle); } static void assertNoNextBuffer(ResultSubpartitionView readView) throws IOException, InterruptedException { - assertNull(readView.getNextBuffer()); + assertThat(readView.getNextBuffer()).isNull(); } void setup(ResultPartitionType resultPartitionType) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java index 0510e42287622..b6ba4fab4d3e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java @@ -17,13 +17,12 @@ package org.apache.flink.runtime.io.network.partition; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.NoSuchElementException; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** * Tests PrioritizedDeque. @@ -31,10 +30,10 @@ *

Note that some tests make use of the {@link java.lang.Integer.IntegerCache} for improved * readability. */ -public class PrioritizedDequeTest { +class PrioritizedDequeTest { @Test - public void testPrioritizeOnAdd() { + void testPrioritizeOnAdd() { final PrioritizedDeque deque = new PrioritizedDeque<>(); deque.add(0); @@ -43,13 +42,11 @@ public void testPrioritizeOnAdd() { deque.add(3); deque.add(3, true, true); - assertArrayEquals( - new Integer[] {3, 0, 1, 2}, - deque.asUnmodifiableCollection().toArray(new Integer[0])); + assertThat(deque.asUnmodifiableCollection()).containsExactly(3, 0, 1, 2); } @Test - public void testPrioritize() { + void testPrioritize() { final PrioritizedDeque deque = new PrioritizedDeque<>(); deque.add(0); @@ -58,13 +55,11 @@ public void testPrioritize() { deque.add(3); deque.prioritize(3); - assertArrayEquals( - new Integer[] {3, 0, 1, 2}, - deque.asUnmodifiableCollection().toArray(new Integer[0])); + assertThat(deque.asUnmodifiableCollection()).containsExactly(3, 0, 1, 2); } @Test - public void testGetAndRemove() { + void testGetAndRemove() { final PrioritizedDeque deque = new PrioritizedDeque<>(); deque.add(0); @@ -73,13 +68,10 @@ public void testGetAndRemove() { deque.add(1); deque.add(3); - assertEquals(1, deque.getAndRemove(v -> v == 1).intValue()); - assertArrayEquals( - new Integer[] {0, 2, 1, 3}, - deque.asUnmodifiableCollection().toArray(new Integer[0])); - assertEquals(1, deque.getAndRemove(v -> v == 1).intValue()); - assertArrayEquals( - new Integer[] {0, 2, 3}, deque.asUnmodifiableCollection().toArray(new Integer[0])); + assertThat(deque.getAndRemove(v -> v == 1).intValue()).isEqualTo(1); + assertThat(deque.asUnmodifiableCollection()).containsExactly(0, 2, 1, 3); + assertThat(deque.getAndRemove(v -> v == 1).intValue()).isEqualTo(1); + assertThat(deque.asUnmodifiableCollection()).containsExactly(0, 2, 3); try { int removed = deque.getAndRemove(v -> v == 1); fail( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java index ff721bb32e641..1ccc6d7ffbf95 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java @@ -21,26 +21,26 @@ import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.util.SerializedThrowable; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link ProducerFailedException}. */ -public class ProducerFailedExceptionTest { +class ProducerFailedExceptionTest { @Test - public void testInstanceOfCancelTaskException() throws Exception { - assertTrue(CancelTaskException.class.isAssignableFrom(ProducerFailedException.class)); + void testInstanceOfCancelTaskException() throws Exception { + assertThat(CancelTaskException.class.isAssignableFrom(ProducerFailedException.class)) + .isTrue(); } @Test - public void testCauseIsSerialized() throws Exception { + void testCauseIsSerialized() throws Exception { // Tests that the cause is stringified, because it might be an instance // of a user level Exception, which can not be deserialized by the // remote receiver's system class loader. ProducerFailedException e = new ProducerFailedException(new Exception()); - assertNotNull(e.getCause()); - assertTrue(e.getCause() instanceof SerializedThrowable); + assertThat(e.getCause()).isNotNull(); + assertThat(e.getCause()).isInstanceOf(SerializedThrowable.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java index 83246ac5b20d1..5ab44fe82b16a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java @@ -24,9 +24,8 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -38,15 +37,10 @@ import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.collection.IsEmptyCollection.empty; -import static org.hamcrest.core.Is.is; +import static org.assertj.core.api.Assertions.assertThat; /** Test for the {@link ResourceManagerPartitionTrackerImpl}. */ -public class ResourceManagerPartitionTrackerImplTest extends TestLogger { +class ResourceManagerPartitionTrackerImplTest { private static final ClusterPartitionReport EMPTY_PARTITION_REPORT = new ClusterPartitionReport(Collections.emptySet()); @@ -58,14 +52,14 @@ public class ResourceManagerPartitionTrackerImplTest extends TestLogger { private static final ResultPartitionID PARTITION_ID_2 = new ResultPartitionID(); @Test - public void testProcessEmptyClusterPartitionReport() { + void testProcessEmptyClusterPartitionReport() { TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); reportEmpty(tracker, TASK_EXECUTOR_ID_1); - assertThat(partitionReleaser.releaseCalls, empty()); - assertThat(tracker.areAllMapsEmpty(), is(true)); + assertThat(partitionReleaser.releaseCalls).isEmpty(); + assertThat(tracker.areAllMapsEmpty()).isTrue(); } /** @@ -73,7 +67,7 @@ public void testProcessEmptyClusterPartitionReport() { * call if a subset of its partitions is lost. */ @Test - public void testReportProcessingWithPartitionLossOnSameTaskExecutor() { + void testReportProcessingWithPartitionLossOnSameTaskExecutor() { TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); final ResourceManagerPartitionTracker tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); @@ -81,9 +75,8 @@ public void testReportProcessingWithPartitionLossOnSameTaskExecutor() { report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1, PARTITION_ID_2); report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_2); - assertThat( - partitionReleaser.releaseCalls, - contains(Tuple2.of(TASK_EXECUTOR_ID_1, Collections.singleton(DATA_SET_ID)))); + assertThat(partitionReleaser.releaseCalls) + .contains(Tuple2.of(TASK_EXECUTOR_ID_1, Collections.singleton(DATA_SET_ID))); } /** @@ -91,7 +84,7 @@ public void testReportProcessingWithPartitionLossOnSameTaskExecutor() { * partition of the data set is lost on another task executor. */ @Test - public void testReportProcessingWithPartitionLossOnOtherTaskExecutor() { + void testReportProcessingWithPartitionLossOnOtherTaskExecutor() { TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); final ResourceManagerPartitionTracker tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); @@ -101,17 +94,16 @@ public void testReportProcessingWithPartitionLossOnOtherTaskExecutor() { reportEmpty(tracker, TASK_EXECUTOR_ID_1); - assertThat( - partitionReleaser.releaseCalls, - contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + assertThat(partitionReleaser.releaseCalls) + .contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID))); } @Test - public void testListDataSetsBasics() { + void testListDataSetsBasics() { final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(new TestClusterPartitionReleaser()); - assertThat(tracker.listDataSets().size(), is(0)); + assertThat(tracker.listDataSets()).isEmpty(); report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 2, PARTITION_ID_1); checkListedDataSets(tracker, 1, 2); @@ -123,22 +115,22 @@ public void testListDataSetsBasics() { checkListedDataSets(tracker, 1, 2); reportEmpty(tracker, TASK_EXECUTOR_ID_2); - assertThat(tracker.listDataSets().size(), is(0)); + assertThat(tracker.listDataSets()).isEmpty(); - assertThat(tracker.areAllMapsEmpty(), is(true)); + assertThat(tracker.areAllMapsEmpty()).isTrue(); } private static void checkListedDataSets( ResourceManagerPartitionTracker tracker, int expectedRegistered, int expectedTotal) { final Map listing = tracker.listDataSets(); - assertThat(listing, hasKey(DATA_SET_ID)); + assertThat(listing.containsKey(DATA_SET_ID)).isTrue(); DataSetMetaInfo metaInfo = listing.get(DATA_SET_ID); - assertThat(metaInfo.getNumRegisteredPartitions().orElse(-1), is(expectedRegistered)); - assertThat(metaInfo.getNumTotalPartitions(), is(expectedTotal)); + assertThat(metaInfo.getNumRegisteredPartitions().orElse(-1)).isEqualTo(expectedRegistered); + assertThat(metaInfo.getNumTotalPartitions()).isEqualTo(expectedTotal); } @Test - public void testReleasePartition() { + void testReleasePartition() { TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); @@ -149,54 +141,52 @@ public void testReleasePartition() { final CompletableFuture partitionReleaseFuture = tracker.releaseClusterPartitions(DATA_SET_ID); - assertThat( - partitionReleaser.releaseCalls, - containsInAnyOrder( + assertThat(partitionReleaser.releaseCalls) + .containsExactlyInAnyOrder( Tuple2.of(TASK_EXECUTOR_ID_1, Collections.singleton(DATA_SET_ID)), - Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID))); // the data set should still be tracked, since the partition release was not confirmed yet // by the task executors - assertThat(tracker.listDataSets().keySet(), contains(DATA_SET_ID)); + assertThat(tracker.listDataSets().keySet()).contains(DATA_SET_ID); // ack the partition release reportEmpty(tracker, TASK_EXECUTOR_ID_1, TASK_EXECUTOR_ID_2); - assertThat(partitionReleaseFuture.isDone(), is(true)); - assertThat(tracker.areAllMapsEmpty(), is(true)); + assertThat(partitionReleaseFuture.isDone()).isTrue(); + assertThat(tracker.areAllMapsEmpty()).isTrue(); } @Test - public void testShutdownProcessing() { + void testShutdownProcessing() { TestClusterPartitionReleaser partitionReleaser = new TestClusterPartitionReleaser(); final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(partitionReleaser); tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_1); - assertThat(partitionReleaser.releaseCalls, empty()); + assertThat(partitionReleaser.releaseCalls).isEmpty(); report(tracker, TASK_EXECUTOR_ID_1, DATA_SET_ID, 3, PARTITION_ID_1, PARTITION_ID_2); report(tracker, TASK_EXECUTOR_ID_2, DATA_SET_ID, 3, new ResultPartitionID()); tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_1); - assertThat( - partitionReleaser.releaseCalls, - contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID)))); + assertThat(partitionReleaser.releaseCalls) + .contains(Tuple2.of(TASK_EXECUTOR_ID_2, Collections.singleton(DATA_SET_ID))); - assertThat(tracker.areAllMapsEmpty(), is(false)); + assertThat(tracker.areAllMapsEmpty()).isFalse(); tracker.processTaskExecutorShutdown(TASK_EXECUTOR_ID_2); - assertThat(tracker.areAllMapsEmpty(), is(true)); + assertThat(tracker.areAllMapsEmpty()).isTrue(); } @Test - public void testGetClusterPartitionShuffleDescriptors() { + void testGetClusterPartitionShuffleDescriptors() { final ResourceManagerPartitionTrackerImpl tracker = new ResourceManagerPartitionTrackerImpl(new TestClusterPartitionReleaser()); - assertThat(tracker.listDataSets().size(), is(0)); + assertThat(tracker.listDataSets()).isEmpty(); List resultPartitionIDS = new ArrayList<>(); for (int i = 0; i < 100; i++) { @@ -212,16 +202,16 @@ public void testGetClusterPartitionShuffleDescriptors() { final List shuffleDescriptors = tracker.getClusterPartitionShuffleDescriptors(DATA_SET_ID); - assertThat(shuffleDescriptors.size(), is(100)); + assertThat(shuffleDescriptors).hasSize(100); assertThat( - shuffleDescriptors.stream() - .map(ShuffleDescriptor::getResultPartitionID) - .collect(Collectors.toList()), - contains(resultPartitionIDS.toArray())); + shuffleDescriptors.stream() + .map(ShuffleDescriptor::getResultPartitionID) + .collect(Collectors.toList())) + .containsExactlyElementsOf(resultPartitionIDS); reportEmpty(tracker, TASK_EXECUTOR_ID_1); reportEmpty(tracker, TASK_EXECUTOR_ID_2); - assertThat(tracker.areAllMapsEmpty(), is(true)); + assertThat(tracker.areAllMapsEmpty()).isTrue(); } private static void reportEmpty( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java index 8e3b2263fbee6..06de23dc50bfa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java @@ -31,12 +31,10 @@ import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Optional; import java.util.concurrent.Executors; @@ -45,7 +43,6 @@ /** Tests for the {@link ResultPartitionFactory}. */ @SuppressWarnings("StaticVariableUsedBeforeInitialization") -@ExtendWith(TestLoggerExtension.class) class ResultPartitionFactoryTest { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java index b9e1dc4a6ae2b..6494101af9707 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManagerTest.java @@ -18,15 +18,13 @@ package org.apache.flink.runtime.io.network.partition; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.verifyCreateSubpartitionViewThrowsException; /** Tests for {@link ResultPartitionManager}. */ -public class ResultPartitionManagerTest extends TestLogger { +class ResultPartitionManagerTest { /** * Tests that {@link ResultPartitionManager#createSubpartitionView(ResultPartitionID, int, @@ -34,7 +32,7 @@ public class ResultPartitionManagerTest extends TestLogger { * was not registered before. */ @Test - public void testThrowPartitionNotFoundException() throws Exception { + void testThrowPartitionNotFoundException() throws Exception { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); @@ -46,7 +44,7 @@ public void testThrowPartitionNotFoundException() throws Exception { * BufferAvailabilityListener)} successful if this partition was already registered before. */ @Test - public void testCreateViewForRegisteredPartition() throws Exception { + void testCreateViewForRegisteredPartition() throws Exception { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); @@ -61,7 +59,7 @@ public void testCreateViewForRegisteredPartition() throws Exception { * partition was already released before. */ @Test - public void testCreateViewForReleasedPartition() throws Exception { + void testCreateViewForReleasedPartition() throws Exception { final ResultPartitionManager partitionManager = new ResultPartitionManager(); final ResultPartition partition = createPartition(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index e44b3732f7754..f6bb1b58691f5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -20,18 +20,15 @@ import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** Basic subpartition behaviour tests. */ -public abstract class SubpartitionTestBase extends TestLogger { +abstract class SubpartitionTestBase { /** Return the subpartition to be tested. */ abstract ResultSubpartition createSubpartition() throws Exception; @@ -41,8 +38,8 @@ public abstract class SubpartitionTestBase extends TestLogger { // ------------------------------------------------------------------------ - @Test - public void createReaderAfterDispose() throws Exception { + @TestTemplate + void createReaderAfterDispose() throws Exception { final ResultSubpartition subpartition = createSubpartition(); subpartition.release(); @@ -54,22 +51,22 @@ public void createReaderAfterDispose() throws Exception { } } - @Test - public void testAddAfterFinish() throws Exception { + @TestTemplate + void testAddAfterFinish() throws Exception { final ResultSubpartition subpartition = createSubpartition(); try { subpartition.finish(); - assertEquals(1, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(4096); - assertEquals(-1, subpartition.add(bufferConsumer)); - assertTrue(bufferConsumer.isRecycled()); + assertThat(subpartition.add(bufferConsumer)).isEqualTo(-1); + assertThat(bufferConsumer.isRecycled()).isTrue(); - assertEquals(1, subpartition.getTotalNumberOfBuffersUnsafe()); - assertEquals(0, subpartition.getBuffersInBacklogUnsafe()); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); } finally { if (subpartition != null) { subpartition.release(); @@ -77,8 +74,8 @@ public void testAddAfterFinish() throws Exception { } } - @Test - public void testAddAfterRelease() throws Exception { + @TestTemplate + void testAddAfterRelease() throws Exception { final ResultSubpartition subpartition = createSubpartition(); try { @@ -86,8 +83,8 @@ public void testAddAfterRelease() throws Exception { BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(4096); - assertEquals(-1, subpartition.add(bufferConsumer)); - assertTrue(bufferConsumer.isRecycled()); + assertThat(subpartition.add(bufferConsumer)).isEqualTo(-1); + assertThat(bufferConsumer.isRecycled()).isTrue(); } finally { if (subpartition != null) { @@ -96,8 +93,8 @@ public void testAddAfterRelease() throws Exception { } } - @Test - public void testReleasingReaderDoesNotReleasePartition() throws Exception { + @TestTemplate + void testReleasingReaderDoesNotReleasePartition() throws Exception { final ResultSubpartition partition = createSubpartition(); partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); @@ -105,19 +102,19 @@ public void testReleasingReaderDoesNotReleasePartition() throws Exception { final ResultSubpartitionView reader = partition.createReadView(new NoOpBufferAvailablityListener()); - assertFalse(partition.isReleased()); - assertFalse(reader.isReleased()); + assertThat(partition.isReleased()).isFalse(); + assertThat(reader.isReleased()).isFalse(); reader.releaseAllResources(); - assertTrue(reader.isReleased()); - assertFalse(partition.isReleased()); + assertThat(reader.isReleased()).isTrue(); + assertThat(partition.isReleased()).isFalse(); partition.release(); } - @Test - public void testReleaseIsIdempotent() throws Exception { + @TestTemplate + void testReleaseIsIdempotent() throws Exception { final ResultSubpartition partition = createSubpartition(); partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); @@ -127,8 +124,8 @@ public void testReleaseIsIdempotent() throws Exception { partition.release(); } - @Test - public void testReadAfterDispose() throws Exception { + @TestTemplate + void testReadAfterDispose() throws Exception { final ResultSubpartition partition = createSubpartition(); partition.add(createFilledFinishedBufferConsumer(BufferBuilderTestUtils.BUFFER_SIZE)); partition.finish(); @@ -145,8 +142,8 @@ public void testReadAfterDispose() throws Exception { // assertNull(reader.getNextBuffer()); } - @Test - public void testRecycleBufferAndConsumerOnFailure() throws Exception { + @TestTemplate + void testRecycleBufferAndConsumerOnFailure() throws Exception { final ResultSubpartition subpartition = createFailingWritesSubpartition(); try { final BufferConsumer consumer = @@ -160,7 +157,7 @@ public void testRecycleBufferAndConsumerOnFailure() throws Exception { // expected } - assertTrue(consumer.isRecycled()); + assertThat(consumer.isRecycled()).isTrue(); } finally { subpartition.release(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java index a2a03689bcef3..2c25f3cad96f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java @@ -31,11 +31,10 @@ import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; import org.apache.flink.runtime.taskexecutor.partition.ClusterPartitionReport; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava31.com.google.common.collect.Iterables; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Collection; @@ -44,22 +43,17 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.hasItem; -import static org.hamcrest.Matchers.hasItems; -import static org.hamcrest.Matchers.is; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link TaskExecutorPartitionTrackerImpl}. */ -public class TaskExecutorPartitionTrackerImplTest extends TestLogger { +class TaskExecutorPartitionTrackerImplTest { @Test - public void createClusterPartitionReport() { + void createClusterPartitionReport() { final TaskExecutorPartitionTrackerImpl partitionTracker = new TaskExecutorPartitionTrackerImpl(new NettyShuffleEnvironmentBuilder().build()); - assertThat(partitionTracker.createClusterPartitionReport().getEntries(), is(empty())); + assertThat(partitionTracker.createClusterPartitionReport().getEntries()).isEmpty(); final IntermediateDataSetID dataSetId = new IntermediateDataSetID(); final JobID jobId = new JobID(); @@ -87,13 +81,13 @@ public void createClusterPartitionReport() { final ClusterPartitionReport.ClusterPartitionReportEntry reportEntry = Iterables.getOnlyElement(clusterPartitionReport.getEntries()); - assertThat(reportEntry.getDataSetId(), is(dataSetId)); - assertThat(reportEntry.getNumTotalPartitions(), is(numberOfPartitions)); - assertThat(reportEntry.getHostedPartitions(), hasItems(clusterPartitionId)); + assertThat(reportEntry.getDataSetId()).isEqualTo(dataSetId); + assertThat(reportEntry.getNumTotalPartitions()).isEqualTo(numberOfPartitions); + assertThat(reportEntry.getHostedPartitions()).contains(clusterPartitionId); } @Test - public void testStopTrackingAndReleaseJobPartitions() throws Exception { + void testStopTrackingAndReleaseJobPartitions() throws Exception { final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); @@ -119,11 +113,11 @@ public void testStopTrackingAndReleaseJobPartitions() throws Exception { partitionTracker.stopTrackingAndReleaseJobPartitions( Collections.singleton(resultPartitionId1)); - assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); + assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); } @Test - public void testStopTrackingAndReleaseJobPartitionsFor() throws Exception { + void testStopTrackingAndReleaseJobPartitionsFor() throws Exception { final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); @@ -150,11 +144,11 @@ public void testStopTrackingAndReleaseJobPartitionsFor() throws Exception { 1)); partitionTracker.stopTrackingAndReleaseJobPartitionsFor(jobId1); - assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); + assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); } @Test - public void promoteJobPartitions() throws Exception { + void promoteJobPartitions() throws Exception { final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); @@ -181,11 +175,11 @@ public void promoteJobPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseJobPartitionsFor(jobId); - assertThat(shuffleReleaseFuture.get(), not(hasItem(resultPartitionId1))); + assertThat(shuffleReleaseFuture.get()).doesNotContain(resultPartitionId1); } @Test - public void stopTrackingAndReleaseAllClusterPartitions() throws Exception { + void stopTrackingAndReleaseAllClusterPartitions() throws Exception { final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); @@ -211,11 +205,11 @@ public void stopTrackingAndReleaseAllClusterPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseAllClusterPartitions(); - assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); + assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); } @Test - public void stopTrackingAndReleaseClusterPartitions() throws Exception { + void stopTrackingAndReleaseClusterPartitions() throws Exception { final TestingShuffleEnvironment testingShuffleEnvironment = new TestingShuffleEnvironment(); final CompletableFuture> shuffleReleaseFuture = new CompletableFuture<>(); @@ -240,7 +234,7 @@ public void stopTrackingAndReleaseClusterPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseClusterPartitions(Collections.singleton(dataSetId1)); - assertThat(shuffleReleaseFuture.get(), hasItem(resultPartitionId1)); + assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); } private static class TestingShuffleEnvironment diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java index bdbc57ab9acdd..109a542dfa651 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Arrays; @@ -35,15 +35,14 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSomeBuffer; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** {@link ChannelStatePersister} test. */ -public class ChannelStatePersisterTest { +class ChannelStatePersisterTest { @Test - public void testNewBarrierNotOverwrittenByStopPersisting() throws Exception { + void testNewBarrierNotOverwrittenByStopPersisting() throws Exception { RecordingChannelStateWriter channelStateWriter = new RecordingChannelStateWriter(); InputChannelInfo channelInfo = new InputChannelInfo(0, 0); ChannelStatePersister persister = @@ -55,10 +54,10 @@ public void testNewBarrierNotOverwrittenByStopPersisting() throws Exception { persister.checkForBarrier(barrier(checkpointId)); persister.startPersisting(checkpointId, Arrays.asList(buildSomeBuffer())); - assertEquals(1, channelStateWriter.getAddedInput().get(channelInfo).size()); + assertThat(channelStateWriter.getAddedInput().get(channelInfo)).hasSize(1); persister.maybePersist(buildSomeBuffer()); - assertEquals(1, channelStateWriter.getAddedInput().get(channelInfo).size()); + assertThat(channelStateWriter.getAddedInput().get(channelInfo)).hasSize(1); // meanwhile, checkpoint coordinator timed out the 1st checkpoint and started the 2nd // now task thread is picking up the barrier and aborts the 1st: @@ -66,36 +65,36 @@ public void testNewBarrierNotOverwrittenByStopPersisting() throws Exception { persister.maybePersist(buildSomeBuffer()); persister.stopPersisting(checkpointId); persister.maybePersist(buildSomeBuffer()); - assertEquals(1, channelStateWriter.getAddedInput().get(channelInfo).size()); + assertThat(channelStateWriter.getAddedInput().get(channelInfo)).hasSize(1); - assertTrue(persister.hasBarrierReceived()); + assertThat(persister.hasBarrierReceived()).isTrue(); } @Test - public void testNewBarrierNotOverwrittenByCheckForBarrier() throws Exception { + void testNewBarrierNotOverwrittenByCheckForBarrier() throws Exception { ChannelStatePersister persister = new ChannelStatePersister(ChannelStateWriter.NO_OP, new InputChannelInfo(0, 0)); persister.startPersisting(1L, Collections.emptyList()); persister.startPersisting(2L, Collections.emptyList()); - assertFalse(persister.checkForBarrier(barrier(1L)).isPresent()); + assertThat(persister.checkForBarrier(barrier(1L)).isPresent()).isFalse(); - assertFalse(persister.hasBarrierReceived()); + assertThat(persister.hasBarrierReceived()).isFalse(); } @Test - public void testLateBarrierOnStartedAndCancelledCheckpoint() throws Exception { + void testLateBarrierOnStartedAndCancelledCheckpoint() throws Exception { testLateBarrier(true, true); } @Test - public void testLateBarrierOnCancelledCheckpoint() throws Exception { + void testLateBarrierOnCancelledCheckpoint() throws Exception { testLateBarrier(false, true); } @Test - public void testLateBarrierOnNotYetCancelledCheckpoint() throws Exception { + void testLateBarrierOnNotYetCancelledCheckpoint() throws Exception { testLateBarrier(false, false); } @@ -124,12 +123,12 @@ private void testLateBarrier( persister.checkForBarrier(barrier(checkpointId)); persister.maybePersist(buildSomeBuffer()); - assertTrue(persister.hasBarrierReceived()); - assertEquals(2, channelStateWriter.getAddedInput().get(channelInfo).size()); + assertThat(persister.hasBarrierReceived()).isTrue(); + assertThat(channelStateWriter.getAddedInput().get(channelInfo)).hasSize(2); } - @Test(expected = CheckpointException.class) - public void testLateBarrierTriggeringCheckpoint() throws Exception { + @Test + void testLateBarrierTriggeringCheckpoint() throws Exception { ChannelStatePersister persister = new ChannelStatePersister(ChannelStateWriter.NO_OP, new InputChannelInfo(0, 0)); @@ -137,7 +136,9 @@ public void testLateBarrierTriggeringCheckpoint() throws Exception { long checkpointId = 2L; persister.checkForBarrier(barrier(checkpointId)); - persister.startPersisting(lateCheckpointId, Collections.emptyList()); + assertThatThrownBy( + () -> persister.startPersisting(lateCheckpointId, Collections.emptyList())) + .isInstanceOf(CheckpointException.class); } private static Buffer barrier(long id) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpecTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpecTest.java index 9891ad9b5fc0a..23280833d0b63 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpecTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/GateBuffersSpecTest.java @@ -22,8 +22,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.util.Optional; @@ -33,7 +31,6 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link GateBuffersSpec}. */ -@RunWith(Parameterized.class) class GateBuffersSpecTest { private static ResultPartitionType[] parameters() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java index 7d4b434995c0d..7f6b9d48940cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java @@ -30,35 +30,35 @@ import org.apache.flink.runtime.io.network.partition.PartitionTestUtils; import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.within; /** Tests the metrics for input buffers usage. */ -public class InputBuffersMetricsTest extends TestLogger { +class InputBuffersMetricsTest { private CloseableRegistry closeableRegistry; - @Before - public void setup() { + @BeforeEach + void setup() { closeableRegistry = new CloseableRegistry(); } - @After - public void tearDown() throws IOException { + @AfterEach + void tearDown() throws IOException { closeableRegistry.close(); } @Test - public void testCalculateTotalBuffersSize() throws Exception { + void testCalculateTotalBuffersSize() throws Exception { int numberOfRemoteChannels = 2; int numberOfLocalChannels = 0; @@ -89,19 +89,17 @@ public void testCalculateTotalBuffersSize() throws Exception { closeableRegistry.registerCloseable(network::close); closeableRegistry.registerCloseable(inputGate1::close); - assertEquals( - numberOfBuffersPerGate, - floatingBuffersUsageGauge.calculateTotalBuffers(inputGate1)); - assertEquals( - numberOfRemoteChannels * numberOfBufferPerChannel, - exclusiveBuffersUsageGauge.calculateTotalBuffers(inputGate1)); - assertEquals( - numberOfRemoteChannels * numberOfBufferPerChannel + numberOfBuffersPerGate, - inputBufferPoolUsageGauge.calculateTotalBuffers(inputGate1)); + assertThat(floatingBuffersUsageGauge.calculateTotalBuffers(inputGate1)) + .isEqualTo(numberOfBuffersPerGate); + assertThat(exclusiveBuffersUsageGauge.calculateTotalBuffers(inputGate1)) + .isEqualTo(numberOfRemoteChannels * numberOfBufferPerChannel); + assertThat(inputBufferPoolUsageGauge.calculateTotalBuffers(inputGate1)) + .isEqualTo( + numberOfRemoteChannels * numberOfBufferPerChannel + numberOfBuffersPerGate); } @Test - public void testExclusiveBuffersUsage() throws Exception { + void testExclusiveBuffersUsage() throws Exception { int numberOfRemoteChannelsGate1 = 2; int numberOfLocalChannelsGate1 = 0; int numberOfRemoteChannelsGate2 = 1; @@ -142,8 +140,8 @@ public void testExclusiveBuffersUsage() throws Exception { new CreditBasedInputBuffersUsageGauge( floatingBuffersUsageGauge, exclusiveBuffersUsageGauge, inputGates); - assertEquals(0.0, exclusiveBuffersUsageGauge.getValue(), 0.0); - assertEquals(0.0, inputBuffersUsageGauge.getValue(), 0.0); + assertThat(exclusiveBuffersUsageGauge.getValue()).isEqualTo(0.0f, within(0.0f)); + assertThat(inputBuffersUsageGauge.getValue()).isEqualTo(0.0f, within(0.0f)); int totalBuffers = extraNetworkBuffersPerGate * inputGates.length @@ -164,7 +162,7 @@ public void testExclusiveBuffersUsage() throws Exception { } @Test - public void testFloatingBuffersUsage() throws Exception { + void testFloatingBuffersUsage() throws Exception { int numberOfRemoteChannelsGate1 = 2; int numberOfLocalChannelsGate1 = 0; @@ -205,8 +203,8 @@ public void testFloatingBuffersUsage() throws Exception { new CreditBasedInputBuffersUsageGauge( floatingBuffersUsageGauge, exclusiveBuffersUsageGauge, inputGates); - assertEquals(0.0, floatingBuffersUsageGauge.getValue(), 0.0); - assertEquals(0.0, inputBuffersUsageGauge.getValue(), 0.0); + assertThat(floatingBuffersUsageGauge.getValue()).isEqualTo(0.0f, within(0.0f)); + assertThat(inputBuffersUsageGauge.getValue()).isEqualTo(0.0f, within(0.0f)); // drain gate1's exclusive buffers drainBuffer(buffersPerChannel, remoteInputChannel1); @@ -222,17 +220,16 @@ public void testFloatingBuffersUsage() throws Exception { remoteInputChannel1.onSenderBacklog(backlog); - assertEquals( - totalRequestedBuffers, - remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()); + assertThat(remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()) + .isEqualTo(totalRequestedBuffers); drainBuffer(totalRequestedBuffers, remoteInputChannel1); - assertEquals(0, remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()); - assertEquals( - (double) (buffersPerChannel + totalRequestedBuffers) / totalBuffers, - inputBuffersUsageGauge.getValue(), - 0.0001); + assertThat(remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()).isEqualTo(0); + assertThat((double) inputBuffersUsageGauge.getValue()) + .isEqualTo( + (double) (buffersPerChannel + totalRequestedBuffers) / totalBuffers, + within(0.0001)); } private void drainAndValidate( @@ -247,16 +244,13 @@ private void drainAndValidate( throws IOException { drainBuffer(numBuffersToRequest, channel); - assertEquals( - totalRequestedBuffers, exclusiveBuffersUsageGauge.calculateUsedBuffers(inputGate)); - assertEquals( - (double) totalRequestedBuffers / totalExclusiveBuffers, - exclusiveBuffersUsageGauge.getValue(), - 0.0001); - assertEquals( - (double) totalRequestedBuffers / totalBuffers, - inputBuffersUsageGauge.getValue(), - 0.0001); + assertThat(exclusiveBuffersUsageGauge.calculateUsedBuffers(inputGate)) + .isEqualTo(totalRequestedBuffers); + assertThat((double) exclusiveBuffersUsageGauge.getValue()) + .isEqualTo((double) totalRequestedBuffers / totalExclusiveBuffers, within(0.0001)); + + assertThat((double) inputBuffersUsageGauge.getValue()) + .isEqualTo((double) totalRequestedBuffers / totalBuffers, within(0.0001)); } private void drainBuffer(int boundary, RemoteInputChannel channel) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java index 16801a19879ef..b5a2ad893567a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java @@ -22,14 +22,12 @@ import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link InputChannelID}. */ -public class InputChannelIDTest { +class InputChannelIDTest { private static final NettyBufferPool ALLOCATOR = new NettyBufferPool(1); @Test @@ -39,7 +37,7 @@ public void testByteBufWriteAndRead() { final ByteBuf byteBuf = ALLOCATOR.directBuffer(byteBufLen, byteBufLen); inputChannelID.writeTo(byteBuf); - assertThat(byteBuf.writerIndex(), is(equalTo(InputChannelID.getByteBufLength()))); - assertThat(InputChannelID.fromByteBuf(byteBuf), is(equalTo(inputChannelID))); + assertThat(byteBuf.writerIndex()).isEqualTo(InputChannelID.getByteBufLength()); + assertThat(InputChannelID.fromByteBuf(byteBuf)).isEqualTo(inputChannelID); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index 73d8860a4d529..6972209263010 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -22,84 +22,82 @@ import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Optional; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.mock; /** Tests for {@link InputChannel}. */ -public class InputChannelTest { +class InputChannelTest { @Test - public void testExponentialBackoff() throws Exception { + void testExponentialBackoff() throws Exception { InputChannel ch = createInputChannel(500, 4000); - assertEquals(0, ch.getCurrentBackoff()); + assertThat(ch.getCurrentBackoff()).isEqualTo(0); - assertTrue(ch.increaseBackoff()); - assertEquals(500, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(500); - assertTrue(ch.increaseBackoff()); - assertEquals(1000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(1000); - assertTrue(ch.increaseBackoff()); - assertEquals(2000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(2000); - assertTrue(ch.increaseBackoff()); - assertEquals(4000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(4000); - assertFalse(ch.increaseBackoff()); - assertEquals(4000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isFalse(); + assertThat(ch.getCurrentBackoff()).isEqualTo(4000); } @Test - public void testExponentialBackoffCappedAtMax() throws Exception { + void testExponentialBackoffCappedAtMax() throws Exception { InputChannel ch = createInputChannel(500, 3000); - assertEquals(0, ch.getCurrentBackoff()); + assertThat(ch.getCurrentBackoff()).isEqualTo(0); - assertTrue(ch.increaseBackoff()); - assertEquals(500, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(500); - assertTrue(ch.increaseBackoff()); - assertEquals(1000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(1000); - assertTrue(ch.increaseBackoff()); - assertEquals(2000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(2000); - assertTrue(ch.increaseBackoff()); - assertEquals(3000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(3000); - assertFalse(ch.increaseBackoff()); - assertEquals(3000, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isFalse(); + assertThat(ch.getCurrentBackoff()).isEqualTo(3000); } @Test - public void testExponentialBackoffSingle() throws Exception { + void testExponentialBackoffSingle() throws Exception { InputChannel ch = createInputChannel(500, 500); - assertEquals(0, ch.getCurrentBackoff()); + assertThat(ch.getCurrentBackoff()).isEqualTo(0); - assertTrue(ch.increaseBackoff()); - assertEquals(500, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isTrue(); + assertThat(ch.getCurrentBackoff()).isEqualTo(500); - assertFalse(ch.increaseBackoff()); - assertEquals(500, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isFalse(); + assertThat(ch.getCurrentBackoff()).isEqualTo(500); } @Test - public void testExponentialNoBackoff() throws Exception { + void testExponentialNoBackoff() throws Exception { InputChannel ch = createInputChannel(0, 0); - assertEquals(0, ch.getCurrentBackoff()); + assertThat(ch.getCurrentBackoff()).isEqualTo(0); - assertFalse(ch.increaseBackoff()); - assertEquals(0, ch.getCurrentBackoff()); + assertThat(ch.increaseBackoff()).isFalse(); + assertThat(ch.getCurrentBackoff()).isEqualTo(0); } private InputChannel createInputChannel(int initialBackoff, int maxBackoff) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java index e7859cc1df367..9415d0b756785 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java @@ -22,21 +22,19 @@ import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import java.util.concurrent.CompletableFuture; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Test base for {@link InputGate}. */ -public abstract class InputGateTestBase { +abstract class InputGateTestBase { int gateIndex; - @Before - public void resetGateIndex() { + @BeforeEach + void resetGateIndex() { gateIndex = 0; } @@ -46,38 +44,38 @@ protected void testIsAvailable( TestInputChannel inputChannelWithNewData) throws Exception { - assertFalse(inputGateToTest.getAvailableFuture().isDone()); - assertFalse(inputGateToTest.pollNext().isPresent()); + assertThat(inputGateToTest.getAvailableFuture().isDone()).isFalse(); + assertThat(inputGateToTest.pollNext()).isEmpty(); CompletableFuture future = inputGateToTest.getAvailableFuture(); - assertFalse(inputGateToTest.getAvailableFuture().isDone()); - assertFalse(inputGateToTest.pollNext().isPresent()); + assertThat(inputGateToTest.getAvailableFuture().isDone()).isFalse(); + assertThat(inputGateToTest.pollNext()).isEmpty(); - assertEquals(future, inputGateToTest.getAvailableFuture()); + assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(future); inputChannelWithNewData.readBuffer(); inputGateToNotify.notifyChannelNonEmpty(inputChannelWithNewData); - assertTrue(future.isDone()); - assertTrue(inputGateToTest.getAvailableFuture().isDone()); - assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.getAvailableFuture()); + assertThat(future.isDone()).isTrue(); + assertThat(inputGateToTest.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(PullingAsyncDataInput.AVAILABLE); } protected void testIsAvailableAfterFinished( InputGate inputGateToTest, Runnable endOfPartitionEvent) throws Exception { CompletableFuture available = inputGateToTest.getAvailableFuture(); - assertFalse(available.isDone()); - assertFalse(inputGateToTest.pollNext().isPresent()); + assertThat(available.isDone()).isFalse(); + assertThat(inputGateToTest.pollNext()).isEmpty(); endOfPartitionEvent.run(); - assertTrue(inputGateToTest.pollNext().isPresent()); // EndOfPartitionEvent + assertThat(inputGateToTest.pollNext()).isNotEmpty(); // EndOfPartitionEvent - assertTrue(available.isDone()); - assertTrue(inputGateToTest.getAvailableFuture().isDone()); - assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.getAvailableFuture()); + assertThat(available.isDone()).isTrue(); + assertThat(inputGateToTest.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(PullingAsyncDataInput.AVAILABLE); } protected SingleInputGate createInputGate() { @@ -104,7 +102,7 @@ protected SingleInputGate createInputGate( } SingleInputGate inputGate = builder.build(); - assertEquals(partitionType, inputGate.getConsumedPartitionType()); + assertThat(inputGate.getConsumedPartitionType()).isEqualTo(partitionType); return inputGate; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 6940707cc9bab..0f04a4a3247b7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -54,8 +54,7 @@ import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; -import org.hamcrest.Matchers; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.stubbing.Answer; import java.io.IOException; @@ -77,13 +76,9 @@ import static org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.TestingResultPartitionManager; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -94,10 +89,10 @@ import static org.mockito.Mockito.when; /** Tests for the {@link LocalInputChannel}. */ -public class LocalInputChannelTest { +class LocalInputChannelTest { @Test - public void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { + void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { CheckpointBarrier barrier = new CheckpointBarrier( 1L, @@ -125,9 +120,9 @@ public void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { channel.checkpointStarted(barrier); // pull data channel.getNextBuffer(); - assertTrue( - "no data should be persisted after receiving a barrier", - stateWriter.getAddedInput().isEmpty()); + assertThat(stateWriter.getAddedInput().isEmpty()) + .withFailMessage("no data should be persisted after receiving a barrier") + .isTrue(); } /** @@ -137,7 +132,7 @@ public void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { * via local input channels. */ @Test - public void testConcurrentConsumeMultiplePartitions() throws Exception { + void testConcurrentConsumeMultiplePartitions() throws Exception { // Config final int parallelism = 32; final int producerBufferPoolSize = parallelism + 1; @@ -228,7 +223,7 @@ public void testConcurrentConsumeMultiplePartitions() throws Exception { } @Test - public void testPartitionRequestExponentialBackoff() throws Exception { + void testPartitionRequestExponentialBackoff() throws Exception { // Config int initialBackoff = 500; int maxBackoff = 3000; @@ -283,8 +278,8 @@ public void testPartitionRequestExponentialBackoff() throws Exception { } } - @Test(expected = CancelTaskException.class) - public void testProducerFailedException() throws Exception { + @Test + void testProducerFailedException() throws Exception { ResultSubpartitionView view = mock(ResultSubpartitionView.class); when(view.isReleased()).thenReturn(true); when(view.getFailureCause()).thenReturn(new Exception("Expected test exception")); @@ -305,7 +300,7 @@ public void testProducerFailedException() throws Exception { ch.requestSubpartition(); // Should throw an instance of CancelTaskException. - ch.getNextBuffer(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(CancelTaskException.class); } /** @@ -314,7 +309,7 @@ public void testProducerFailedException() throws Exception { * ResultPartitionManager} and no backoff. */ @Test - public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exception { + void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exception { final SingleInputGate inputGate = createSingleInputGate(1); final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); @@ -324,7 +319,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce fail("Should throw a PartitionNotFoundException."); } catch (PartitionNotFoundException notFound) { - assertThat(localChannel.getPartitionId(), Matchers.is(notFound.getPartitionId())); + assertThat(localChannel.getPartitionId()).isEqualTo(notFound.getPartitionId()); } } @@ -334,7 +329,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce * PartitionNotFoundException} within backoff. */ @Test - public void testRetriggerPartitionRequestWhilePartitionNotFound() throws Exception { + void testRetriggerPartitionRequestWhilePartitionNotFound() throws Exception { final SingleInputGate inputGate = createSingleInputGate(1); final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager(), 1, 1); @@ -343,7 +338,7 @@ public void testRetriggerPartitionRequestWhilePartitionNotFound() throws Excepti localChannel.requestSubpartition(); // The timer should be initialized at the first time of retriggering partition request. - assertNotNull(inputGate.getRetriggerLocalRequestTimer()); + assertThat(inputGate.getRetriggerLocalRequestTimer()).isNotNull(); } /** @@ -351,7 +346,7 @@ public void testRetriggerPartitionRequestWhilePartitionNotFound() throws Excepti * PartitionNotFoundException} which is set onto the input channel then. */ @Test - public void testChannelErrorWhileRetriggeringRequest() { + void testChannelErrorWhileRetriggeringRequest() { final SingleInputGate inputGate = createSingleInputGate(1); final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); @@ -367,9 +362,8 @@ public void schedule(TimerTask task, long delay) { fail("Should throw a PartitionNotFoundException."); } catch (PartitionNotFoundException notFound) { - assertThat( - localChannel.partitionId, - Matchers.is(notFound.getPartitionId())); + assertThat(localChannel.partitionId) + .isEqualTo(notFound.getPartitionId()); } catch (IOException ex) { fail("Should throw a PartitionNotFoundException."); } @@ -405,7 +399,7 @@ public void schedule(TimerTask task, long delay) { * any input gate callbacks while holding the channel's lock. I decided to do both. */ @Test - public void testConcurrentReleaseAndRetriggerPartitionRequest() throws Exception { + void testConcurrentReleaseAndRetriggerPartitionRequest() throws Exception { final SingleInputGate gate = createSingleInputGate(1); ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); @@ -455,7 +449,7 @@ public void testConcurrentReleaseAndRetriggerPartitionRequest() throws Exception * don't lead to NPEs. */ @Test - public void testGetNextAfterPartitionReleased() throws Exception { + void testGetNextAfterPartitionReleased() throws Exception { ResultSubpartitionView subpartitionView = InputChannelTestUtils.createResultSubpartitionView(false); TestingResultPartitionManager partitionManager = @@ -464,7 +458,7 @@ public void testGetNextAfterPartitionReleased() throws Exception { createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); channel.requestSubpartition(); - assertFalse(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isEmpty(); // release the subpartition view subpartitionView.releaseAllResources(); @@ -476,12 +470,12 @@ public void testGetNextAfterPartitionReleased() throws Exception { } channel.releaseAllResources(); - assertFalse(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isEmpty(); } /** Verifies that buffer is not compressed when getting from a {@link LocalInputChannel}. */ @Test - public void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Exception { + void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Exception { ResultSubpartitionView subpartitionView = InputChannelTestUtils.createResultSubpartitionView(true); TestingResultPartitionManager partitionManager = @@ -493,22 +487,23 @@ public void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Excepti channel.requestSubpartition(); Optional bufferAndAvailability = channel.getNextBuffer(); - assertTrue(bufferAndAvailability.isPresent()); - assertFalse(bufferAndAvailability.get().buffer().isCompressed()); + assertThat(bufferAndAvailability).isPresent(); + assertThat(bufferAndAvailability.get().buffer().isCompressed()).isFalse(); } - @Test(expected = IllegalStateException.class) - public void testUnblockReleasedChannel() throws Exception { + @Test + void testUnblockReleasedChannel() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); localChannel.releaseAllResources(); - localChannel.resumeConsumption(); + assertThatThrownBy(localChannel::resumeConsumption) + .isInstanceOf(IllegalStateException.class); } - @Test(expected = IllegalStateException.class) - public void testAnnounceBufferSize() throws Exception { + @Test + void testAnnounceBufferSize() throws Exception { // given: Initialized local input channel. AtomicInteger lastBufferSize = new AtomicInteger(0); TestingResultPartitionManager partitionManager = @@ -524,11 +519,12 @@ public void testAnnounceBufferSize() throws Exception { localChannel.releaseAllResources(); // then: Announcement buffer size should lead to exception. - localChannel.announceBufferSize(12); + assertThatThrownBy(() -> localChannel.announceBufferSize(12)) + .isInstanceOf(IllegalStateException.class); } @Test - public void testEnqueueAvailableChannelWhenResuming() throws IOException, InterruptedException { + void testEnqueueAvailableChannelWhenResuming() throws IOException, InterruptedException { PipelinedResultPartition parent = (PipelinedResultPartition) PartitionTestUtils.createPartition( @@ -548,24 +544,24 @@ public void testEnqueueAvailableChannelWhenResuming() throws IOException, Interr new CheckpointBarrier( 1, 1, CheckpointOptions.forCheckpointWithDefaultLocation()), false)); - assertTrue(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isPresent(); // Add more data subpartition.add(createFilledFinishedBufferConsumer(4096)); subpartition.flush(); // No buffer since the subpartition is blocked. - assertFalse(channel.inputGate.pollNext().isPresent()); + assertThat(channel.inputGate.pollNext()).isEmpty(); // Resumption makes the subpartition available. channel.resumeConsumption(); Optional nextBuffer = channel.inputGate.pollNext(); - assertTrue(nextBuffer.isPresent()); - assertTrue(nextBuffer.get().isBuffer()); + assertThat(nextBuffer).isPresent(); + assertThat(nextBuffer.get().isBuffer()).isTrue(); } @Test - public void testCheckpointingInflightData() throws Exception { + void testCheckpointingInflightData() throws Exception { SingleInputGate inputGate = new SingleInputGateBuilder().build(); PipelinedResultPartition parent = @@ -595,23 +591,23 @@ public void testCheckpointingInflightData() throws Exception { // add 1 buffer before barrier and 1 buffer afterwards. Only the first buffer should be // written. subpartition.add(createFilledFinishedBufferConsumer(1)); - assertTrue(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isPresent(); subpartition.add(EventSerializer.toBufferConsumer(barrier, true)); - assertTrue(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isPresent(); subpartition.add(createFilledFinishedBufferConsumer(2)); - assertTrue(channel.getNextBuffer().isPresent()); + assertThat(channel.getNextBuffer()).isPresent(); - assertArrayEquals( - stateWriter.getAddedInput().get(channel.getChannelInfo()).stream() - .mapToInt(Buffer::getSize) - .toArray(), - new int[] {1}); + assertThat( + stateWriter.getAddedInput().get(channel.getChannelInfo()).stream() + .mapToInt(Buffer::getSize) + .toArray()) + .isEqualTo(new int[] {1}); } @Test - public void testAnnounceNewBufferSize() throws IOException, InterruptedException { + void testAnnounceNewBufferSize() throws IOException, InterruptedException { // given: Configured LocalInputChannel and pipelined subpartition. PipelinedResultPartition parent = (PipelinedResultPartition) @@ -636,20 +632,22 @@ public void testAnnounceNewBufferSize() throws IOException, InterruptedException channel1.requestSubpartition(); // and: Preferable buffer size is default value. - assertEquals(Integer.MAX_VALUE, subpartition0.add(createFilledFinishedBufferConsumer(16))); - assertEquals(Integer.MAX_VALUE, subpartition1.add(createFilledFinishedBufferConsumer(16))); + assertThat(subpartition0.add(createFilledFinishedBufferConsumer(16))) + .isEqualTo(Integer.MAX_VALUE); + assertThat(subpartition1.add(createFilledFinishedBufferConsumer(16))) + .isEqualTo(Integer.MAX_VALUE); // when: Announce the different buffer size for different channels via LocalInputChannel. channel0.announceBufferSize(9); channel1.announceBufferSize(20); // then: The corresponded subpartitions have the new size. - assertEquals(9, subpartition0.add(createFilledFinishedBufferConsumer(16))); - assertEquals(20, subpartition1.add(createFilledFinishedBufferConsumer(16))); + assertThat(subpartition0.add(createFilledFinishedBufferConsumer(16))).isEqualTo(9); + assertThat(subpartition1.add(createFilledFinishedBufferConsumer(16))).isEqualTo(20); } @Test - public void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() throws Exception { + void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() throws Exception { // given: Local input channel without initialized subpartition view. ResultSubpartitionView subpartitionView = InputChannelTestUtils.createResultSubpartitionView( @@ -664,13 +662,13 @@ public void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() thro inputGate.setInputChannels(localChannel); // then: Buffers in use should be equal to 0 until subpartition view initialization. - assertEquals(0, localChannel.getBuffersInUseCount()); + assertThat(localChannel.getBuffersInUseCount()).isEqualTo(0); // when: The subpartition view is initialized. localChannel.requestSubpartition(); // then: Buffers in use should show correct value. - assertEquals(3, localChannel.getBuffersInUseCount()); + assertThat(localChannel.getBuffersInUseCount()).isEqualTo(3); } // --------------------------------------------------------------------------------------------- @@ -682,7 +680,7 @@ private static class TestPartitionProducerBufferSource implements TestProducerSo private final List channelIndexes; - public TestPartitionProducerBufferSource( + TestPartitionProducerBufferSource( int parallelism, int bufferSize, int numberOfBuffersToProduce) { this.bufferSize = bufferSize; @@ -723,7 +721,7 @@ private static class TestLocalInputChannelConsumer implements Callable { private final int numberOfExpectedBuffersPerChannel; - public TestLocalInputChannelConsumer( + TestLocalInputChannelConsumer( int subpartitionIndex, int numberOfInputChannels, int numberOfExpectedBuffersPerChannel, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java index eb3fb01180a15..aaf32079ec414 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java @@ -24,32 +24,40 @@ import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.junit.Test; - -import java.io.IOException; +import org.junit.jupiter.api.Test; import static org.apache.flink.runtime.checkpoint.CheckpointOptions.unaligned; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link RecoveredInputChannel}. */ -public class RecoveredInputChannelTest { +class RecoveredInputChannelTest { - @Test(expected = IllegalStateException.class) - public void testConversionOnlyPossibleAfterConsumed() throws IOException { - buildChannel().toInputChannel(); + @Test + void testConversionOnlyPossibleAfterConsumed() { + assertThatThrownBy(() -> buildChannel().toInputChannel()) + .isInstanceOf(IllegalStateException.class); } - @Test(expected = UnsupportedOperationException.class) - public void testRequestPartitionsImpossible() { - buildChannel().requestSubpartition(); + @Test + void testRequestPartitionsImpossible() { + assertThatThrownBy(() -> buildChannel().requestSubpartition()) + .isInstanceOf(UnsupportedOperationException.class); } - @Test(expected = CheckpointException.class) - public void testCheckpointStartImpossible() throws CheckpointException { - buildChannel() - .checkpointStarted( - new CheckpointBarrier( - 0L, 0L, unaligned(CheckpointType.CHECKPOINT, getDefault()))); + @Test + void testCheckpointStartImpossible() { + assertThatThrownBy( + () -> + buildChannel() + .checkpointStarted( + new CheckpointBarrier( + 0L, + 0L, + unaligned( + CheckpointType.CHECKPOINT, + getDefault())))) + .isInstanceOf(CheckpointException.class); } private RecoveredInputChannel buildChannel() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 8b2e42e5544a9..c04696cbbf79a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -58,15 +58,14 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TestTaskBuilder; import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.testutils.executor.TestExecutorResource; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.shaded.guava31.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; import javax.annotation.Nullable; @@ -100,18 +99,9 @@ import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.hasProperty; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.isA; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -120,10 +110,10 @@ import static org.mockito.Mockito.when; /** Tests for the {@link RemoteInputChannel}. */ -public class RemoteInputChannelTest { - @ClassRule - public static final TestExecutorResource EXECUTOR_RESOURCE = - TestingUtils.defaultExecutorResource(); +class RemoteInputChannelTest { + @RegisterExtension + private static final TestExecutorExtension EXECUTOR_EXTENSION = + TestingUtils.defaultExecutorExtension(); private static final long CHECKPOINT_ID = 1L; private static final CheckpointOptions UNALIGNED = @@ -132,7 +122,7 @@ public class RemoteInputChannelTest { alignedWithTimeout(CheckpointType.CHECKPOINT, getDefault(), 10); @Test - public void testGateNotifiedOnBarrierConversion() throws IOException, InterruptedException { + void testGateNotifiedOnBarrierConversion() throws IOException, InterruptedException { final int sequenceNumber = 0; final NetworkBufferPool networkBufferPool = new NetworkBufferPool(1, 4096); try { @@ -164,7 +154,7 @@ public void testGateNotifiedOnBarrierConversion() throws IOException, Interrupte inputGate.pollNext(); // process announcement to allow the gate remember the SQN channel.convertToPriorityEvent(sequenceNumber); - assertTrue(inputGate.getPriorityEventAvailableFuture().isDone()); + assertThat(inputGate.getPriorityEventAvailableFuture().isDone()).isTrue(); } finally { networkBufferPool.destroy(); @@ -172,7 +162,7 @@ public void testGateNotifiedOnBarrierConversion() throws IOException, Interrupte } @Test - public void testExceptionOnReordering() throws Exception { + void testExceptionOnReordering() throws Exception { // Setup final SingleInputGate inputGate = createSingleInputGate(1); final RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate); @@ -189,15 +179,15 @@ public void testExceptionOnReordering() throws Exception { fail("Did not throw expected exception after enqueuing an out-of-order buffer."); } catch (Exception expected) { - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); // free remaining buffer instances inputChannel.releaseAllResources(); - assertTrue(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isTrue(); } } @Test - public void testExceptionOnPersisting() throws Exception { + void testExceptionOnPersisting() throws Exception { // Setup final SingleInputGate inputGate = createSingleInputGate(1); final RemoteInputChannel inputChannel = @@ -228,7 +218,7 @@ public void addInputData( final Buffer buffer = createBuffer(TestBufferFactory.BUFFER_SIZE); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); try { inputChannel.onBuffer(buffer, 0, -1); fail("This should have failed"); @@ -248,14 +238,14 @@ public void addInputData( // In other words, if you end up reading this after refactoring RemoteInputChannel, it might // be safe to remove this assertion. Just make sure double recycling of the same buffer is // still throwing IllegalReferenceCountException. - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); inputChannel.releaseAllResources(); - assertTrue(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isTrue(); } @Test - public void testConcurrentOnBufferAndRelease() throws Exception { + void testConcurrentOnBufferAndRelease() throws Exception { testConcurrentReleaseAndSomething( 8192, (inputChannel, buffer, j) -> { @@ -265,7 +255,7 @@ public void testConcurrentOnBufferAndRelease() throws Exception { } @Test - public void testConcurrentNotifyBufferAvailableAndRelease() throws Exception { + void testConcurrentNotifyBufferAvailableAndRelease() throws Exception { testConcurrentReleaseAndSomething( 1024, (inputChannel, buffer, j) -> @@ -333,30 +323,31 @@ private void testConcurrentReleaseAndSomething( result.get(); } - assertEquals( - "Resource leak during concurrent release and notifyBufferAvailable.", - 0, - inputChannel.getNumberOfQueuedBuffers()); + assertThat(inputChannel.getNumberOfQueuedBuffers()) + .withFailMessage( + "Resource leak during concurrent release and notifyBufferAvailable.") + .isEqualTo(0); } } finally { executor.shutdown(); - assertFalse(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isFalse(); buffer.recycleBuffer(); - assertTrue(buffer.isRecycled()); + assertThat(buffer.isRecycled()).isTrue(); } } - @Test(expected = IllegalStateException.class) - public void testRetriggerWithoutPartitionRequest() throws Exception { + @Test + void testRetriggerWithoutPartitionRequest() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); RemoteInputChannel ch = createRemoteInputChannel(inputGate, 0, 500, 3000); - ch.retriggerSubpartitionRequest(); + assertThatThrownBy(ch::retriggerSubpartitionRequest) + .isInstanceOf(IllegalStateException.class); } @Test - public void testPartitionRequestExponentialBackoff() throws Exception { + void testPartitionRequestExponentialBackoff() throws Exception { // Start with initial backoff, then keep doubling, and cap at max. int[] expectedDelays = {500, 1000, 2000, 3000}; @@ -389,7 +380,7 @@ public void testPartitionRequestExponentialBackoff() throws Exception { } @Test - public void testPartitionRequestSingleBackoff() throws Exception { + void testPartitionRequestSingleBackoff() throws Exception { // Setup SingleInputGate inputGate = createSingleInputGate(1); ResultPartitionID partitionId = new ResultPartitionID(); @@ -416,7 +407,7 @@ public void testPartitionRequestSingleBackoff() throws Exception { } @Test - public void testPartitionRequestNoBackoff() throws Exception { + void testPartitionRequestNoBackoff() throws Exception { // Setup SingleInputGate inputGate = createSingleInputGate(1); ResultPartitionID partitionId = new ResultPartitionID(); @@ -439,7 +430,7 @@ public void testPartitionRequestNoBackoff() throws Exception { } @Test - public void testOnFailedPartitionRequest() throws Exception { + void testOnFailedPartitionRequest() throws Exception { final ResultPartitionID partitionId = new ResultPartitionID(); final TestPartitionProducerStateProvider provider = new TestPartitionProducerStateProvider(partitionId); @@ -452,11 +443,11 @@ public void testOnFailedPartitionRequest() throws Exception { ch.onFailedPartitionRequest(); - assertTrue(provider.isInvoked()); + assertThat(provider.isInvoked()).isTrue(); } - @Test(expected = CancelTaskException.class) - public void testProducerFailedException() throws Exception { + @Test + void testProducerFailedException() throws Exception { ConnectionManager connManager = mock(ConnectionManager.class); when(connManager.createPartitionRequestClient(any(ConnectionID.class))) @@ -471,11 +462,11 @@ public void testProducerFailedException() throws Exception { ch.requestSubpartition(); // Should throw an instance of CancelTaskException. - ch.getNextBuffer(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(CancelTaskException.class); } - @Test(expected = PartitionConnectionException.class) - public void testPartitionConnectionException() throws IOException { + @Test + void testPartitionConnectionException() { final ConnectionManager connManager = new TestingExceptionConnectionManager(); final SingleInputGate gate = createSingleInputGate(1); final RemoteInputChannel ch = @@ -484,7 +475,7 @@ public void testPartitionConnectionException() throws IOException { gate.requestPartitions(); - ch.getNextBuffer(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(PartitionConnectionException.class); } /** @@ -498,7 +489,7 @@ public void testPartitionConnectionException() throws IOException { * */ @Test - public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { + void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // Setup final NetworkBufferPool networkBufferPool = new NetworkBufferPool(16, 32); final int numFloatingBuffers = 14; @@ -516,14 +507,14 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); - assertNotNull(exclusiveBuffer); + assertThat(exclusiveBuffer).isNotNull(); final int numRecycleFloatingBuffers = 2; final ArrayDeque floatingBufferQueue = new ArrayDeque<>(numRecycleFloatingBuffers); for (int i = 0; i < numRecycleFloatingBuffers; i++) { Buffer floatingBuffer = bufferPool.requestBuffer(); - assertNotNull(floatingBuffer); + assertThat(floatingBuffer).isNotNull(); floatingBufferQueue.add(floatingBuffer); } @@ -537,19 +528,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // It does not get enough floating buffers and register as buffer listener verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 13 buffers available in the channel", - 13, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 16 buffers required in the channel", - 16, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 13 buffers available in the channel") + .isEqualTo(13); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 16 buffers required in the channel") + .isEqualTo(16); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Increase the backlog inputChannel.onSenderBacklog(16); @@ -558,19 +546,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // more verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 13 buffers available in the channel", - 13, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 18 buffers required in the channel", - 18, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 13 buffers available in the channel") + .isEqualTo(13); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 18 buffers required in the channel") + .isEqualTo(18); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one exclusive buffer exclusiveBuffer.recycleBuffer(); @@ -578,19 +563,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // The exclusive buffer is returned to the channel directly verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 18 buffers required in the channel", - 18, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 18 buffers required in the channel") + .isEqualTo(18); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one floating buffer floatingBufferQueue.poll().recycleBuffer(); @@ -599,19 +581,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // floating buffers verify(bufferPool, times(16)).requestBuffer(); verify(bufferPool, times(2)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 15 buffers available in the channel", - 15, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 18 buffers required in the channel", - 18, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 15 buffers available in the channel") + .isEqualTo(15); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 18 buffers required in the channel") + .isEqualTo(18); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Decrease the backlog inputChannel.onSenderBacklog(13); @@ -619,19 +598,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // Only the number of required buffers is changed by (backlog + numExclusiveBuffers) verify(bufferPool, times(16)).requestBuffer(); verify(bufferPool, times(2)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 15 buffers available in the channel", - 15, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 15 buffers required in the channel", - 15, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 15 buffers available in the channel") + .isEqualTo(15); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 15 buffers required in the channel") + .isEqualTo(15); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one more floating buffer floatingBufferQueue.poll().recycleBuffer(); @@ -640,19 +616,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // floating buffers verify(bufferPool, times(16)).requestBuffer(); verify(bufferPool, times(2)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 15 buffers available in the channel", - 15, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 15 buffers required in the channel", - 15, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 1 buffers available in local pool", - 1, - bufferPool.getNumberOfAvailableMemorySegments()); - assertFalse(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 15 buffers available in the channel") + .isEqualTo(15); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 15 buffers required in the channel") + .isEqualTo(15); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 1 buffers available in local pool") + .isEqualTo(1); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isFalse(); // Increase the backlog again inputChannel.onSenderBacklog(15); @@ -661,19 +634,16 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { // as listener again. verify(bufferPool, times(18)).requestBuffer(); verify(bufferPool, times(3)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 16 buffers available in the channel", - 16, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 17 buffers required in the channel", - 17, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); - assertTrue(inputChannel.isWaitingForFloatingBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 16 buffers available in the channel") + .isEqualTo(16); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 17 buffers required in the channel") + .isEqualTo(17); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); + assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); } catch (Throwable t) { thrown = t; } finally { @@ -686,7 +656,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { * available buffers equals to required buffers. */ @Test - public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { + void testAvailableBuffersEqualToRequiredBuffers() throws Exception { // Setup final NetworkBufferPool networkBufferPool = new NetworkBufferPool(16, 32); final int numFloatingBuffers = 14; @@ -704,9 +674,9 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); - assertNotNull(exclusiveBuffer); + assertThat(exclusiveBuffer).isNotNull(); final Buffer floatingBuffer = bufferPool.requestBuffer(); - assertNotNull(floatingBuffer); + assertThat(floatingBuffer).isNotNull(); verify(bufferPool, times(1)).requestBuffer(); // Receive the producer's backlog @@ -716,18 +686,15 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { // and gets enough floating buffers verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 14 buffers required in the channel", - 14, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 14 buffers required in the channel") + .isEqualTo(14); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); // Recycle one floating buffer floatingBuffer.recycleBuffer(); @@ -737,18 +704,15 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { // for floating buffers verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 14 buffers required in the channel", - 14, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 1 buffer available in local pool", - 1, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 14 buffers required in the channel") + .isEqualTo(14); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 1 buffers available in local pool") + .isEqualTo(1); // Recycle one exclusive buffer exclusiveBuffer.recycleBuffer(); @@ -758,18 +722,15 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { // already equals to required buffers verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 14 buffers required in the channel", - 14, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 2 buffers available in local pool", - 2, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 14 buffers required in the channel") + .isEqualTo(14); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 2 buffers available in local pool") + .isEqualTo(2); } catch (Throwable t) { thrown = t; } finally { @@ -782,7 +743,7 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { * available buffers is more than required buffers by decreasing the sender's backlog. */ @Test - public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { + void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // Setup final NetworkBufferPool networkBufferPool = new NetworkBufferPool(16, 32); final int numFloatingBuffers = 14; @@ -800,10 +761,10 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); - assertNotNull(exclusiveBuffer); + assertThat(exclusiveBuffer).isNotNull(); final Buffer floatingBuffer = bufferPool.requestBuffer(); - assertNotNull(floatingBuffer); + assertThat(floatingBuffer).isNotNull(); verify(bufferPool, times(1)).requestBuffer(); @@ -813,18 +774,15 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // The channel gets enough floating buffers from local pool verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 14 buffers required in the channel", - 14, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 14 buffers required in the channel") + .isEqualTo(14); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); // Decrease the backlog to make the number of available buffers more than required // buffers @@ -833,18 +791,15 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // Only the number of required buffers is changed by (backlog + numExclusiveBuffers) verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 12 buffers required in the channel", - 12, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 0 buffers available in local pool", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 12 buffers required in the channel") + .isEqualTo(12); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 0 buffers available in local pool") + .isEqualTo(0); // Recycle one exclusive buffer exclusiveBuffer.recycleBuffer(); @@ -854,18 +809,15 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // is more than required buffers verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 12 buffers required in the channel", - 12, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 1 buffer available in local pool", - 1, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 12 buffers required in the channel") + .isEqualTo(12); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 1 buffers available in local pool") + .isEqualTo(1); // Recycle one floating buffer floatingBuffer.recycleBuffer(); @@ -875,18 +827,15 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { // floating buffers verify(bufferPool, times(14)).requestBuffer(); verify(bufferPool, times(0)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be 14 buffers available in the channel", - 14, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 12 buffers required in the channel", - 12, - inputChannel.getNumberOfRequiredBuffers()); - assertEquals( - "There should be 2 buffers available in local pool", - 2, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 14 buffers available in the channel") + .isEqualTo(14); + assertThat(inputChannel.getNumberOfRequiredBuffers()) + .withFailMessage("There should be 12 buffers required in the channel") + .isEqualTo(12); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 2 buffers available in local pool") + .isEqualTo(2); } catch (Throwable t) { thrown = t; } finally { @@ -899,7 +848,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { * channel listeners in a fair way. */ @Test - public void testFairDistributionFloatingBuffers() throws Exception { + void testFairDistributionFloatingBuffers() throws Exception { // Setup final int numExclusiveBuffers = 2; final NetworkBufferPool networkBufferPool = new NetworkBufferPool(12, 32); @@ -926,7 +875,7 @@ public void testFairDistributionFloatingBuffers() throws Exception { final List floatingBuffers = new ArrayList<>(numFloatingBuffers); for (int i = 0; i < numFloatingBuffers; i++) { Buffer buffer = bufferPool.requestBuffer(); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); floatingBuffers.add(buffer); } @@ -935,12 +884,11 @@ public void testFairDistributionFloatingBuffers() throws Exception { for (RemoteInputChannel inputChannel : inputChannels) { inputChannel.onSenderBacklog(8); verify(bufferPool, times(1)).addBufferListener(inputChannel.getBufferManager()); - assertEquals( - "There should be " - + numExclusiveBuffers - + " buffers available in the channel", - numExclusiveBuffers, - inputChannel.getNumberOfAvailableBuffers()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage( + "There should be %d buffers available in the channel", + numExclusiveBuffers) + .isEqualTo(numExclusiveBuffers); } // Recycle three floating buffers to trigger notify buffer available @@ -949,14 +897,12 @@ public void testFairDistributionFloatingBuffers() throws Exception { } for (RemoteInputChannel inputChannel : inputChannels) { - assertEquals( - "There should be 3 buffers available in the channel", - 3, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 1 unannounced credits in the channel", - 1, - inputChannel.getUnannouncedCredit()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be 3 buffers available in the channel") + .isEqualTo(3); + assertThat(inputChannel.getUnannouncedCredit()) + .withFailMessage("There should be 1 unannounced credits in the channel") + .isEqualTo(1); } } catch (Throwable t) { thrown = t; @@ -971,7 +917,7 @@ public void testFairDistributionFloatingBuffers() throws Exception { * listener will be notified in this case. */ @Test - public void testFailureInNotifyBufferAvailable() throws Exception { + void testFailureInNotifyBufferAvailable() throws Exception { // Setup final int numExclusiveBuffers = 1; final int numFloatingBuffers = 1; @@ -1003,24 +949,24 @@ public void testFailureInNotifyBufferAvailable() throws Exception { // this exception will be swallowed and set as an error in failingRemoteIC buffer.recycleBuffer(); buffer = null; - try { - failingRemoteIC.checkError(); - fail( - "The input channel should have an error based on the failure in RemoteInputChannel#notifyBufferAvailable()"); - } catch (IOException e) { - assertThat(e, hasProperty("cause", isA(IllegalStateException.class))); - } + + assertThatThrownBy(failingRemoteIC::checkError) + .isInstanceOf(IOException.class) + .hasCauseInstanceOf(IllegalStateException.class); + // currently, the buffer is still enqueued in the bufferQueue of failingRemoteIC - assertEquals(0, bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); buffer = successfulRemoteIC.requestBuffer(); - assertNull("buffer should still remain in failingRemoteIC", buffer); + assertThat(buffer) + .withFailMessage("buffer should still remain in failingRemoteIC") + .isNull(); // releasing resources in failingRemoteIC should free the buffer again and immediately // recycle it into successfulRemoteIC failingRemoteIC.releaseAllResources(); - assertEquals(0, bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); buffer = successfulRemoteIC.requestBuffer(); - assertNotNull("no buffer given to successfulRemoteIC", buffer); + assertThat(buffer).withFailMessage("no buffer given to successfulRemoteIC").isNotNull(); } catch (Throwable t) { thrown = t; } finally { @@ -1034,7 +980,7 @@ public void testFailureInNotifyBufferAvailable() throws Exception { * channel. */ @Test - public void testConcurrentOnSenderBacklogAndRelease() throws Exception { + void testConcurrentOnSenderBacklogAndRelease() throws Exception { // Setup final NetworkBufferPool networkBufferPool = new NetworkBufferPool(130, 32); final int numFloatingBuffers = 128; @@ -1080,15 +1026,14 @@ public Void call() throws Exception { // Submit tasks and wait to finish submitTasksAndWaitForResults(executor, new Callable[] {requestBufferTask, releaseTask}); - assertEquals( - "There should be no buffers available in the channel.", - 0, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be 130 buffers available in local pool.", - 130, - bufferPool.getNumberOfAvailableMemorySegments() - + networkBufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be no buffers available in the channel.") + .isEqualTo(0); + assertThat( + bufferPool.getNumberOfAvailableMemorySegments() + + networkBufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be 130 buffers available in local pool.") + .isEqualTo(130); } catch (Throwable t) { thrown = t; } finally { @@ -1102,7 +1047,7 @@ public Void call() throws Exception { * exclusive buffers. */ @Test - public void testConcurrentOnSenderBacklogAndRecycle() throws Exception { + void testConcurrentOnSenderBacklogAndRecycle() throws Exception { // Setup final int numExclusiveSegments = 120; final NetworkBufferPool networkBufferPool = new NetworkBufferPool(248, 32); @@ -1144,16 +1089,14 @@ public Void call() throws Exception { requestBufferTask }); - assertEquals( - "There should be " - + inputChannel.getNumberOfRequiredBuffers() - + " buffers available in channel.", - inputChannel.getNumberOfRequiredBuffers(), - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be no buffers available in local pool.", - 0, - bufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage( + "There should be %d buffers available in the channel.", + numFloatingBuffers) + .isEqualTo(inputChannel.getNumberOfRequiredBuffers()); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage("There should be no buffers available in local pool.") + .isEqualTo(0); } catch (Throwable t) { thrown = t; } finally { @@ -1167,7 +1110,7 @@ public Void call() throws Exception { * channel. */ @Test - public void testConcurrentRecycleAndRelease() throws Exception { + void testConcurrentRecycleAndRelease() throws Exception { // Setup final int numExclusiveSegments = 120; final NetworkBufferPool networkBufferPool = new NetworkBufferPool(248, 32); @@ -1206,20 +1149,19 @@ public Void call() throws Exception { releaseTask }); - assertEquals( - "There should be no buffers available in the channel.", - 0, - inputChannel.getNumberOfAvailableBuffers()); - assertEquals( - "There should be " + numFloatingBuffers + " buffers available in local pool.", - numFloatingBuffers, - bufferPool.getNumberOfAvailableMemorySegments()); - assertEquals( - "There should be " - + numExclusiveSegments - + " buffers available in global pool.", - numExclusiveSegments, - networkBufferPool.getNumberOfAvailableMemorySegments()); + assertThat(inputChannel.getNumberOfAvailableBuffers()) + .withFailMessage("There should be no buffers available in the channel.") + .isEqualTo(0); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage( + "There should be %d buffers available in local pool.", + numFloatingBuffers) + .isEqualTo(numFloatingBuffers); + assertThat(networkBufferPool.getNumberOfAvailableMemorySegments()) + .withFailMessage( + "There should be %d buffers available in global pool.", + numExclusiveSegments) + .isEqualTo(numExclusiveSegments); } catch (Throwable t) { thrown = t; } finally { @@ -1233,7 +1175,7 @@ public Void call() throws Exception { * recycling of the exclusive buffer triggers recycling a floating buffer (FLINK-9676). */ @Test - public void testConcurrentRecycleAndRelease2() throws Exception { + void testConcurrentRecycleAndRelease2() throws Exception { // Setup final int retries = 1_000; final int numExclusiveBuffers = 2; @@ -1292,7 +1234,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { floatingBuffers.forEach(Buffer::recycleBuffer); floatingBuffers.clear(); - assertEquals(numExclusiveBuffers, exclusiveBuffers.size()); + assertThat(exclusiveBuffers).hasSize(numExclusiveBuffers); inputChannel.onSenderBacklog( 0); // trigger subscription to buffer pool // note: if we got a floating buffer by increasing the backlog, it @@ -1319,7 +1261,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { } @Test - public void testConcurrentGetNextBufferAndRelease() throws Exception { + void testConcurrentGetNextBufferAndRelease() throws Exception { final int numTotalBuffers = 1_000; final int numFloatingBuffers = 998; final NetworkBufferPool networkBufferPool = new NetworkBufferPool(numTotalBuffers, 32); @@ -1379,7 +1321,7 @@ public void testConcurrentGetNextBufferAndRelease() throws Exception { * PartitionNotFoundException} if backoff is 0. */ @Test - public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exception { + void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exception { final RemoteInputChannel inputChannel = InputChannelTestUtils.createRemoteInputChannel( createSingleInputGate(1), 0, new TestingConnectionManager()); @@ -1394,7 +1336,7 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce fail("Should throw a PartitionNotFoundException."); } catch (PartitionNotFoundException notFound) { - assertThat(inputChannel.getPartitionId(), is(notFound.getPartitionId())); + assertThat(inputChannel.getPartitionId()).isEqualTo(notFound.getPartitionId()); } } @@ -1404,7 +1346,7 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce * PartitionConnectionException} during {@link RemoteInputChannel#requestSubpartition()}. */ @Test - public void testPartitionConnectionExceptionWhileRequestingPartition() throws Exception { + void testPartitionConnectionExceptionWhileRequestingPartition() throws Exception { final RemoteInputChannel inputChannel = InputChannelTestUtils.createRemoteInputChannel( createSingleInputGate(1), 0, new TestingExceptionConnectionManager()); @@ -1412,30 +1354,32 @@ public void testPartitionConnectionExceptionWhileRequestingPartition() throws Ex inputChannel.requestSubpartition(); fail("Expected PartitionConnectionException."); } catch (PartitionConnectionException ex) { - assertThat(inputChannel.getPartitionId(), is(ex.getPartitionId())); + assertThat(inputChannel.getPartitionId()).isEqualTo(ex.getPartitionId()); } } - @Test(expected = IllegalStateException.class) - public void testUnblockReleasedChannel() throws Exception { + @Test + void testUnblockReleasedChannel() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); RemoteInputChannel remoteChannel = createRemoteInputChannel(inputGate); remoteChannel.releaseAllResources(); - remoteChannel.resumeConsumption(); + assertThatThrownBy(remoteChannel::resumeConsumption) + .isInstanceOf(IllegalStateException.class); } - @Test(expected = IllegalStateException.class) - public void testReleasedChannelAnnounceBufferSize() throws Exception { + @Test + void testReleasedChannelAnnounceBufferSize() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); RemoteInputChannel remoteChannel = createRemoteInputChannel(inputGate); remoteChannel.releaseAllResources(); - remoteChannel.announceBufferSize(10); + assertThatThrownBy(() -> remoteChannel.announceBufferSize(10)) + .isInstanceOf(IllegalStateException.class); } @Test - public void testOnUpstreamBlockedAndResumed() throws Exception { + void testOnUpstreamBlockedAndResumed() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); @@ -1448,8 +1392,8 @@ public void testOnUpstreamBlockedAndResumed() throws Exception { remoteChannel1.onSenderBacklog(2); remoteChannel2.onSenderBacklog(2); - assertEquals(4, remoteChannel1.getNumberOfAvailableBuffers()); - assertEquals(2, remoteChannel2.getNumberOfAvailableBuffers()); + assertThat(remoteChannel1.getNumberOfAvailableBuffers()).isEqualTo(4); + assertThat(remoteChannel2.getNumberOfAvailableBuffers()).isEqualTo(2); Buffer barrier = EventSerializer.toBuffer( @@ -1464,27 +1408,27 @@ public void testOnUpstreamBlockedAndResumed() throws Exception { remoteChannel1.onBuffer(barrier, 0, 0); remoteChannel2.onBuffer(barrier, 0, 0); - assertEquals(4, remoteChannel1.getNumberOfAvailableBuffers()); - assertEquals(0, remoteChannel2.getNumberOfAvailableBuffers()); + assertThat(remoteChannel1.getNumberOfAvailableBuffers()).isEqualTo(4); + assertThat(remoteChannel2.getNumberOfAvailableBuffers()).isEqualTo(0); remoteChannel1.resumeConsumption(); remoteChannel2.resumeConsumption(); - assertEquals(4, remoteChannel1.getUnannouncedCredit()); - assertEquals(0, remoteChannel2.getUnannouncedCredit()); + assertThat(remoteChannel1.getUnannouncedCredit()).isEqualTo(4); + assertThat(remoteChannel2.getUnannouncedCredit()).isEqualTo(0); remoteChannel1.onSenderBacklog(4); remoteChannel2.onSenderBacklog(4); - assertEquals(6, remoteChannel1.getNumberOfAvailableBuffers()); - assertEquals(4, remoteChannel2.getNumberOfAvailableBuffers()); + assertThat(remoteChannel1.getNumberOfAvailableBuffers()).isEqualTo(6); + assertThat(remoteChannel2.getNumberOfAvailableBuffers()).isEqualTo(4); - assertEquals(6, remoteChannel1.getUnannouncedCredit()); - assertEquals(4, remoteChannel2.getUnannouncedCredit()); + assertThat(remoteChannel1.getUnannouncedCredit()).isEqualTo(6); + assertThat(remoteChannel2.getUnannouncedCredit()).isEqualTo(4); } @Test - public void testRequestBuffer() throws Exception { + void testRequestBuffer() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); @@ -1498,18 +1442,18 @@ public void testRequestBuffer() throws Exception { remoteChannel2.onSenderBacklog(2); for (int i = 4; i >= 0; --i) { - assertEquals(i, remoteChannel1.getNumberOfRequiredBuffers()); + assertThat(remoteChannel1.getNumberOfRequiredBuffers()).isEqualTo(i); remoteChannel1.requestBuffer(); } for (int i = 2; i >= 0; --i) { - assertEquals(i, remoteChannel2.getNumberOfRequiredBuffers()); + assertThat(remoteChannel2.getNumberOfRequiredBuffers()).isEqualTo(i); remoteChannel2.requestBuffer(); } } @Test - public void testPrioritySequenceNumbers() throws Exception { + void testPrioritySequenceNumbers() throws Exception { int sequenceNumber = 0; int bufferSize = 1; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1523,7 +1467,7 @@ public void testPrioritySequenceNumbers() throws Exception { } @Test - public void testGetInflightBuffers() throws Exception { + void testGetInflightBuffers() throws Exception { int bufferSize = 1; int sequenceNumber = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1536,7 +1480,7 @@ public void testGetInflightBuffers() throws Exception { } @Test - public void testGetAllInflightBuffers() throws Exception { + void testGetAllInflightBuffers() throws Exception { int sequenceNumber = Integer.MAX_VALUE - 2; int bufferSize = 1; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1548,7 +1492,7 @@ public void testGetAllInflightBuffers() throws Exception { } @Test - public void testGetInflightBuffersOverflow() throws Exception { + void testGetInflightBuffersOverflow() throws Exception { for (int startingSequence = Integer.MAX_VALUE - 10; startingSequence != Integer.MIN_VALUE + 2; startingSequence++) { @@ -1560,15 +1504,14 @@ public void testGetInflightBuffersOverflow() throws Exception { sendBarrier(channel, sequenceNumber++, UNALIGNED); sendBuffer(channel, sequenceNumber++, bufferSize++); sendBuffer(channel, sequenceNumber++, bufferSize++); - assertThat( - "For starting sequence " + startingSequence, - toBufferSizes(channel.getInflightBuffers(CHECKPOINT_ID)), - contains(1, 2)); + assertThat(toBufferSizes(channel.getInflightBuffers(CHECKPOINT_ID))) + .withFailMessage("For starting sequence " + startingSequence) + .contains(1, 2); } } @Test - public void testGetInflightBuffersAfterPollingBuffer() throws Exception { + void testGetInflightBuffersAfterPollingBuffer() throws Exception { int bufferSize = 1; int sequenceNumber = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1588,7 +1531,7 @@ private static List toBufferSizes(List inflightBuffers) { } @Test - public void testRequiresAnnouncement() throws Exception { + void testRequiresAnnouncement() throws Exception { int sequenceNumber = 0; int bufferSize = 1; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1599,23 +1542,23 @@ public void testRequiresAnnouncement() throws Exception { sendBuffer(channel, sequenceNumber++, bufferSize++); BufferAndAvailability nextBuffer = channel.getNextBuffer().get(); - assertEquals(2, nextBuffer.getSequenceNumber()); - assertFalse(nextBuffer.morePriorityEvents()); - assertTrue(nextBuffer.moreAvailable()); - assertEquals(DataType.PRIORITIZED_EVENT_BUFFER, nextBuffer.buffer().getDataType()); + assertThat(nextBuffer.getSequenceNumber()).isEqualTo(2); + assertThat(nextBuffer.morePriorityEvents()).isFalse(); + assertThat(nextBuffer.moreAvailable()).isTrue(); + assertThat(nextBuffer.buffer().getDataType()).isEqualTo(DataType.PRIORITIZED_EVENT_BUFFER); assertGetNextBufferSequenceNumbers(channel, 0, 1); nextBuffer = channel.getNextBuffer().get(); - assertEquals(2, nextBuffer.getSequenceNumber()); - assertEquals( - DataType.TIMEOUTABLE_ALIGNED_CHECKPOINT_BARRIER, nextBuffer.buffer().getDataType()); + assertThat(nextBuffer.getSequenceNumber()).isEqualTo(2); + assertThat(nextBuffer.buffer().getDataType()) + .isEqualTo(DataType.TIMEOUTABLE_ALIGNED_CHECKPOINT_BARRIER); - assertEquals(3, channel.getNextBuffer().get().getSequenceNumber()); + assertThat(channel.getNextBuffer().get().getSequenceNumber()).isEqualTo(3); } @Test - public void testGetInflightBuffersBeforeProcessingAnnouncement() throws Exception { + void testGetInflightBuffersBeforeProcessingAnnouncement() throws Exception { int bufferSize = 1; int sequenceNumber = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1628,7 +1571,7 @@ public void testGetInflightBuffersBeforeProcessingAnnouncement() throws Exceptio } @Test - public void testGetInflightBuffersAfterProcessingAnnouncement() throws Exception { + void testGetInflightBuffersAfterProcessingAnnouncement() throws Exception { int bufferSize = 1; int sequenceNumber = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1642,7 +1585,7 @@ public void testGetInflightBuffersAfterProcessingAnnouncement() throws Exception } @Test - public void testGetInflightBuffersAfterProcessingAnnouncementAndBuffer() throws Exception { + void testGetInflightBuffersAfterProcessingAnnouncementAndBuffer() throws Exception { int bufferSize = 1; int sequenceNumber = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); @@ -1656,18 +1599,18 @@ public void testGetInflightBuffersAfterProcessingAnnouncementAndBuffer() throws } @Test - public void testSizeOfQueuedBuffers() throws Exception { + void testSizeOfQueuedBuffers() throws Exception { int sequenceNumber = 0; int bufferSize = 1; int queueSize = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); - assertEquals(0, channel.unsynchronizedGetSizeOfQueuedBuffers()); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(0); // Receive a couple of buffers. for (int i = 0; i < 2; i++) { queueSize += bufferSize; sendBuffer(channel, sequenceNumber++, bufferSize++); - assertEquals(queueSize, channel.unsynchronizedGetSizeOfQueuedBuffers()); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(queueSize); } // Receive the event. @@ -1675,16 +1618,16 @@ public void testSizeOfQueuedBuffers() throws Exception { EventSerializer.toSerializedEvent(new CheckpointBarrier(1L, 123L, UNALIGNED)) .remaining(); sendBarrier(channel, sequenceNumber++, UNALIGNED); - assertEquals(queueSize, channel.unsynchronizedGetSizeOfQueuedBuffers()); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(queueSize); // Poll all received buffers. for (int i = 0; i < 3; i++) { Optional nextBuffer = channel.getNextBuffer(); queueSize -= nextBuffer.get().buffer().getSize(); - assertEquals(queueSize, channel.unsynchronizedGetSizeOfQueuedBuffers()); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(queueSize); } - assertEquals(0, channel.unsynchronizedGetSizeOfQueuedBuffers()); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(0); } private void sendBarrier( @@ -1711,9 +1654,8 @@ private void send(RemoteInputChannel channel, int sequenceNumber, Buffer buffer) private void assertInflightBufferSizes(RemoteInputChannel channel, Integer... bufferSizes) throws CheckpointException { - assertEquals( - Arrays.asList(bufferSizes), - toBufferSizes(channel.getInflightBuffers(CHECKPOINT_ID))); + assertThat(toBufferSizes(channel.getInflightBuffers(CHECKPOINT_ID))) + .isEqualTo(Arrays.asList(bufferSizes)); } private void assertGetNextBufferSequenceNumbers( @@ -1724,7 +1666,7 @@ private void assertGetNextBufferSequenceNumbers( .map(BufferAndAvailability::getSequenceNumber) .ifPresent(actualSequenceNumbers::add); } - assertThat(actualSequenceNumbers, contains(sequenceNumbers)); + assertThat(actualSequenceNumbers).contains(sequenceNumbers); } // --------------------------------------------------------------------------------------------- @@ -1793,7 +1735,7 @@ private SingleInputGate buildInputGate() throws IOException { /** Test to guard against FLINK-13249. */ @Test - public void testOnFailedPartitionRequestDoesNotBlockNetworkThreads() throws Exception { + void testOnFailedPartitionRequestDoesNotBlockNetworkThreads() throws Exception { final long testBlockedWaitTimeoutMillis = 30_000L; @@ -1805,7 +1747,7 @@ public void testOnFailedPartitionRequestDoesNotBlockNetworkThreads() throws Exce final Task task = new TestTaskBuilder(shuffleEnvironment) .setPartitionProducerStateChecker(partitionProducerStateChecker) - .build(EXECUTOR_RESOURCE.getExecutor()); + .build(EXECUTOR_EXTENSION.getExecutor()); final SingleInputGate inputGate = new SingleInputGateBuilder().setPartitionProducerStateProvider(task).build(); @@ -1865,13 +1807,14 @@ public PartitionRequestClient createPartitionRequestClient( simulatedNetworkThread.join(); - Assert.assertFalse( - "Test ended by timeout or interruption - this indicates that the network thread was blocked.", - timedOutOrInterrupted.get()); + assertThat(timedOutOrInterrupted.get()) + .withFailMessage( + "Test ended by timeout or interruption - this indicates that the network thread was blocked.") + .isFalse(); } @Test - public void testNotifyOnPriority() throws IOException { + void testNotifyOnPriority() throws IOException { SingleInputGate inputGate = new SingleInputGateBuilder().build(); RemoteInputChannel channel = InputChannelTestUtils.createRemoteInputChannel(inputGate, 0); @@ -1912,7 +1855,7 @@ public void testNotifyOnPriority() throws IOException { } @Test - public void testBuffersInUseCount() throws Exception { + void testBuffersInUseCount() throws Exception { // Setup RemoteInputChannel remoteInputChannel = buildInputGateAndGetChannel(); @@ -1921,28 +1864,28 @@ public void testBuffersInUseCount() throws Exception { // Receiving the buffer with backlog. remoteInputChannel.onBuffer(buffer.retainBuffer(), 0, 1); // 1 buffer + 1 backlog. - assertEquals(2, remoteInputChannel.getBuffersInUseCount()); + assertThat(remoteInputChannel.getBuffersInUseCount()).isEqualTo(2); remoteInputChannel.onBuffer(buffer.retainBuffer(), 1, 3); // 2 buffer + 3 backlog. - assertEquals(5, remoteInputChannel.getBuffersInUseCount()); + assertThat(remoteInputChannel.getBuffersInUseCount()).isEqualTo(5); // 1 buffer + 3 backlog. remoteInputChannel.getNextBuffer(); - assertEquals(4, remoteInputChannel.getBuffersInUseCount()); + assertThat(remoteInputChannel.getBuffersInUseCount()).isEqualTo(4); // 0 buffer + 3 backlog. remoteInputChannel.getNextBuffer(); - assertEquals(3, remoteInputChannel.getBuffersInUseCount()); + assertThat(remoteInputChannel.getBuffersInUseCount()).isEqualTo(3); // 0 buffer + 3 backlog. Nothing changes from previous case because receivedBuffers was // already empty. remoteInputChannel.getNextBuffer(); - assertEquals(3, remoteInputChannel.getBuffersInUseCount()); + assertThat(remoteInputChannel.getBuffersInUseCount()).isEqualTo(3); } @Test - public void testReleasedChannelNotifyRequiredSegmentId() throws Exception { + void testReleasedChannelNotifyRequiredSegmentId() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); RemoteInputChannel remoteChannel = createRemoteInputChannel(inputGate); @@ -1972,7 +1915,7 @@ private Callable recycleBufferTask( // Exhaust all the exclusive buffers for (int i = 0; i < numExclusiveSegments; i++) { Buffer buffer = inputChannel.requestBuffer(); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); exclusiveBuffers.add(buffer); } @@ -1980,7 +1923,7 @@ private Callable recycleBufferTask( // Exhaust all the floating buffers for (int i = 0; i < numFloatingBuffers; i++) { Buffer buffer = bufferPool.requestBuffer(); - assertNotNull(buffer); + assertThat(buffer).isNotNull(); floatingBuffers.add(buffer); } @@ -2094,7 +2037,7 @@ public void requestPartitionProducerState( ResultPartitionID resultPartitionId, Consumer responseConsumer) { - assertEquals(partitionId, resultPartitionId); + assertThat(resultPartitionId).isEqualTo(partitionId); isInvoked = true; } @@ -2135,9 +2078,9 @@ public void requestSubpartition( void verifyResult( ResultPartitionID expectedId, int expectedSubpartitionIndex, int expectedDelayMs) { - assertEquals(expectedId, partitionId); - assertEquals(expectedSubpartitionIndex, subpartitionIndex); - assertEquals(expectedDelayMs, delayMs); + assertThat(partitionId).isEqualTo(expectedId); + assertThat(subpartitionIndex).isEqualTo(expectedSubpartitionIndex); + assertThat(delayMs).isEqualTo(expectedDelayMs); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index 296124942ac39..ffc8cddf61fd1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -41,7 +41,7 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.fail; /** A mocked input channel. */ public class TestInputChannel extends InputChannel { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index de507e2c147ad..0e16a2e0ba926 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -26,20 +26,17 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.TestingResultPartitionManager; -import org.hamcrest.Matchers; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import static org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.verifyBufferOrEvent; import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link UnionInputGate}. */ -public class UnionInputGateTest extends InputGateTestBase { +class UnionInputGateTest extends InputGateTestBase { /** * Tests basic correctness of buffer-or-event interleaving and correct null return @@ -48,17 +45,17 @@ public class UnionInputGateTest extends InputGateTestBase { *

For buffer-or-event instances, it is important to verify that they have been set off to * the correct logical index. */ - @Test(timeout = 120 * 1000) - public void testBasicGetNextLogic() throws Exception { + @Test + @Timeout(120) + void testBasicGetNextLogic() throws Exception { // Setup final SingleInputGate ig1 = createInputGate(3); final SingleInputGate ig2 = createInputGate(5); final UnionInputGate union = new UnionInputGate(new SingleInputGate[] {ig1, ig2}); - assertEquals( - ig1.getNumberOfInputChannels() + ig2.getNumberOfInputChannels(), - union.getNumberOfInputChannels()); + assertThat(union.getNumberOfInputChannels()) + .isEqualTo(ig1.getNumberOfInputChannels() + ig2.getNumberOfInputChannels()); final TestInputChannel[][] inputChannels = new TestInputChannel[][] { @@ -116,15 +113,15 @@ public void testBasicGetNextLogic() throws Exception { verifyBufferOrEvent(union, false, 0, true); // gate 1, channel 0 verifyBufferOrEvent(union, false, 4, true); // gate 1, channel 1 verifyBufferOrEvent(union, false, 1, true); // gate 1, channel 1 - assertEquals( - PullingAsyncDataInput.EndOfDataStatus.NOT_END_OF_DATA, - union.hasReceivedEndOfData()); + assertThat(union.hasReceivedEndOfData()) + .isEqualTo(PullingAsyncDataInput.EndOfDataStatus.NOT_END_OF_DATA); verifyBufferOrEvent(union, false, 5, true); // gate 2, channel 2 verifyBufferOrEvent(union, false, 2, true); // gate 1, channel 2 verifyBufferOrEvent(union, false, 6, true); // gate 2, channel 3 verifyBufferOrEvent(union, false, 7, true); // gate 2, channel 4 - assertEquals(PullingAsyncDataInput.EndOfDataStatus.DRAINED, union.hasReceivedEndOfData()); - assertFalse(union.isFinished()); + assertThat(union.hasReceivedEndOfData()) + .isEqualTo(PullingAsyncDataInput.EndOfDataStatus.DRAINED); + assertThat(union.isFinished()).isFalse(); verifyBufferOrEvent(union, false, 3, true); // gate 2, channel 0 verifyBufferOrEvent(union, false, 4, true); // gate 2, channel 1 verifyBufferOrEvent(union, false, 5, true); // gate 2, channel 2 @@ -132,12 +129,12 @@ public void testBasicGetNextLogic() throws Exception { verifyBufferOrEvent(union, false, 7, false); // gate 2, channel 4 // Return null when the input gate has received all end-of-partition events - assertTrue(union.isFinished()); - assertFalse(union.getNext().isPresent()); + assertThat(union.isFinished()).isTrue(); + assertThat(union.getNext()).isEmpty(); } @Test - public void testDrainFlagComputation() throws Exception { + void testDrainFlagComputation() throws Exception { // Setup final SingleInputGate inputGate1 = createInputGate(); final SingleInputGate inputGate2 = createInputGate(); @@ -170,20 +167,18 @@ public void testDrainFlagComputation() throws Exception { verifyBufferOrEvent(unionInputGate, false, 0, true); verifyBufferOrEvent(unionInputGate, false, 2, true); // we have received EndOfData on a single input only - assertEquals( - PullingAsyncDataInput.EndOfDataStatus.NOT_END_OF_DATA, - unionInputGate.hasReceivedEndOfData()); + assertThat(unionInputGate.hasReceivedEndOfData()) + .isEqualTo(PullingAsyncDataInput.EndOfDataStatus.NOT_END_OF_DATA); verifyBufferOrEvent(unionInputGate, false, 1, true); verifyBufferOrEvent(unionInputGate, false, 3, true); // both channels received EndOfData, one channel said we should not drain - assertEquals( - PullingAsyncDataInput.EndOfDataStatus.STOPPED, - unionInputGate.hasReceivedEndOfData()); + assertThat(unionInputGate.hasReceivedEndOfData()) + .isEqualTo(PullingAsyncDataInput.EndOfDataStatus.STOPPED); } @Test - public void testIsAvailable() throws Exception { + void testIsAvailable() throws Exception { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); inputGate1.setInputChannels(inputChannel1); @@ -196,7 +191,7 @@ public void testIsAvailable() throws Exception { } @Test - public void testAvailability() throws IOException, InterruptedException { + void testAvailability() throws IOException, InterruptedException { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0, false, true); inputGate1.setInputChannels(inputChannel1); @@ -208,15 +203,15 @@ public void testAvailability() throws IOException, InterruptedException { UnionInputGate inputGate = new UnionInputGate(inputGate1, inputGate2); inputChannel1.read(BufferBuilderTestUtils.buildSomeBuffer(1)); - assertTrue(inputGate.getAvailableFuture().isDone()); + assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); inputChannel1.read(BufferBuilderTestUtils.buildSomeBuffer(2)); - assertTrue(inputGate.getAvailableFuture().isDone()); - assertEquals(1, inputGate.getNext().get().getBuffer().getSize()); - assertTrue(inputGate.getAvailableFuture().isDone()); + assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getNext().get().getBuffer().getSize()).isEqualTo(1); + assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); } @Test - public void testIsAvailableAfterFinished() throws Exception { + void testIsAvailableAfterFinished() throws Exception { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); inputGate1.setInputChannels(inputChannel1); @@ -236,7 +231,7 @@ public void testIsAvailableAfterFinished() throws Exception { } @Test - public void testUpdateInputChannel() throws Exception { + void testUpdateInputChannel() throws Exception { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); inputGate1.setInputChannels(inputChannel1); @@ -257,13 +252,13 @@ public void testUpdateInputChannel() throws Exception { location, createRemoteWithIdAndLocation(resultPartitionID.getPartitionId(), location)); - assertThat(unionInputGate.getChannel(0), Matchers.is(inputChannel1)); + assertThat(unionInputGate.getChannel(0)).isEqualTo(inputChannel1); // Check that updated input channel is visible via UnionInputGate - assertThat(unionInputGate.getChannel(1), Matchers.is(inputGate2.getChannel(0))); + assertThat(unionInputGate.getChannel(1)).isEqualTo(inputGate2.getChannel(0)); } @Test - public void testGetChannelWithShiftedGateIndexes() { + void testGetChannelWithShiftedGateIndexes() { gateIndex = 2; final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0); @@ -275,13 +270,13 @@ public void testGetChannelWithShiftedGateIndexes() { UnionInputGate unionInputGate = new UnionInputGate(inputGate1, inputGate2); - assertThat(unionInputGate.getChannel(0), Matchers.is(inputChannel1)); + assertThat(unionInputGate.getChannel(0)).isEqualTo(inputChannel1); // Check that updated input channel is visible via UnionInputGate - assertThat(unionInputGate.getChannel(1), Matchers.is(inputChannel2)); + assertThat(unionInputGate.getChannel(1)).isEqualTo(inputChannel2); } @Test - public void testEmptyPull() throws IOException, InterruptedException { + void testEmptyPull() throws IOException, InterruptedException { final SingleInputGate inputGate1 = createInputGate(1); TestInputChannel inputChannel1 = new TestInputChannel(inputGate1, 0, false, true); inputGate1.setInputChannels(inputChannel1); @@ -293,8 +288,8 @@ public void testEmptyPull() throws IOException, InterruptedException { UnionInputGate inputGate = new UnionInputGate(inputGate1, inputGate2); inputChannel1.notifyChannelNonEmpty(); - assertTrue(inputGate.getAvailableFuture().isDone()); - assertFalse(inputGate.pollNext().isPresent()); - assertFalse(inputGate.getAvailableFuture().isDone()); + assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.pollNext()).isEmpty(); + assertThat(inputGate.getAvailableFuture().isDone()).isFalse(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java index 7b54105ec0d64..c3bf1ad875d8e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java @@ -20,11 +20,9 @@ import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.ReadableRegion; import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.SpilledBuffer; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import java.nio.file.Path; @@ -37,7 +35,6 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link HsFileDataIndexImpl}. */ -@ExtendWith(TestLoggerExtension.class) class HsFileDataIndexImplTest { private static final int NUM_SUBPARTITIONS = 2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java index 202f8e8ba8d7c..b1c7345b33d84 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java @@ -27,14 +27,12 @@ import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil; import org.apache.flink.runtime.io.network.partition.NoOpBufferAvailablityListener; import org.apache.flink.runtime.io.network.partition.ResultSubpartition; -import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.BiConsumerWithException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import java.io.IOException; @@ -60,7 +58,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link HsFileDataManager}. */ -@ExtendWith(TestLoggerExtension.class) class HsFileDataManagerTest { private static final int BUFFER_SIZE = 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpillerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpillerTest.java index 4efd3894a660d..77a8f807ec163 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpillerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataSpillerTest.java @@ -26,11 +26,9 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; import org.apache.flink.runtime.io.network.partition.BufferReaderWriterUtil; import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.SpilledBuffer; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -52,7 +50,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link HsMemoryDataSpiller}. */ -@ExtendWith(TestLoggerExtension.class) class HsMemoryDataSpillerTest { private static final int BUFFER_SIZE = Integer.BYTES; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java index cea52653a70c7..1050d3156af85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java @@ -32,12 +32,10 @@ import org.apache.flink.runtime.io.network.partition.hybrid.HsFileDataIndex.SpilledBuffer; import org.apache.flink.runtime.io.network.partition.hybrid.HsSubpartitionFileReaderImpl.BufferIndexOrError; import org.apache.flink.util.IOUtils; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -66,7 +64,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link HsSubpartitionFileReaderImpl}. */ -@ExtendWith(TestLoggerExtension.class) class HsSubpartitionFileReaderImplTest { private static final int bufferSize = Integer.BYTES; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/DeserializationUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/DeserializationUtils.java index b999be4ecea82..871eabdd5c2e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/DeserializationUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/DeserializationUtils.java @@ -22,10 +22,10 @@ import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; import org.apache.flink.testutils.serialization.types.SerializationTestType; -import org.junit.Assert; - import java.util.ArrayDeque; +import static org.assertj.core.api.Assertions.assertThat; + /** Utility class to help deserialization for testing. */ public final class DeserializationUtils { @@ -49,7 +49,7 @@ public static int deserializeRecords( final DeserializationResult result = deserializer.getNextRecord(actual); if (result.isFullRecord()) { - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); deserializedRecords++; } else { records.addFirst(expected); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java index b828d2503e40d..780c4e500dc4a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestConsumerCallback.java @@ -24,7 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; public interface TestConsumerCallback { @@ -83,7 +83,7 @@ public void onBuffer(Buffer buffer) { int expected = getNumberOfReadBuffers() * (segment.size() / 4); for (int i = 0; i < segment.size(); i += 4) { - assertEquals(expected, segment.getInt(i)); + assertThat(segment.getInt(i)).isEqualTo(expected); expected++; } From 477173b8f66140f9a3abf284d06fdfd5578091c5 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sun, 13 Aug 2023 23:59:14 +0800 Subject: [PATCH 2/7] Address Rui's comments --- .../disk/BatchShuffleReadBufferPoolTest.java | 8 +- .../runtime/io/disk/ChannelViewsTest.java | 11 +- .../io/disk/FileChannelManagerImplTest.java | 2 +- .../io/disk/FileChannelStreamsITCase.java | 25 +- .../SeekableFileChannelInputViewTest.java | 73 ++--- .../runtime/io/disk/SpillingBufferTest.java | 84 +++-- .../AsynchronousBufferFileWriterTest.java | 14 +- .../AsynchronousFileIOChannelTest.java | 6 +- .../io/network/api/CheckpointBarrierTest.java | 22 +- .../api/reader/AbstractReaderTest.java | 54 ++-- .../SpanningRecordSerializationTest.java | 4 +- .../api/writer/RecordWriterDelegateTest.java | 4 +- .../network/buffer/AbstractByteBufTest.java | 303 +++++++++--------- .../buffer/BufferBuilderAndConsumerTest.java | 18 +- ...erConsumerWithPartialRecordLengthTest.java | 8 +- .../network/buffer/BufferPoolFactoryTest.java | 12 +- .../network/buffer/LocalBufferPoolTest.java | 4 +- .../network/buffer/NetworkBufferPoolTest.java | 29 +- .../io/network/buffer/NetworkBufferTest.java | 58 ++-- .../buffer/ReadOnlySlicedBufferTest.java | 32 +- ...asedPartitionRequestClientHandlerTest.java | 8 +- ...tBasedSequenceNumberingViewReaderTest.java | 2 +- .../io/network/netty/NettyBufferPoolTest.java | 2 +- .../netty/PartitionRequestQueueTest.java | 12 +- ...dBlockingSubpartitionAvailabilityTest.java | 4 +- .../BoundedBlockingSubpartitionTest.java | 19 +- .../io/network/partition/DataBufferTest.java | 8 +- .../partition/PipelinedSubpartitionTest.java | 33 +- ...PipelinedSubpartitionWithReadViewTest.java | 100 +++--- .../partition/PrioritizedDequeTest.java | 4 +- .../partition/ResultPartitionFactoryTest.java | 2 +- .../partition/ResultPartitionTest.java | 10 +- ...MergeResultPartitionReadSchedulerTest.java | 4 +- .../SortMergeResultPartitionTest.java | 4 +- .../SortMergeSubpartitionReaderTest.java | 14 +- .../partition/SubpartitionTestBase.java | 28 +- .../consumer/InputBuffersMetricsTest.java | 2 +- .../partition/consumer/InputChannelTest.java | 10 +- .../consumer/LocalInputChannelTest.java | 16 +- .../consumer/RemoteInputChannelTest.java | 95 +++--- .../consumer/SingleInputGateTest.java | 40 +-- .../consumer/UnionInputGateTest.java | 2 +- .../partition/hybrid/HsBufferContextTest.java | 6 +- .../hybrid/HsFileDataIndexImplTest.java | 4 +- .../hybrid/HsFileDataManagerTest.java | 4 +- .../hybrid/HsMemoryDataManagerTest.java | 2 +- .../hybrid/HsResultPartitionTest.java | 4 +- ...artitionConsumerMemoryDataManagerTest.java | 4 +- .../HsSubpartitionFileReaderImplTest.java | 6 +- .../hybrid/HsSubpartitionViewTest.java | 6 +- .../hybrid/index/FileDataIndexCacheTest.java | 12 +- .../netty/NettyConnectionReaderTest.java | 2 +- .../storage/HashBufferAccumulatorTest.java | 3 +- .../TieredStorageMemoryManagerImplTest.java | 20 +- .../TieredStorageProducerClientTest.java | 2 +- 55 files changed, 587 insertions(+), 678 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java index da40914e83163..710cd8a47e47f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java @@ -138,7 +138,7 @@ void testBufferFulfilledByRecycledBuffers() throws Exception { owners[i] = new Object(); buffers.put(owners[i], bufferPool.requestBuffers()); } - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(bufferPool.getAvailableBuffers()).isZero(); Thread[] requestThreads = new Thread[numRequestThreads]; for (int i = 0; i < numRequestThreads; ++i) { @@ -172,7 +172,7 @@ void testBufferFulfilledByRecycledBuffers() throws Exception { } assertThat(exception.get()).isNull(); - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(bufferPool.getAvailableBuffers()).isZero(); assertThat(buffers).hasSize(8); } finally { for (Object owner : buffers.keySet()) { @@ -241,7 +241,7 @@ void testDestroy() throws Exception { bufferPool.destroy(); assertThat(bufferPool.isDestroyed()).isTrue(); - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(bufferPool.getAvailableBuffers()).isZero(); } @Test @@ -260,7 +260,7 @@ void testRecycleAfterDestroyed() throws Exception { bufferPool.destroy(); bufferPool.recycle(buffers); - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(bufferPool.getAvailableBuffers()).isZero(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java index d6efc7a016a67..3cb7d39d41be8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java @@ -44,6 +44,7 @@ import java.util.List; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; /** */ @@ -240,7 +241,7 @@ void testReadTooMany() throws Exception { // read and re-generate all records and compare them try { final Tuple2 readRec = new Tuple2<>(); - for (int i = 0; i < NUM_PAIRS_SHORT + 1; i++) { + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { generator.next(rec); serializer.deserialize(readRec, inView); final int k1 = rec.f0; @@ -251,9 +252,11 @@ void testReadTooMany() throws Exception { .withFailMessage("The re-generated and the read record do not match.") .isTrue(); } - fail("Expected an EOFException which did not occur."); - } catch (EOFException eofex) { - // expected + + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Expected an EOFException which did not occur.") + .isInstanceOf(EOFException.class); } catch (Throwable t) { // unexpected fail("Unexpected Exception: " + t.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java index 6a6d8474c308b..0454e52b89c9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java @@ -134,7 +134,7 @@ private void testDirectoriesCleanupOnKill(boolean callerHasHook) throws Exceptio kill.waitFor(); assertThat(kill.exitValue()) .withFailMessage("Failed to send SIG_TERM to process") - .isEqualTo(0); + .isZero(); Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); while (fileChannelManagerTestProcess.isAlive() && deadline.hasTimeLeft()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java index 0ed8fb18aeed7..a902fe3ea0fbd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java @@ -40,6 +40,7 @@ import java.util.List; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; class FileChannelStreamsITCase { @@ -243,20 +244,20 @@ void testReadTooMany() { generator.reset(); // read and re-generate all records and compare them - try { - Pair readPair = new Pair(); - for (int i = 0; i < NUM_PAIRS_SHORT + 1; i++) { - generator.next(pair); - readPair.read(inView); - assertThat(readPair) - .withFailMessage("The re-generated and the read record do not match.") - .isEqualTo(pair); - } - fail("Expected an EOFException which did not occur."); - } catch (EOFException eofex) { - // expected + Pair readPair = new Pair(); + for (int i = 0; i < NUM_PAIRS_SHORT; i++) { + generator.next(pair); + readPair.read(inView); + assertThat(readPair) + .withFailMessage("The re-generated and the read record do not match.") + .isEqualTo(pair); } + generator.next(pair); + assertThatThrownBy(() -> readPair.read(inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + inView.close(); reader.deleteChannel(); } catch (Exception e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java index d1650057dabd2..2052d7874dba6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java @@ -34,6 +34,7 @@ import java.util.List; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; class SeekableFileChannelInputViewTest { @@ -76,11 +77,9 @@ void testSeek() { for (int i = 0; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the middle of the 3rd page int i = 2 * PAGE_SIZE + PAGE_SIZE / 4; @@ -88,11 +87,9 @@ void testSeek() { for (; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the end i = 120000 - 4; @@ -100,11 +97,9 @@ void testSeek() { for (; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the beginning i = 0; @@ -112,11 +107,9 @@ void testSeek() { for (; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to after a page i = PAGE_SIZE; @@ -124,11 +117,9 @@ void testSeek() { for (; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to after a page i = 3 * PAGE_SIZE; @@ -136,32 +127,24 @@ void testSeek() { for (; i < NUM_RECORDS; i += 4) { assertThat(in.readInt()).isEqualTo(i); } - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek to the end i = NUM_RECORDS; in.seek(i); - try { - in.readInt(); - fail("should throw EOF exception"); - } catch (EOFException ignored) { - } + assertThatThrownBy(in::readInt) + .withFailMessage("should throw EOF exception") + .isInstanceOf(EOFException.class); // seek out of bounds - try { - in.seek(-10); - fail("should throw an exception"); - } catch (IllegalArgumentException ignored) { - } - try { - in.seek(NUM_RECORDS + 1); - fail("should throw an exception"); - } catch (IllegalArgumentException ignored) { - } + assertThatThrownBy(() -> in.seek(-10)) + .withFailMessage("should throw an exception") + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> in.seek(NUM_RECORDS + 1)) + .withFailMessage("should throw an exception") + .isInstanceOf(IllegalArgumentException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java index e096e87e33476..e600da1c864ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java @@ -41,7 +41,7 @@ import java.util.ArrayList; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; class SpillingBufferTest { @@ -188,35 +188,34 @@ void testWriteReadTooMuchInMemory() throws Exception { // notifyNonEmpty and re-generate all records and compare them final Tuple2 readRec = new Tuple2<>(); - try { - for (int i = 0; i < NUM_PAIRS_INMEM + 1; i++) { - generator.next(rec); - serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - assertThat(k1 == k2 && v1.equals(v2)) - .withFailMessage( - "The re-generated and the notifyNonEmpty record do not match.") - .isTrue(); - } - fail("Read too much, expected EOFException."); - } catch (EOFException eofex) { - // expected + for (int i = 0; i < NUM_PAIRS_INMEM; i++) { + generator.next(rec); + serializer.deserialize(readRec, inView); + + int k1 = rec.f0; + String v1 = rec.f1; + + int k2 = readRec.f0; + String v2 = readRec.f1; + + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + // re-notifyNonEmpty the data - inView = outView.flip(); + DataInputView nextInView = outView.flip(); generator.reset(); // notifyNonEmpty and re-generate all records and compare them for (int i = 0; i < NUM_PAIRS_INMEM; i++) { generator.next(rec); - serializer.deserialize(readRec, inView); + serializer.deserialize(readRec, nextInView); int k1 = rec.f0; String v1 = rec.f1; @@ -334,35 +333,34 @@ void testWriteReadTooMuchExternal() throws Exception { // notifyNonEmpty and re-generate all records and compare them final Tuple2 readRec = new Tuple2<>(); - try { - for (int i = 0; i < NUM_PAIRS_EXTERNAL + 1; i++) { - generator.next(rec); - serializer.deserialize(readRec, inView); - - int k1 = rec.f0; - String v1 = rec.f1; - - int k2 = readRec.f0; - String v2 = readRec.f1; - - assertThat(k1 == k2 && v1.equals(v2)) - .withFailMessage( - "The re-generated and the notifyNonEmpty record do not match.") - .isTrue(); - } - fail("Read too much, expected EOFException."); - } catch (EOFException eofex) { - // expected + for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) { + generator.next(rec); + serializer.deserialize(readRec, inView); + + int k1 = rec.f0; + String v1 = rec.f1; + + int k2 = readRec.f0; + String v2 = readRec.f1; + + assertThat(k1 == k2 && v1.equals(v2)) + .withFailMessage("The re-generated and the notifyNonEmpty record do not match.") + .isTrue(); } + generator.next(rec); + assertThatThrownBy(() -> serializer.deserialize(readRec, inView)) + .withFailMessage("Read too much, expected EOFException.") + .isInstanceOf(EOFException.class); + // re-notifyNonEmpty the data - inView = outView.flip(); + DataInputView nextInView = outView.flip(); generator.reset(); // notifyNonEmpty and re-generate all records and compare them for (int i = 0; i < NUM_PAIRS_EXTERNAL; i++) { generator.next(rec); - serializer.deserialize(readRec, inView); + serializer.deserialize(readRec, nextInView); int k1 = rec.f0; String v1 = rec.f1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java index c50d198d39c38..3385f49be4021 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -67,12 +67,12 @@ void testAddAndHandleRequest() throws Exception { addRequest(); assertThat(writer.getNumberOfOutstandingRequests()) .withFailMessage("Didn't increment number of outstanding requests.") - .isEqualTo(1); + .isOne(); handleRequest(); assertThat(writer.getNumberOfOutstandingRequests()) .withFailMessage("Didn't decrement number of outstanding requests.") - .isEqualTo(0); + .isZero(); } @Test @@ -94,7 +94,7 @@ void testAddWithFailingWriter() throws Exception { } assertThat(writer.getNumberOfOutstandingRequests()) .withFailMessage("Shouldn't increment number of outstanding requests.") - .isEqualTo(0); + .isZero(); } @Test @@ -117,7 +117,7 @@ void testSubscribe() throws Exception { assertThat(listener.getNumberOfNotifications()) .withFailMessage("Listener was not notified.") - .isEqualTo(1); + .isOne(); } @Test @@ -157,7 +157,7 @@ public void run() { assertThat(listener.getNumberOfNotifications()) .withFailMessage("Listener was not notified.") - .isEqualTo(1); + .isOne(); } @Test @@ -200,12 +200,12 @@ public Void call() throws Exception { assertThat(listener.getNumberOfNotifications()) .withFailMessage( "Race: Successfully subscribed, but was never notified.") - .isEqualTo(1); + .isOne(); } else { assertThat(listener.getNumberOfNotifications()) .withFailMessage( "Race: Never subscribed successfully, but was notified.") - .isEqualTo(0); + .isZero(); } } catch (Throwable t) { System.out.println(i); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java index 751acb6a80d14..4be054ec474a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java @@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; class AsynchronousFileIOChannelTest { @@ -359,10 +360,7 @@ public void writeBlock(MemorySegment segment) throws IOException { writer.writeBlock(seg); } - writer.close(); - fail("did not forward exception"); - } catch (IOException e) { - // expected + assertThatThrownBy(writer::close).isInstanceOf(IOException.class); } finally { try { writer.closeAndDelete(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java index d704b2833ee3a..9b34ee62a647f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/CheckpointBarrierTest.java @@ -24,7 +24,7 @@ import org.junit.jupiter.api.Test; -import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link CheckpointBarrier} type. */ class CheckpointBarrierTest { @@ -34,25 +34,17 @@ class CheckpointBarrierTest { * serialization, in order to be immutable. */ @Test - void testSerialization() throws Exception { + void testSerialization() { long id = Integer.MAX_VALUE + 123123L; long timestamp = Integer.MAX_VALUE + 1228L; CheckpointOptions options = CheckpointOptions.forCheckpointWithDefaultLocation(); CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp, options); - try { - barrier.write(new DataOutputSerializer(1024)); - fail("should throw an exception"); - } catch (UnsupportedOperationException e) { - // expected - } - - try { - barrier.read(new DataInputDeserializer(new byte[32])); - fail("should throw an exception"); - } catch (UnsupportedOperationException e) { - // expected - } + assertThatThrownBy(() -> barrier.write(new DataOutputSerializer(1024))) + .isInstanceOf(UnsupportedOperationException.class); + + assertThatThrownBy(() -> barrier.read(new DataInputDeserializer(new byte[32]))) + .isInstanceOf(UnsupportedOperationException.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java index 842f82de4625e..8bfdca88d24d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/AbstractReaderTest.java @@ -32,7 +32,7 @@ import java.io.IOException; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -81,23 +81,16 @@ void testExceptionsNonIterativeReader() throws Exception { // Non-iterative reader cannot reach end of superstep assertThat(reader.hasReachedEndOfSuperstep()).isFalse(); - try { - reader.startNextSuperstep(); + assertThatThrownBy(reader::startNextSuperstep) + .withFailMessage( + "Did not throw expected exception when starting next superstep with non-iterative reader.") + .isInstanceOf(IllegalStateException.class); - fail( - "Did not throw expected exception when starting next superstep with non-iterative reader."); - } catch (Throwable t) { - // All good, expected exception. - } - - try { - reader.handleEvent(EndOfSuperstepEvent.INSTANCE); - - fail( - "Did not throw expected exception when handling end of superstep event with non-iterative reader."); - } catch (Throwable t) { - // All good, expected exception. - } + assertThatThrownBy(() -> reader.handleEvent(EndOfSuperstepEvent.INSTANCE)) + .withFailMessage( + "Did not throw expected exception when handling end of superstep event with non-iterative reader.") + .hasCauseInstanceOf(IllegalStateException.class) + .isInstanceOf(IOException.class); } @Test @@ -108,15 +101,11 @@ void testEndOfSuperstepEventLogic() throws IOException { reader.setIterativeReader(); - try { - // The first superstep does not need not to be explicitly started - reader.startNextSuperstep(); - - fail( - "Did not throw expected exception when starting next superstep before receiving all end of superstep events."); - } catch (Throwable t) { - // All good, expected exception. - } + // The first superstep does not need not to be explicitly started + assertThatThrownBy(reader::startNextSuperstep) + .withFailMessage( + "Did not throw expected exception when starting next superstep before receiving all end of superstep events.") + .isInstanceOf(IllegalStateException.class); EndOfSuperstepEvent eos = EndOfSuperstepEvent.INSTANCE; @@ -130,15 +119,10 @@ void testEndOfSuperstepEventLogic() throws IOException { assertThat(reader.handleEvent(eos)).isTrue(); assertThat(reader.hasReachedEndOfSuperstep()).isTrue(); - try { - // Verify exception, when receiving too many end of superstep events. - reader.handleEvent(eos); - - fail( - "Did not throw expected exception when receiving too many end of superstep events."); - } catch (Throwable t) { - // All good, expected exception. - } + assertThatThrownBy(() -> reader.handleEvent(eos)) + .withFailMessage( + "Did not throw expected exception when receiving too many end of superstep events.") + .isInstanceOf(IOException.class); // Start next superstep. reader.startNextSuperstep(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java index 43d86bfdb4576..73ab9d699d3be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java @@ -190,7 +190,7 @@ private static void testSerializationRoundTrip( } // assert that all records have been serialized and deserialized - assertThat(numRecords).isEqualTo(0); + assertThat(numRecords).isZero(); } @Test @@ -307,7 +307,7 @@ private static Buffer appendLeftOverBytes(Buffer buffer, byte[] leftOverBytes) { private static void assertUnconsumedBuffer( ByteArrayOutputStream expected, CloseableIterator actual) throws Exception { if (!actual.hasNext()) { - assertThat(expected.size()).isEqualTo(0); + assertThat(expected.size()).isZero(); } ByteBuffer expectedByteBuffer = ByteBuffer.wrap(expected.toByteArray()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java index e24355217fdd4..c41e6824d49ff 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java @@ -53,7 +53,7 @@ class RecordWriterDelegateTest { @BeforeEach void setup() { - assertThat(memorySegmentSize % recordSize).as("Illegal memory segment size").isEqualTo(0); + assertThat(memorySegmentSize % recordSize).as("Illegal memory segment size").isZero(); globalPool = new NetworkBufferPool(numberOfBuffers, memorySegmentSize); } @@ -169,7 +169,7 @@ private void verifyBroadcastEvent( // verify the added messages in all the queues for (ResultPartition partition : partitions) { for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) { - assertThat(partition.getNumberOfQueuedBuffers(i)).isEqualTo(1); + assertThat(partition.getNumberOfQueuedBuffers(i)).isOne(); ResultSubpartitionView view = partition.createSubpartitionView(i, new NoOpBufferAvailablityListener()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index 2a2599a35df47..9fcffa5361656 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -104,7 +104,7 @@ void init() { void dispose() { if (buffer != null) { assertThat(buffer.release()).isTrue(); - assertThat(buffer.refCnt()).isEqualTo(0); + assertThat(buffer.refCnt()).isZero(); try { buffer.release(); @@ -137,7 +137,7 @@ void comparableInterfaceNotViolated() { @Test void initialState() { assertThat(buffer.capacity()).isEqualTo(CAPACITY); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); } @Test @@ -892,7 +892,7 @@ void testSequentialByteAccess() { buffer.writeByte(value); } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -920,7 +920,7 @@ void testSequentialUnsignedByteAccess() { buffer.writeByte(value); } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -961,7 +961,7 @@ private void testSequentialShortAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -1006,7 +1006,7 @@ private void testSequentialUnsignedShortAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -1051,7 +1051,7 @@ private void testSequentialMediumAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); @@ -1069,7 +1069,7 @@ private void testSequentialMediumAccess(boolean testBigEndian) { assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity() / 3 * 3); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); - assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.readableBytes()).isZero(); assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); } @@ -1096,7 +1096,7 @@ private void testSequentialUnsignedMediumAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); @@ -1114,7 +1114,7 @@ private void testSequentialUnsignedMediumAccess(boolean testBigEndian) { assertThat(buffer.readerIndex()).isEqualTo(buffer.capacity() / 3 * 3); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity() / 3 * 3); - assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.readableBytes()).isZero(); assertThat(buffer.writableBytes()).isEqualTo(buffer.capacity() % 3); } @@ -1141,7 +1141,7 @@ private void testSequentialIntAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -1186,7 +1186,7 @@ private void testSequentialUnsignedIntAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -1231,7 +1231,7 @@ private void testSequentialLongAccess(boolean testBigEndian) { } } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.isWritable()).isFalse(); @@ -1333,7 +1333,7 @@ void testRandomHeapBufferTransfer1() { random.nextBytes(expectedValueContent); value.clear(); buffer.getBytes(i, value); - assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); assertThat(value.writerIndex()).isEqualTo(BLOCK_SIZE); for (int j = 0; j < BLOCK_SIZE; j++) { assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); @@ -1418,7 +1418,7 @@ void testSequentialByteArrayTransfer1() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1442,7 +1442,7 @@ void testSequentialByteArrayTransfer2() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(value); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); buffer.writeBytes(value, readerIndex, BLOCK_SIZE); @@ -1469,10 +1469,10 @@ void testSequentialHeapBufferTransfer1() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); assertThat(value.writerIndex()).isEqualTo(valueContent.length); } @@ -1488,7 +1488,7 @@ void testSequentialHeapBufferTransfer1() { for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertThat(value.readerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); assertThat(value.writerIndex()).isEqualTo(valueContent.length); } } @@ -1500,7 +1500,7 @@ void testSequentialHeapBufferTransfer2() { buffer.writerIndex(0); for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(readerIndex); @@ -1537,11 +1537,11 @@ void testSequentialDirectBufferTransfer1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertThat(value.readerIndex()).isEqualTo(0); - assertThat(value.writerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); + assertThat(value.writerIndex()).isZero(); } random.setSeed(seed); @@ -1557,8 +1557,8 @@ void testSequentialDirectBufferTransfer1() { for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertThat(value.readerIndex()).isEqualTo(0); - assertThat(value.writerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); + assertThat(value.writerIndex()).isZero(); } value.release(); expectedValue.release(); @@ -1572,7 +1572,7 @@ void testSequentialDirectBufferTransfer2() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(0); @@ -1614,11 +1614,11 @@ void testSequentialByteBufferBackedHeapBufferTransfer1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value, random.nextInt(BLOCK_SIZE), BLOCK_SIZE); - assertThat(value.readerIndex()).isEqualTo(0); - assertThat(value.writerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); + assertThat(value.writerIndex()).isZero(); } random.setSeed(seed); @@ -1634,8 +1634,8 @@ void testSequentialByteBufferBackedHeapBufferTransfer1() { for (int j = valueOffset; j < valueOffset + BLOCK_SIZE; j++) { assertThat(value.getByte(j)).isEqualTo(expectedValue.getByte(j)); } - assertThat(value.readerIndex()).isEqualTo(0); - assertThat(value.writerIndex()).isEqualTo(0); + assertThat(value.readerIndex()).isZero(); + assertThat(value.writerIndex()).isZero(); } } @@ -1648,7 +1648,7 @@ void testSequentialByteBufferBackedHeapBufferTransfer2() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { random.nextBytes(valueContent); value.setBytes(0, valueContent); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); int readerIndex = random.nextInt(BLOCK_SIZE); value.readerIndex(0); @@ -1710,7 +1710,7 @@ void testSequentialCopiedBufferTransfer1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { byte[] value = new byte[BLOCK_SIZE]; random.nextBytes(value); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1737,7 +1737,7 @@ void testSequentialSlice1() { for (int i = 0; i < buffer.capacity() - BLOCK_SIZE + 1; i += BLOCK_SIZE) { byte[] value = new byte[BLOCK_SIZE]; random.nextBytes(value); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(i); buffer.writeBytes(value); } @@ -1774,7 +1774,7 @@ void testWriteZero() { i += length; } - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity()); for (int i = 0; i < buffer.capacity(); i++) { @@ -1799,7 +1799,7 @@ void testDiscardReadBytes() { buffer.writerIndex(CAPACITY / 2); buffer.discardReadBytes(); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2); assertThat(buffer.slice(0, CAPACITY / 2)).isEqualTo(copy.slice(0, CAPACITY / 2)); buffer.resetReaderIndex(); @@ -1812,7 +1812,7 @@ void testDiscardReadBytes() { buffer.writerIndex(CAPACITY / 2); buffer.discardReadBytes(); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2 - 1); assertThat(buffer.slice(0, CAPACITY / 2 - 1)).isEqualTo(copy.slice(1, CAPACITY / 2 - 1)); @@ -1851,7 +1851,7 @@ void testDiscardReadBytes2() { // Discard the first (CAPACITY / 2 - 1) bytes. buffer.setIndex(CAPACITY / 2 - 1, CAPACITY - 1); buffer.discardReadBytes(); - assertThat(buffer.readerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); assertThat(buffer.writerIndex()).isEqualTo(CAPACITY / 2); for (int i = 0; i < CAPACITY / 2; i++) { assertThat(buffer.slice(i, CAPACITY / 2 - i)) @@ -1915,7 +1915,7 @@ void testCopy() { // Make sure all properties are copied. ByteBuf copy = buffer.copy(); - assertThat(copy.readerIndex()).isEqualTo(0); + assertThat(copy.readerIndex()).isZero(); assertThat(copy.writerIndex()).isEqualTo(buffer.readableBytes()); assertThat(copy.capacity()).isEqualTo(buffer.readableBytes()); assertThat(copy.order()).isSameAs(buffer.order()); @@ -1946,7 +1946,7 @@ void testDuplicate() { ByteBuf duplicate = buffer.duplicate(); assertThat(duplicate.order()).isSameAs(buffer.order()); assertThat(duplicate.readableBytes()).isEqualTo(buffer.readableBytes()); - assertThat(buffer.compareTo(duplicate)).isEqualTo(0); + assertThat(buffer.compareTo(duplicate)).isZero(); // Make sure the buffer content is shared. buffer.setByte(readerIndex, (byte) (buffer.getByte(readerIndex) + 1)); @@ -1968,10 +1968,10 @@ void testSliceEndianness() throws Exception { @Test void testSliceIndex() throws Exception { - assertThat(buffer.slice(0, buffer.capacity()).readerIndex()).isEqualTo(0); - assertThat(buffer.slice(0, buffer.capacity() - 1).readerIndex()).isEqualTo(0); - assertThat(buffer.slice(1, buffer.capacity() - 1).readerIndex()).isEqualTo(0); - assertThat(buffer.slice(1, buffer.capacity() - 2).readerIndex()).isEqualTo(0); + assertThat(buffer.slice(0, buffer.capacity()).readerIndex()).isZero(); + assertThat(buffer.slice(0, buffer.capacity() - 1).readerIndex()).isZero(); + assertThat(buffer.slice(1, buffer.capacity() - 1).readerIndex()).isZero(); + assertThat(buffer.slice(1, buffer.capacity() - 2).readerIndex()).isZero(); assertThat(buffer.slice(0, buffer.capacity()).writerIndex()).isEqualTo(buffer.capacity()); assertThat(buffer.slice(0, buffer.capacity() - 1).writerIndex()) @@ -1985,19 +1985,19 @@ void testSliceIndex() throws Exception { @Test void testRetainedSliceIndex() throws Exception { ByteBuf retainedSlice = buffer.retainedSlice(0, buffer.capacity()); - assertThat(retainedSlice.readerIndex()).isEqualTo(0); + assertThat(retainedSlice.readerIndex()).isZero(); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, buffer.capacity() - 1); - assertThat(retainedSlice.readerIndex()).isEqualTo(0); + assertThat(retainedSlice.readerIndex()).isZero(); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 1); - assertThat(retainedSlice.readerIndex()).isEqualTo(0); + assertThat(retainedSlice.readerIndex()).isZero(); retainedSlice.release(); retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 2); - assertThat(retainedSlice.readerIndex()).isEqualTo(0); + assertThat(retainedSlice.readerIndex()).isZero(); retainedSlice.release(); retainedSlice = buffer.retainedSlice(0, buffer.capacity()); @@ -2053,8 +2053,8 @@ void testCompareTo() { buffer.setIndex(0, value.length); buffer.setBytes(0, value); - assertThat(buffer.compareTo(wrappedBuffer(value))).isEqualTo(0); - assertThat(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN))).isEqualTo(0); + assertThat(buffer.compareTo(wrappedBuffer(value))).isZero(); + assertThat(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN))).isZero(); value[0]++; assertThat(buffer.compareTo(wrappedBuffer(value)) < 0).isTrue(); @@ -2164,7 +2164,7 @@ void testIndexOf() { assertThat(buffer.indexOf(1, 4, (byte) 1)).isEqualTo(-1); assertThat(buffer.indexOf(4, 1, (byte) 1)).isEqualTo(-1); - assertThat(buffer.indexOf(1, 4, (byte) 2)).isEqualTo(1); + assertThat(buffer.indexOf(1, 4, (byte) 2)).isOne(); assertThat(buffer.indexOf(4, 1, (byte) 2)).isEqualTo(3); } @@ -2251,13 +2251,13 @@ void testHashCode() { assertThat(set.remove(buffer)).isTrue(); assertThat(set.contains(elemA)).isFalse(); - assertThat(set.size()).isEqualTo(1); + assertThat(set.size()).isOne(); buffer.clear(); buffer.writeBytes(elemB.duplicate()); assertThat(set.remove(buffer)).isTrue(); assertThat(set.contains(elemB)).isFalse(); - assertThat(set.size()).isEqualTo(0); + assertThat(set.size()).isZero(); elemA.release(); elemB.release(); elemACopy.release(); @@ -2369,7 +2369,7 @@ void testInternalNioBuffer() { private void testInternalNioBuffer(int a) { ByteBuf buffer = newBuffer(2); ByteBuffer buf = buffer.internalNioBuffer(buffer.readerIndex(), 1); - assertThat(buf.remaining()).isEqualTo(1); + assertThat(buf.remaining()).isOne(); byte[] data = new byte[a]; PlatformDependent.threadLocalRandom().nextBytes(data); @@ -2579,7 +2579,7 @@ void testNioBufferExposeOnlyRegion() { buffer.writeBytes(data); ByteBuffer nioBuf = buffer.nioBuffer(1, data.length - 2); - assertThat(nioBuf.position()).isEqualTo(0); + assertThat(nioBuf.position()).isZero(); assertThat(nioBuf.remaining()).isEqualTo(6); for (int i = 1; nioBuf.hasRemaining(); i++) { @@ -3509,7 +3509,7 @@ private static void assertSliceFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); assertThatThrownBy(buf::slice).isInstanceOf(IllegalReferenceCountException.class); } } @@ -3571,7 +3571,7 @@ private static void assertRetainedSliceFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); assertThatThrownBy(buf::retainedSlice) .isInstanceOf(IllegalReferenceCountException.class); } @@ -3634,7 +3634,7 @@ private static void assertDuplicateFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); assertThatThrownBy(buf::duplicate).isInstanceOf(IllegalReferenceCountException.class); } } @@ -3669,7 +3669,7 @@ private static void assertRetainedDuplicateFailAfterRelease(ByteBuf... bufs) { } } for (ByteBuf buf : bufs) { - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); assertThatThrownBy(buf::retainedDuplicate) .isInstanceOf(IllegalReferenceCountException.class); } @@ -3699,9 +3699,9 @@ void testRetainedDuplicateAfterReleaseRetainedSlice() { @Test void testSliceRelease() { ByteBuf buf = newBuffer(8); - assertThat(buf.refCnt()).isEqualTo(1); + assertThat(buf.refCnt()).isOne(); assertThat(buf.slice().release()).isTrue(); - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); } @Test @@ -3712,7 +3712,7 @@ void testReadSliceOutOfBounds() { @Test void testReadRetainedSliceOutOfBounds() { - assertThatThrownBy(() -> testReadSliceOutOfBounds(false)) + assertThatThrownBy(() -> testReadSliceOutOfBounds(true)) .isInstanceOf(IndexOutOfBoundsException.class); } @@ -3931,8 +3931,8 @@ void testRetainedSliceAndRetainedDuplicateContentIsExpected() { expected4.writeBytes(new byte[] {5, 6, 7}); ByteBuf slice1 = buf.retainedSlice(buf.readerIndex() + 1, 6); - assertThat(slice1.compareTo(expected1)).isEqualTo(0); - assertThat(slice1.compareTo(buf.slice(buf.readerIndex() + 1, 6))).isEqualTo(0); + assertThat(slice1.compareTo(expected1)).isZero(); + assertThat(slice1.compareTo(buf.slice(buf.readerIndex() + 1, 6))).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -3940,21 +3940,21 @@ void testRetainedSliceAndRetainedDuplicateContentIsExpected() { slice1.readByte(); ByteBuf dup1 = slice1.retainedDuplicate(); - assertThat(dup1.compareTo(expected2)).isEqualTo(0); - assertThat(dup1.compareTo(slice1.duplicate())).isEqualTo(0); + assertThat(dup1.compareTo(expected2)).isZero(); + assertThat(dup1.compareTo(slice1.duplicate())).isZero(); // Advance the reader index on dup1. dup1.readByte(); ByteBuf dup2 = dup1.duplicate(); - assertThat(dup2.compareTo(expected3)).isEqualTo(0); + assertThat(dup2.compareTo(expected3)).isZero(); // Advance the reader index on dup2. dup2.readByte(); ByteBuf slice2 = dup2.retainedSlice(dup2.readerIndex(), 3); - assertThat(slice2.compareTo(expected4)).isEqualTo(0); - assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex(), 3))).isEqualTo(0); + assertThat(slice2.compareTo(expected4)).isZero(); + assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex(), 3))).isZero(); // Cleanup the expected buffers used for testing. assertThat(expected1.release()).isTrue(); @@ -3974,11 +3974,11 @@ void testRetainedSliceAndRetainedDuplicateContentIsExpected() { // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(slice1.refCnt()).isEqualTo(0); - assertThat(slice2.refCnt()).isEqualTo(0); - assertThat(dup1.refCnt()).isEqualTo(0); - assertThat(dup2.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(slice1.refCnt()).isZero(); + assertThat(slice2.refCnt()).isZero(); + assertThat(dup1.refCnt()).isZero(); + assertThat(dup2.refCnt()).isZero(); } @Test @@ -3993,8 +3993,8 @@ void testRetainedDuplicateAndRetainedSliceContentIsExpected() { expected3.writeBytes(new byte[] {5, 6, 7}); ByteBuf dup1 = buf.retainedDuplicate(); - assertThat(dup1.compareTo(buf)).isEqualTo(0); - assertThat(dup1.compareTo(buf.slice())).isEqualTo(0); + assertThat(dup1.compareTo(buf)).isZero(); + assertThat(dup1.compareTo(buf.slice())).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -4002,21 +4002,21 @@ void testRetainedDuplicateAndRetainedSliceContentIsExpected() { dup1.readByte(); ByteBuf slice1 = dup1.retainedSlice(dup1.readerIndex(), 6); - assertThat(slice1.compareTo(expected1)).isEqualTo(0); - assertThat(slice1.compareTo(slice1.duplicate())).isEqualTo(0); + assertThat(slice1.compareTo(expected1)).isZero(); + assertThat(slice1.compareTo(slice1.duplicate())).isZero(); // Advance the reader index on slice1. slice1.readByte(); ByteBuf dup2 = slice1.duplicate(); - assertThat(dup2.compareTo(slice1)).isEqualTo(0); + assertThat(dup2.compareTo(slice1)).isZero(); // Advance the reader index on dup2. dup2.readByte(); ByteBuf slice2 = dup2.retainedSlice(dup2.readerIndex() + 1, 3); - assertThat(slice2.compareTo(expected3)).isEqualTo(0); - assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex() + 1, 3))).isEqualTo(0); + assertThat(slice2.compareTo(expected3)).isZero(); + assertThat(slice2.compareTo(dup2.slice(dup2.readerIndex() + 1, 3))).isZero(); // Cleanup the expected buffers used for testing. assertThat(expected1.release()).isTrue(); @@ -4035,11 +4035,11 @@ void testRetainedDuplicateAndRetainedSliceContentIsExpected() { // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(slice1.refCnt()).isEqualTo(0); - assertThat(slice2.refCnt()).isEqualTo(0); - assertThat(dup1.refCnt()).isEqualTo(0); - assertThat(dup2.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(slice1.refCnt()).isZero(); + assertThat(slice2.refCnt()).isZero(); + assertThat(dup1.refCnt()).isZero(); + assertThat(dup2.refCnt()).isZero(); } @Test @@ -4254,8 +4254,8 @@ private void testRetainedSliceUnreleasable( assertThat(buf2.release()).isFalse(); buf1.release(); assertThat(buf.release()).isTrue(); - assertThat(buf1.refCnt()).isEqualTo(0); - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf1.refCnt()).isZero(); + assertThat(buf.refCnt()).isZero(); } private void testReadRetainedSliceUnreleasable( @@ -4271,8 +4271,8 @@ private void testReadRetainedSliceUnreleasable( assertThat(buf2.release()).isFalse(); buf1.release(); assertThat(buf.release()).isTrue(); - assertThat(buf1.refCnt()).isEqualTo(0); - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf1.refCnt()).isZero(); + assertThat(buf.refCnt()).isZero(); } private void testRetainedDuplicateUnreleasable( @@ -4286,8 +4286,8 @@ private void testRetainedDuplicateUnreleasable( assertThat(buf2.release()).isFalse(); buf1.release(); assertThat(buf.release()).isTrue(); - assertThat(buf1.refCnt()).isEqualTo(0); - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf1.refCnt()).isZero(); + assertThat(buf.refCnt()).isZero(); } private void testDuplicateCapacityChange(boolean retainedDuplicate) { @@ -4358,12 +4358,12 @@ private void testSliceContents(boolean retainedSlice) { ? buf.retainedSlice(buf.readerIndex() + 3, 3) : buf.slice(buf.readerIndex() + 3, 3); try { - assertThat(slice.compareTo(expected)).isEqualTo(0); - assertThat(slice.compareTo(slice.duplicate())).isEqualTo(0); + assertThat(slice.compareTo(expected)).isZero(); + assertThat(slice.compareTo(slice.duplicate())).isZero(); ByteBuf b = slice.retainedDuplicate(); - assertThat(slice.compareTo(b)).isEqualTo(0); + assertThat(slice.compareTo(b)).isZero(); b.release(); - assertThat(slice.compareTo(slice.slice(0, slice.capacity()))).isEqualTo(0); + assertThat(slice.compareTo(slice.slice(0, slice.capacity()))).isZero(); } finally { if (retainedSlice) { slice.release(); @@ -4384,7 +4384,7 @@ private void testSliceReleaseOriginal(boolean retainedSlice1, boolean retainedSl retainedSlice1 ? buf.retainedSlice(buf.readerIndex() + 5, 3) : buf.slice(buf.readerIndex() + 5, 3).retain(); - assertThat(slice1.compareTo(expected1)).isEqualTo(0); + assertThat(slice1.compareTo(expected1)).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -4392,7 +4392,7 @@ private void testSliceReleaseOriginal(boolean retainedSlice1, boolean retainedSl retainedSlice2 ? slice1.retainedSlice(slice1.readerIndex() + 1, 2) : slice1.slice(slice1.readerIndex() + 1, 2).retain(); - assertThat(slice2.compareTo(expected2)).isEqualTo(0); + assertThat(slice2.compareTo(expected2)).isZero(); // Cleanup the expected buffers used for testing. assertThat(expected1.release()).isTrue(); @@ -4407,9 +4407,9 @@ private void testSliceReleaseOriginal(boolean retainedSlice1, boolean retainedSl // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(slice1.refCnt()).isEqualTo(0); - assertThat(slice2.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(slice1.refCnt()).isZero(); + assertThat(slice2.refCnt()).isZero(); } private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boolean doSlice2) { @@ -4427,36 +4427,36 @@ private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boo expected4DupSlice.writeBytes(new byte[] {4}); ByteBuf slice1 = buf.retainedSlice(buf.readerIndex() + 1, 6); - assertThat(slice1.compareTo(expected1)).isEqualTo(0); + assertThat(slice1.compareTo(expected1)).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf slice2 = slice1.retainedSlice(slice1.readerIndex() + 1, 4); - assertThat(slice2.compareTo(expected2)).isEqualTo(0); - assertThat(slice2.compareTo(slice2.duplicate())).isEqualTo(0); - assertThat(slice2.compareTo(slice2.slice())).isEqualTo(0); + assertThat(slice2.compareTo(expected2)).isZero(); + assertThat(slice2.compareTo(slice2.duplicate())).isZero(); + assertThat(slice2.compareTo(slice2.slice())).isZero(); ByteBuf tmpBuf = slice2.retainedDuplicate(); - assertThat(slice2.compareTo(tmpBuf)).isEqualTo(0); + assertThat(slice2.compareTo(tmpBuf)).isZero(); tmpBuf.release(); tmpBuf = slice2.retainedSlice(); - assertThat(slice2.compareTo(tmpBuf)).isEqualTo(0); + assertThat(slice2.compareTo(tmpBuf)).isZero(); tmpBuf.release(); ByteBuf slice3 = doSlice1 ? slice2.slice(slice2.readerIndex() + 1, 2) : slice2.duplicate(); if (doSlice1) { - assertThat(slice3.compareTo(expected3)).isEqualTo(0); + assertThat(slice3.compareTo(expected3)).isZero(); } else { - assertThat(slice3.compareTo(expected2)).isEqualTo(0); + assertThat(slice3.compareTo(expected2)).isZero(); } ByteBuf slice4 = doSlice2 ? slice3.slice(slice3.readerIndex() + 1, 1) : slice3.duplicate(); if (doSlice1 && doSlice2) { - assertThat(slice4.compareTo(expected4SliceSlice)).isEqualTo(0); + assertThat(slice4.compareTo(expected4SliceSlice)).isZero(); } else if (doSlice2) { - assertThat(slice4.compareTo(expected4DupSlice)).isEqualTo(0); + assertThat(slice4.compareTo(expected4DupSlice)).isZero(); } else { - assertThat(slice3.compareTo(slice4)).isEqualTo(0); + assertThat(slice3.compareTo(slice4)).isZero(); } // Cleanup the expected buffers used for testing. @@ -4477,10 +4477,10 @@ private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boo // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(slice1.refCnt()).isEqualTo(0); - assertThat(slice2.refCnt()).isEqualTo(0); - assertThat(slice3.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(slice1.refCnt()).isZero(); + assertThat(slice2.refCnt()).isZero(); + assertThat(slice3.refCnt()).isZero(); } private void testDuplicateReleaseOriginal( @@ -4490,12 +4490,12 @@ private void testDuplicateReleaseOriginal( buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); expected.writeBytes(buf, buf.readerIndex(), buf.readableBytes()); ByteBuf dup1 = retainedDuplicate1 ? buf.retainedDuplicate() : buf.duplicate().retain(); - assertThat(dup1.compareTo(expected)).isEqualTo(0); + assertThat(dup1.compareTo(expected)).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf dup2 = retainedDuplicate2 ? dup1.retainedDuplicate() : dup1.duplicate().retain(); - assertThat(dup2.compareTo(expected)).isEqualTo(0); + assertThat(dup2.compareTo(expected)).isZero(); // Cleanup the expected buffers used for testing. assertThat(expected.release()).isTrue(); @@ -4509,9 +4509,9 @@ private void testDuplicateReleaseOriginal( // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(dup1.refCnt()).isEqualTo(0); - assertThat(dup2.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(dup1.refCnt()).isZero(); + assertThat(dup2.refCnt()).isZero(); } private void testMultipleRetainedSliceReleaseOriginal( @@ -4528,7 +4528,7 @@ private void testMultipleRetainedSliceReleaseOriginal( retainedSlice1 ? buf.retainedSlice(buf.readerIndex() + 5, 3) : buf.slice(buf.readerIndex() + 5, 3).retain(); - assertThat(slice1.compareTo(expected1)).isEqualTo(0); + assertThat(slice1.compareTo(expected1)).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); @@ -4536,13 +4536,13 @@ private void testMultipleRetainedSliceReleaseOriginal( retainedSlice2 ? slice1.retainedSlice(slice1.readerIndex() + 1, 2) : slice1.slice(slice1.readerIndex() + 1, 2).retain(); - assertThat(slice2.compareTo(expected2)).isEqualTo(0); + assertThat(slice2.compareTo(expected2)).isZero(); // The handler created a slice of the slice and is now done with it. slice2.release(); ByteBuf slice3 = slice1.retainedSlice(slice1.readerIndex(), 2); - assertThat(slice3.compareTo(expected3)).isEqualTo(0); + assertThat(slice3.compareTo(expected3)).isZero(); // The handler created another slice of the slice and is now done with it. slice3.release(); @@ -4558,10 +4558,10 @@ private void testMultipleRetainedSliceReleaseOriginal( // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(slice1.refCnt()).isEqualTo(0); - assertThat(slice2.refCnt()).isEqualTo(0); - assertThat(slice3.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(slice1.refCnt()).isZero(); + assertThat(slice2.refCnt()).isZero(); + assertThat(slice3.refCnt()).isZero(); } private void testMultipleRetainedDuplicateReleaseOriginal( @@ -4571,27 +4571,27 @@ private void testMultipleRetainedDuplicateReleaseOriginal( buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); expected.writeBytes(buf, buf.readerIndex(), buf.readableBytes()); ByteBuf dup1 = retainedDuplicate1 ? buf.retainedDuplicate() : buf.duplicate().retain(); - assertThat(dup1.compareTo(expected)).isEqualTo(0); + assertThat(dup1.compareTo(expected)).isZero(); // Simulate a handler that releases the original buffer, and propagates a slice. buf.release(); ByteBuf dup2 = retainedDuplicate2 ? dup1.retainedDuplicate() : dup1.duplicate().retain(); - assertThat(dup2.compareTo(expected)).isEqualTo(0); - assertThat(dup2.compareTo(dup2.duplicate())).isEqualTo(0); - assertThat(dup2.compareTo(dup2.slice())).isEqualTo(0); + assertThat(dup2.compareTo(expected)).isZero(); + assertThat(dup2.compareTo(dup2.duplicate())).isZero(); + assertThat(dup2.compareTo(dup2.slice())).isZero(); ByteBuf tmpBuf = dup2.retainedDuplicate(); - assertThat(dup2.compareTo(tmpBuf)).isEqualTo(0); + assertThat(dup2.compareTo(tmpBuf)).isZero(); tmpBuf.release(); tmpBuf = dup2.retainedSlice(); - assertThat(dup2.compareTo(tmpBuf)).isEqualTo(0); + assertThat(dup2.compareTo(tmpBuf)).isZero(); tmpBuf.release(); // The handler created a slice of the slice and is now done with it. dup2.release(); ByteBuf dup3 = dup1.retainedDuplicate(); - assertThat(dup3.compareTo(expected)).isEqualTo(0); + assertThat(dup3.compareTo(expected)).isZero(); // The handler created another slice of the slice and is now done with it. dup3.release(); @@ -4605,10 +4605,10 @@ private void testMultipleRetainedDuplicateReleaseOriginal( // Reference counting may be shared, or may be independently tracked, but at this point all // buffers should // be deallocated and have a reference count of 0. - assertThat(buf.refCnt()).isEqualTo(0); - assertThat(dup1.refCnt()).isEqualTo(0); - assertThat(dup2.refCnt()).isEqualTo(0); - assertThat(dup3.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); + assertThat(dup1.refCnt()).isZero(); + assertThat(dup2.refCnt()).isZero(); + assertThat(dup3.refCnt()).isZero(); } private void testDuplicateContents(boolean retainedDuplicate) { @@ -4616,13 +4616,12 @@ private void testDuplicateContents(boolean retainedDuplicate) { buf.writeBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8}); ByteBuf dup = retainedDuplicate ? buf.retainedDuplicate() : buf.duplicate(); try { - assertThat(dup.compareTo(buf)).isEqualTo(0); - assertThat(dup.compareTo(dup.duplicate())).isEqualTo(0); + assertThat(dup.compareTo(buf)).isZero(); + assertThat(dup.compareTo(dup.duplicate())).isZero(); ByteBuf b = dup.retainedDuplicate(); - assertThat(dup.compareTo(b)).isEqualTo(0); + assertThat(dup.compareTo(b)).isZero(); b.release(); - assertThat(dup.compareTo(dup.slice(dup.readerIndex(), dup.readableBytes()))) - .isEqualTo(0); + assertThat(dup.compareTo(dup.slice(dup.readerIndex(), dup.readableBytes()))).isZero(); } finally { if (retainedDuplicate) { dup.release(); @@ -4634,9 +4633,9 @@ private void testDuplicateContents(boolean retainedDuplicate) { @Test void testDuplicateRelease() { ByteBuf buf = newBuffer(8); - assertThat(buf.refCnt()).isEqualTo(1); + assertThat(buf.refCnt()).isOne(); assertThat(buf.duplicate().release()).isTrue(); - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); } // Test-case trying to reproduce: @@ -4659,7 +4658,7 @@ void testEmptyNioBuffers() throws Exception { buffer.clear(); assertThat(buffer.isReadable()).isFalse(); ByteBuffer[] nioBuffers = buffer.nioBuffers(); - assertThat(nioBuffers.length).isEqualTo(1); + assertThat(nioBuffers.length).isOne(); assertThat(nioBuffers[0].hasRemaining()).isFalse(); buffer.release(); } @@ -4687,7 +4686,7 @@ private void testGetReadOnlyDst(boolean direct) { ByteBuffer readOnlyDst = dst.asReadOnlyBuffer(); assertThatThrownBy(() -> buffer.getBytes(0, readOnlyDst)) .isInstanceOf(ReadOnlyBufferException.class); - assertThat(readOnlyDst.position()).isEqualTo(0); + assertThat(readOnlyDst.position()).isZero(); buffer.release(); } @@ -4789,9 +4788,9 @@ void testReadBytes() { assertThat(buffer2.alloc()).isSameAs(buffer.alloc()); assertThat(buffer.readerIndex()).isEqualTo(4); assertThat(buffer.release()).isTrue(); - assertThat(buffer.refCnt()).isEqualTo(0); + assertThat(buffer.refCnt()).isZero(); assertThat(buffer2.release()).isTrue(); - assertThat(buffer2.refCnt()).isEqualTo(0); + assertThat(buffer2.refCnt()).isZero(); } @Test @@ -4865,7 +4864,7 @@ private void testRefCnt0(final boolean parameter) throws Exception { final CountDownLatch innerLatch = new CountDownLatch(1); final ByteBuf buffer = newBuffer(4); - assertThat(buffer.refCnt()).isEqualTo(1); + assertThat(buffer.refCnt()).isOne(); final AtomicInteger cnt = new AtomicInteger(Integer.MAX_VALUE); Thread t1 = new Thread( @@ -4901,7 +4900,7 @@ public void run() { t1.start(); latch.await(); - assertThat(cnt.get()).isEqualTo(0); + assertThat(cnt.get()).isZero(); innerLatch.countDown(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java index 2a83b32a9c518..2df4de9ed94b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java @@ -157,7 +157,7 @@ void copy() { void buildEmptyBuffer() { try (BufferBuilder bufferBuilder = createBufferBuilder()) { Buffer buffer = buildSingleBuffer(bufferBuilder); - assertThat(buffer.getSize()).isEqualTo(0); + assertThat(buffer.getSize()).isZero(); assertContent(buffer, FreeingBufferRecycler.INSTANCE); } } @@ -222,7 +222,7 @@ void testWritableBytes() { void testWritableBytesWhenFull() { BufferBuilder bufferBuilder = createBufferBuilder(); bufferBuilder.append(toByteBuffer(new int[bufferBuilder.getMaxCapacity()])); - assertThat(bufferBuilder.getWritableBytes()).isEqualTo(0); + assertThat(bufferBuilder.getWritableBytes()).isZero(); } @Test @@ -236,7 +236,7 @@ void recycleWithoutConsumer() { bufferBuilder.close(); // then: Recycling successfully finished. - assertThat(recycler.recycleInvocationCounter).isEqualTo(1); + assertThat(recycler.recycleInvocationCounter).isOne(); } @Test @@ -253,13 +253,13 @@ void recycleConsumerAndBufferBuilder() { bufferBuilder.close(); // then: Nothing happened because BufferBuilder has already consumer. - assertThat(recycler.recycleInvocationCounter).isEqualTo(0); + assertThat(recycler.recycleInvocationCounter).isZero(); // when: Close the consumer. bufferConsumer.close(); // then: Recycling successfully finished. - assertThat(recycler.recycleInvocationCounter).isEqualTo(1); + assertThat(recycler.recycleInvocationCounter).isOne(); } @Test @@ -271,7 +271,7 @@ void trimToAvailableSize() { assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE / 2); bufferBuilder.trim(0); - assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(0); + assertThat(bufferBuilder.getMaxCapacity()).isZero(); } @Test @@ -280,7 +280,7 @@ void trimToNegativeSize() { assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE); bufferBuilder.trim(-1); - assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(0); + assertThat(bufferBuilder.getMaxCapacity()).isZero(); } @Test @@ -315,7 +315,7 @@ private static void testIsFinished(int writes) { assertThat(bufferBuilder.isFinished()).isFalse(); assertThat(bufferConsumer.isFinished()).isFalse(); - assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(0); + assertThat(bufferConsumer.getWrittenBytes()).isZero(); bufferConsumer.build(); assertThat(bufferBuilder.isFinished()).isFalse(); @@ -334,7 +334,7 @@ private static void testIsFinished(int writes) { assertThat(bufferConsumer.isFinished()).isFalse(); assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(expectedWrittenBytes); - assertThat(bufferConsumer.build().getSize()).isEqualTo(0); + assertThat(bufferConsumer.build().getSize()).isZero(); assertThat(bufferBuilder.isFinished()).isTrue(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java index 190643f411825..e9382b5eaf071 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferConsumerWithPartialRecordLengthTest.java @@ -54,7 +54,7 @@ void partialRecordTestCase() { // buffer starts with a full record BufferConsumerWithPartialRecordLength consumer1 = buffers.poll(); - assertThat(requireNonNull(consumer1).getPartialRecordLength()).isEqualTo(0); + assertThat(requireNonNull(consumer1).getPartialRecordLength()).isZero(); assertThat(consumer1.cleanupPartialRecord()).isTrue(); assertContent(consumer1.build(), FreeingBufferRecycler.INSTANCE, 0, 1, 2, 3); @@ -62,7 +62,7 @@ void partialRecordTestCase() { // skip the partial record, return an empty buffer BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); assertThat(requireNonNull(consumer2).cleanupPartialRecord()).isTrue(); - assertThat(consumer2.build().readableBytes()).isEqualTo(0); + assertThat(consumer2.build().readableBytes()).isZero(); } @Test @@ -77,7 +77,7 @@ void partialLongRecordSpanningBufferTestCase() { BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); assertThat(requireNonNull(consumer2).getPartialRecordLength()).isEqualTo(BUFFER_SIZE); assertThat(consumer2.cleanupPartialRecord()).isFalse(); - assertThat(consumer2.build().readableBytes()).isEqualTo(0); + assertThat(consumer2.build().readableBytes()).isZero(); BufferConsumerWithPartialRecordLength consumer3 = buffers.poll(); assertThat(requireNonNull(consumer3).cleanupPartialRecord()).isTrue(); @@ -96,7 +96,7 @@ void partialLongRecordEndsWithFullBufferTestCase() { BufferConsumerWithPartialRecordLength consumer2 = buffers.poll(); assertThat(requireNonNull(consumer2).getPartialRecordLength()).isEqualTo(BUFFER_SIZE); assertThat(consumer2.cleanupPartialRecord()).isFalse(); - assertThat(consumer2.build().readableBytes()).isEqualTo(0); + assertThat(consumer2.build().readableBytes()).isZero(); BufferConsumerWithPartialRecordLength consumer3 = buffers.poll(); assertThat(requireNonNull(consumer3).cleanupPartialRecord()).isTrue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java index 0152476ce2e32..a71e2f82dce45 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java @@ -149,7 +149,7 @@ void testOverprovisioned() throws IOException { buffers.remove(0).recycleBuffer(); // recycle returns the excess buffer to the network buffer pool from where it's eagerly // fetched by pool 2 - assertThat(networkBufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(networkBufferPool.getNumberOfAvailableMemorySegments()).isZero(); // verify the number of buffers taken from the pools assertThat( bufferPool1.bestEffortGetNumOfUsedBuffers() @@ -175,7 +175,7 @@ void testOverprovisioned() throws IOException { @Test void testBoundedPools() throws IOException { BufferPool bufferPool1 = networkBufferPool.createBufferPool(1, 1); - assertThat(bufferPool1.getNumBuffers()).isEqualTo(1); + assertThat(bufferPool1.getNumBuffers()).isOne(); BufferPool bufferPool2 = networkBufferPool.createBufferPool(1, 2); assertThat(bufferPool2.getNumBuffers()).isEqualTo(2); @@ -297,15 +297,15 @@ void testUniformDistributionBounded3() throws IOException { assertThat(second.getNumBuffers()).isNotEqualTo(3); BufferPool third = globalPool.createBufferPool(1, 10); - assertThat(first.getNumBuffers()).isEqualTo(1); - assertThat(second.getNumBuffers()).isEqualTo(1); - assertThat(third.getNumBuffers()).isEqualTo(1); + assertThat(first.getNumBuffers()).isOne(); + assertThat(second.getNumBuffers()).isOne(); + assertThat(third.getNumBuffers()).isOne(); // similar to #verifyAllBuffersReturned() String msg = "Wrong number of available segments after creating buffer pools."; assertThat(globalPool.getNumberOfAvailableMemorySegments()) .withFailMessage(msg) - .isEqualTo(0); + .isZero(); } finally { // in case buffers have actually been requested, we must release them again globalPool.destroyAllBufferPools(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 8aaa2163bdf44..c161a6fff3758 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -73,7 +73,7 @@ void setupLocalBufferPool() { networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize); localBufferPool = new LocalBufferPool(networkBufferPool, 1); - assertThat(localBufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(1); + assertThat(localBufferPool.getNumberOfAvailableMemorySegments()).isOne(); } @AfterEach @@ -116,7 +116,7 @@ void testReserveSegments() throws Exception { bufferPool2.lazyDestroy(); BufferPool bufferPool3 = networkBufferPool.createBufferPool(2, 10); - assertThat(bufferPool3.getNumberOfAvailableMemorySegments()).isEqualTo(1); + assertThat(bufferPool3.getNumberOfAvailableMemorySegments()).isOne(); bufferPool3.reserveSegments(2); assertThat(bufferPool3.getNumberOfAvailableMemorySegments()).isEqualTo(2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 4de1f97fc230a..0590c3cd56739 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -56,7 +56,7 @@ void testCreatePoolAfterDestroy() { final int numBuffers = 10; NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize); - assertThat(globalPool.getNumberOfRegisteredBufferPools()).isEqualTo(0); + assertThat(globalPool.getNumberOfRegisteredBufferPools()).isZero(); globalPool.destroy(); @@ -88,7 +88,7 @@ void testMemoryUsageInTheContextOfMemoryPoolCreation() { assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(numBuffers); assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers); - assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isZero(); assertThat(globalPool.getTotalMemory()).isEqualTo((long) numBuffers * bufferSize); assertThat(globalPool.getAvailableMemory()).isEqualTo((long) numBuffers * bufferSize); @@ -107,7 +107,7 @@ void testMemoryUsageInTheContextOfMemorySegmentAllocation() { assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(numBuffers); assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers - 1); - assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(1); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isOne(); assertThat(globalPool.getTotalMemory()).isEqualTo((long) numBuffers * bufferSize); assertThat(globalPool.getAvailableMemory()).isEqualTo((long) (numBuffers - 1) * bufferSize); @@ -123,9 +123,9 @@ void testMemoryUsageInTheContextOfMemoryPoolDestruction() { globalPool.destroy(); - assertThat(globalPool.getTotalNumberOfMemorySegments()).isEqualTo(0); - assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); - assertThat(globalPool.getNumberOfUsedMemorySegments()).isEqualTo(0); + assertThat(globalPool.getTotalNumberOfMemorySegments()).isZero(); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero(); + assertThat(globalPool.getNumberOfUsedMemorySegments()).isZero(); assertThat(globalPool.getTotalMemory()).isEqualTo(0L); assertThat(globalPool.getAvailableMemory()).isEqualTo(0L); @@ -141,7 +141,7 @@ void testDestroyAll() throws IOException { BufferPool nonFixedPool = globalPool.createBufferPool(5, Integer.MAX_VALUE); assertThat(fixedPool.getNumberOfRequiredMemorySegments()).isEqualTo(2); - assertThat(boundedPool.getNumberOfRequiredMemorySegments()).isEqualTo(1); + assertThat(boundedPool.getNumberOfRequiredMemorySegments()).isOne(); assertThat(nonFixedPool.getNumberOfRequiredMemorySegments()).isEqualTo(5); // actually, the buffer pool sizes may be different due to rounding and based on the @@ -178,10 +178,10 @@ void testDestroyAll() throws IOException { assertThat(boundedPool.isDestroyed()).isTrue(); assertThat(nonFixedPool.isDestroyed()).isTrue(); - assertThat(globalPool.getNumberOfRegisteredBufferPools()).isEqualTo(0); + assertThat(globalPool.getNumberOfRegisteredBufferPools()).isZero(); // buffers are not yet recycled - assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero(); // the recycled buffers should go to the global pool for (Buffer b : buffers) { @@ -242,9 +242,8 @@ void testRequestMemorySegmentsMoreThanTotalBuffers() { NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128); try { - globalPool.requestUnpooledMemorySegments(numBuffers + 1); - fail("Should throw an IOException"); - } catch (IOException e) { + assertThatThrownBy(() -> globalPool.requestUnpooledMemorySegments(numBuffers + 1)) + .isInstanceOf(IOException.class); assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers); } finally { globalPool.destroy(); @@ -295,7 +294,7 @@ void testEmptyPoolSegmentsUsage() throws IOException { try (CloseableRegistry closeableRegistry = new CloseableRegistry()) { NetworkBufferPool globalPool = new NetworkBufferPool(0, 128); closeableRegistry.registerCloseable(globalPool::destroy); - assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(0); + assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isZero(); } } @@ -537,7 +536,7 @@ void testRequestMemorySegmentsTimeout() throws Exception { localBufferPool.requestBuffer(); } - assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero(); CheckedThread asyncRequest = new CheckedThread() { @@ -742,7 +741,7 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt latch.await(); assertThat(cause.get()).isNull(); - assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero(); assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); for (BufferPool localPool : localBufferPools) { assertThat(localPool.getAvailableFuture().isDone()).isFalse(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java index 36b3dbb1f5511..38071903547a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java @@ -80,8 +80,8 @@ private static NetworkBuffer newBuffer( buffer.setAllocator(NETTY_BUFFER_POOL); assertThat(buffer.order()).isSameAs(ByteOrder.BIG_ENDIAN); - assertThat(buffer.readerIndex()).isEqualTo(0); - assertThat(buffer.writerIndex()).isEqualTo(0); + assertThat(buffer.readerIndex()).isZero(); + assertThat(buffer.writerIndex()).isZero(); return buffer; } @@ -165,7 +165,7 @@ private static void testRecycleBuffer(boolean isBuffer) { assertThat(buffer.isRecycled()).isFalse(); buffer.recycleBuffer(); assertThat(buffer.isRecycled()).isTrue(); - assertThat(buffer.refCnt()).isEqualTo(0); + assertThat(buffer.refCnt()).isZero(); } @Test @@ -205,14 +205,14 @@ private static void testCreateSlice1(boolean isBuffer) { buffer.setSize(10); // fake some data ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isZero(); assertThat(slice.getSize()).isEqualTo(10); assertThat(slice.unwrap().unwrap()).isSameAs(buffer); // slice indices should be independent: buffer.setSize(8); buffer.setReaderIndex(2); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isZero(); assertThat(slice.getSize()).isEqualTo(10); } @@ -231,14 +231,14 @@ private static void testCreateSlice2(boolean isBuffer) { buffer.setSize(2); // fake some data ReadOnlySlicedNetworkBuffer slice = buffer.readOnlySlice(1, 10); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isZero(); assertThat(slice.getSize()).isEqualTo(10); assertThat(slice.unwrap().unwrap()).isSameAs(buffer); // slice indices should be independent: buffer.setSize(8); buffer.setReaderIndex(2); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(slice.getReaderIndex()).isZero(); assertThat(slice.getSize()).isEqualTo(10); } @@ -276,13 +276,13 @@ void testEventBufferGetSetReaderIndex() { */ private static void testGetSetReaderIndex(boolean isBuffer) { NetworkBuffer buffer = newBuffer(100, 1024, isBuffer); - assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); // fake some data buffer.setSize(100); - assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); buffer.setReaderIndex(1); - assertThat(buffer.getReaderIndex()).isEqualTo(1); + assertThat(buffer.getReaderIndex()).isOne(); } @Test @@ -298,14 +298,14 @@ void testEventBufferSetGetSize() { private static void testSetGetSize(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertThat(buffer.getSize()).isEqualTo(0); // initially 0 + assertThat(buffer.getSize()).isZero(); // initially 0 assertThat(buffer.writerIndex()).isEqualTo(buffer.getSize()); - assertThat(buffer.readerIndex()).isEqualTo(0); // initially 0 + assertThat(buffer.readerIndex()).isZero(); // initially 0 buffer.setSize(10); assertThat(buffer.getSize()).isEqualTo(10); assertThat(buffer.writerIndex()).isEqualTo(buffer.getSize()); - assertThat(buffer.readerIndex()).isEqualTo(0); // independent + assertThat(buffer.readerIndex()).isZero(); // independent } @Test @@ -321,13 +321,13 @@ void testEventBufferReadableBytes() { private static void testReadableBytes(boolean isBuffer) { NetworkBuffer buffer = newBuffer(1024, 1024, isBuffer); - assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.readableBytes()).isZero(); buffer.setSize(10); assertThat(buffer.readableBytes()).isEqualTo(10); buffer.setReaderIndex(2); assertThat(buffer.readableBytes()).isEqualTo(8); buffer.setReaderIndex(10); - assertThat(buffer.readableBytes()).isEqualTo(0); + assertThat(buffer.readableBytes()).isZero(); } @Test @@ -345,16 +345,16 @@ private void testGetNioBufferReadable(boolean isBuffer) { ByteBuffer byteBuffer = buffer.getNioBufferReadable(); assertThat(byteBuffer.isReadOnly()).isFalse(); - assertThat(byteBuffer.remaining()).isEqualTo(0); - assertThat(byteBuffer.limit()).isEqualTo(0); - assertThat(byteBuffer.capacity()).isEqualTo(0); + assertThat(byteBuffer.remaining()).isZero(); + assertThat(byteBuffer.limit()).isZero(); + assertThat(byteBuffer.capacity()).isZero(); // add some data buffer.setSize(10); // nothing changes in the byteBuffer - assertThat(byteBuffer.remaining()).isEqualTo(0); - assertThat(byteBuffer.limit()).isEqualTo(0); - assertThat(byteBuffer.capacity()).isEqualTo(0); + assertThat(byteBuffer.remaining()).isZero(); + assertThat(byteBuffer.limit()).isZero(); + assertThat(byteBuffer.capacity()).isZero(); // get a new byteBuffer (should have updated indices) byteBuffer = buffer.getNioBufferReadable(); assertThat(byteBuffer.isReadOnly()).isFalse(); @@ -364,7 +364,7 @@ private void testGetNioBufferReadable(boolean isBuffer) { // modify byteBuffer position and verify nothing has changed in the original buffer byteBuffer.position(1); - assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); assertThat(buffer.getSize()).isEqualTo(10); } @@ -401,16 +401,16 @@ private void testGetNioBuffer(boolean isBuffer) { ByteBuffer byteBuffer = buffer.getNioBuffer(1, 1); assertThat(byteBuffer.isReadOnly()).isFalse(); - assertThat(byteBuffer.remaining()).isEqualTo(1); - assertThat(byteBuffer.limit()).isEqualTo(1); - assertThat(byteBuffer.capacity()).isEqualTo(1); + assertThat(byteBuffer.remaining()).isOne(); + assertThat(byteBuffer.limit()).isOne(); + assertThat(byteBuffer.capacity()).isOne(); // add some data buffer.setSize(10); // nothing changes in the byteBuffer - assertThat(byteBuffer.remaining()).isEqualTo(1); - assertThat(byteBuffer.limit()).isEqualTo(1); - assertThat(byteBuffer.capacity()).isEqualTo(1); + assertThat(byteBuffer.remaining()).isOne(); + assertThat(byteBuffer.limit()).isOne(); + assertThat(byteBuffer.capacity()).isOne(); // get a new byteBuffer (should have updated indices) byteBuffer = buffer.getNioBuffer(1, 2); assertThat(byteBuffer.isReadOnly()).isFalse(); @@ -420,7 +420,7 @@ private void testGetNioBuffer(boolean isBuffer) { // modify byteBuffer position and verify nothing has changed in the original buffer byteBuffer.position(1); - assertThat(buffer.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); assertThat(buffer.getSize()).isEqualTo(10); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java index bcb21492ae6ff..1f03949748ea3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/ReadOnlySlicedBufferTest.java @@ -148,7 +148,7 @@ void testCreateSlice1() { ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(); assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); assertThat(slice2.getMemorySegment()).isEqualTo(slice1.getMemorySegment()); - assertThat(slice2.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice2.getMemorySegmentOffset()).isOne(); assertThat(slice2.getMemorySegmentOffset()).isEqualTo(slice1.getMemorySegmentOffset()); assertReadableBytes(slice1, 1, 2, 3, 4, 5, 6, 7, 8, 9); @@ -163,7 +163,7 @@ void testCreateSlice2() { ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(1, 2); assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); assertThat(slice2.getMemorySegment()).isEqualTo(slice1.getMemorySegment()); - assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice1.getMemorySegmentOffset()).isOne(); assertThat(slice2.getMemorySegmentOffset()).isEqualTo(2); assertReadableBytes(slice1, 1, 2, 3, 4, 5, 6, 7, 8, 9); @@ -177,8 +177,8 @@ void testCreateSlice3() { ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(); assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); assertThat(slice2.getMemorySegment()).isSameAs(slice1.getMemorySegment()); - assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); - assertThat(slice2.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice1.getMemorySegmentOffset()).isOne(); + assertThat(slice2.getMemorySegmentOffset()).isOne(); assertReadableBytes(slice1, 1, 2); assertReadableBytes(slice2, 1, 2); @@ -191,7 +191,7 @@ void testCreateSlice4() { ReadOnlySlicedNetworkBuffer slice2 = slice1.readOnlySlice(1, 2); assertThat(slice2.unwrap().unwrap()).isSameAs(buffer); assertThat(slice2.getMemorySegment()).isSameAs(slice1.getMemorySegment()); - assertThat(slice1.getMemorySegmentOffset()).isEqualTo(1); + assertThat(slice1.getMemorySegmentOffset()).isOne(); assertThat(slice2.getMemorySegmentOffset()).isEqualTo(2); assertReadableBytes(slice1, 1, 2, 3, 4, 5); @@ -225,11 +225,11 @@ void testGetSetReaderIndex2() { } private void testGetSetReaderIndex(ReadOnlySlicedNetworkBuffer slice) { - assertThat(buffer.getReaderIndex()).isEqualTo(0); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); + assertThat(slice.getReaderIndex()).isZero(); slice.setReaderIndex(1); - assertThat(buffer.getReaderIndex()).isEqualTo(0); - assertThat(slice.getReaderIndex()).isEqualTo(1); + assertThat(buffer.getReaderIndex()).isZero(); + assertThat(slice.getReaderIndex()).isOne(); } /** @@ -283,8 +283,8 @@ private void testGetNioBufferReadable(ReadOnlySlicedNetworkBuffer slice, int sli // modify sliceByteBuffer position and verify nothing has changed in the original buffer sliceByteBuffer.position(1); - assertThat(buffer.getReaderIndex()).isEqualTo(0); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); + assertThat(slice.getReaderIndex()).isZero(); assertThat(buffer.getSize()).isEqualTo(DATA_SIZE); assertThat(slice.getSize()).isEqualTo(sliceSize); } @@ -302,14 +302,14 @@ void testGetNioBuffer2() { private void testGetNioBuffer(ReadOnlySlicedNetworkBuffer slice, int sliceSize) { ByteBuffer sliceByteBuffer = slice.getNioBuffer(1, 1); assertThat(sliceByteBuffer.isReadOnly()).isTrue(); - assertThat(sliceByteBuffer.remaining()).isEqualTo(1); - assertThat(sliceByteBuffer.limit()).isEqualTo(1); - assertThat(sliceByteBuffer.capacity()).isEqualTo(1); + assertThat(sliceByteBuffer.remaining()).isOne(); + assertThat(sliceByteBuffer.limit()).isOne(); + assertThat(sliceByteBuffer.capacity()).isOne(); // modify sliceByteBuffer position and verify nothing has changed in the original buffer sliceByteBuffer.position(1); - assertThat(buffer.getReaderIndex()).isEqualTo(0); - assertThat(slice.getReaderIndex()).isEqualTo(0); + assertThat(buffer.getReaderIndex()).isZero(); + assertThat(slice.getReaderIndex()).isZero(); assertThat(buffer.getSize()).isEqualTo(DATA_SIZE); assertThat(slice.getSize()).isEqualTo(sliceSize); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index 6b39979fac828..a3c134189863c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -191,7 +191,7 @@ void testReceiveBuffer() throws Exception { new NetworkBufferAllocator(handler)); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse); - assertThat(inputChannel.getNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(inputChannel.getNumberOfQueuedBuffers()).isOne(); assertThat(inputChannel.getSenderBacklog()).isEqualTo(2); } finally { releaseResource(inputGate, networkBufferPool); @@ -307,7 +307,7 @@ void testThrowExceptionForNoAvailableBuffer() throws Exception { assertThat(inputChannel.getNumberOfAvailableBuffers()) .as("There should be no buffers available in the channel.") - .isEqualTo(0); + .isZero(); final BufferResponse bufferResponse = createBufferResponse( @@ -471,7 +471,7 @@ void testNotifyCreditAvailable() throws Exception { allocator); handler.channelRead(mock(ChannelHandlerContext.class), bufferResponse3); - assertThat(inputChannels[0].getUnannouncedCredit()).isEqualTo(1); + assertThat(inputChannels[0].getUnannouncedCredit()).isOne(); assertThat(inputChannels[1].getUnannouncedCredit()).isZero(); channel.runPendingTasks(); @@ -488,7 +488,7 @@ void testNotifyCreditAvailable() throws Exception { assertThat(channel.isWritable()).isTrue(); readFromOutbound = channel.readOutbound(); assertThat(readFromOutbound).isInstanceOf(AddCredit.class); - assertThat(((AddCredit) readFromOutbound).credit).isEqualTo(1); + assertThat(((AddCredit) readFromOutbound).credit).isOne(); assertThat(inputChannels[0].getUnannouncedCredit()).isZero(); assertThat(inputChannels[1].getUnannouncedCredit()).isZero(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java index 86b6539ee052e..dcbba01b2e0b5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReaderTest.java @@ -50,7 +50,7 @@ void testResumeConsumption() throws Exception { assertThat(reader2.getNumCreditsAvailable()).isEqualTo(numCredits); reader2.resumeConsumption(); - assertThat(reader2.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader2.getNumCreditsAvailable()).isZero(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java index 2b0a1d5dcfd2a..bcb842d34a444 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyBufferPoolTest.java @@ -43,7 +43,7 @@ void tearDown() { // Checks in a separate loop in case we have sliced buffers. for (ByteBuf buf : needReleasing) { - assertThat(buf.refCnt()).isEqualTo(0); + assertThat(buf.refCnt()).isZero(); } } finally { needReleasing.clear(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java index 718d8af0f0a86..a224fcf89bb1f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java @@ -259,14 +259,14 @@ void testEnqueueReaderByNotifyingEventBuffer() throws Exception { // The reader is enqueued in the pipeline because the next buffer is an event, even though // no credits are available assertThat(queue.getAvailableReaders()).contains(reader); // contains only (this) one! - assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader.getNumCreditsAvailable()).isZero(); // Flush the buffer to make the channel writable again and see the final results channel.flush(); assertThat((ByteBuf) channel.readOutbound()).isSameAs(channelBlockingBuffer); assertThat(queue.getAvailableReaders()).isEmpty(); - assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader.getNumCreditsAvailable()).isZero(); assertThat((Object) channel.readOutbound()).isNull(); } @@ -317,7 +317,7 @@ void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { assertThat(queue.getAvailableReaders()).isEmpty(); assertThat(reader.hasBuffersAvailable().isAvailable()).isTrue(); assertThat(reader.isRegisteredAsAvailable()).isFalse(); - assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader.getNumCreditsAvailable()).isZero(); // Notify available credits to trigger enqueue the reader again final int notifyNumCredits = 3; @@ -340,7 +340,7 @@ void testEnqueueReaderByNotifyingBufferAndCredit() throws Exception { assertThat((ByteBuf) channel.readOutbound()).isSameAs(channelBlockingBuffer); assertThat(queue.getAvailableReaders()).isEmpty(); - assertThat(reader.getNumCreditsAvailable()).isEqualTo(0); + assertThat(reader.getNumCreditsAvailable()).isZero(); assertThat(reader.hasBuffersAvailable().isAvailable()).isTrue(); assertThat(reader.isRegisteredAsAvailable()).isFalse(); for (int i = 1; i <= notifyNumCredits; i++) { @@ -382,12 +382,12 @@ void testEnqueueReaderByResumingConsumption() throws Exception { reader.notifyDataAvailable(); channel.runPendingTasks(); assertThat(reader.getAvailabilityAndBacklog().isAvailable()).isFalse(); - assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isOne(); queue.addCreditOrResumeConsumption( receiverId, NetworkSequenceViewReader::resumeConsumption); assertThat(reader.getAvailabilityAndBacklog().isAvailable()).isFalse(); - assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(0); + assertThat(subpartition.unsynchronizedGetNumberOfQueuedBuffers()).isZero(); Object data1 = channel.readOutbound(); assertThat(((NettyMessage.BufferResponse) data1).buffer.getDataType()).isEqualTo(dataType1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java index 7c6efd94e9909..62977e53e8efc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionAvailabilityTest.java @@ -53,7 +53,7 @@ void testInitiallyNotAvailable() throws Exception { final ResultSubpartitionView subpartitionView = createView(subpartition, listener); // assert - assertThat(listener.numNotifications).isEqualTo(0); + assertThat(listener.numNotifications).isZero(); // cleanup subpartitionView.releaseAllResources(); @@ -93,7 +93,7 @@ void testAvailabilityNotificationWhenBuffersReturn() throws Exception { // assert assertThat(reader.getAvailabilityAndBacklog(Integer.MAX_VALUE).isAvailable()).isTrue(); - assertThat(listener.numNotifications).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); // cleanup reader.releaseAllResources(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java index 1d47f80c2badb..28bd5be1e2847 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java @@ -45,7 +45,7 @@ import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Behavior tests for the {@link BoundedBlockingSubpartition} and the {@link @@ -97,11 +97,8 @@ public BoundedBlockingSubpartitionTest( void testCreateReaderBeforeFinished() throws Exception { final ResultSubpartition partition = createSubpartition(); - try { - partition.createReadView(new NoOpBufferAvailablityListener()); - fail("expected exception"); - } catch (IllegalStateException ignored) { - } + assertThatThrownBy(() -> partition.createReadView(new NoOpBufferAvailablityListener())) + .isInstanceOf(IllegalStateException.class); partition.release(); } @@ -137,13 +134,9 @@ void testRecycleCurrentBufferOnFailure(BoundedBlockingSubpartitionType type, boo BufferBuilderTestUtils.createFilledFinishedBufferConsumer(100); try { - try { - subpartition.add(consumer); - subpartition.createReadView(new NoOpBufferAvailablityListener()); - fail("should fail with an exception"); - } catch (Exception ignored) { - // expected - } + subpartition.add(consumer); + assertThatThrownBy( + () -> subpartition.createReadView(new NoOpBufferAvailablityListener())); assertThat(consumer.isRecycled()).isFalse(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java index a845d2e5a0aa4..009b543e49d5a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java @@ -137,7 +137,7 @@ void testWriteAndReadDataBuffer() throws Exception { } } - assertThat(dataBuffer.numTotalBytes()).isEqualTo(0); + assertThat(dataBuffer.numTotalBytes()).isZero(); checkWriteReadResult( numSubpartitions, numBytesWritten, numBytesRead, dataWritten, buffersRead); } @@ -410,14 +410,14 @@ void testReleaseDataBuffer() throws Exception { assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(bufferPoolSize); assertThat(dataBuffer.hasRemaining()).isTrue(); - assertThat(dataBuffer.numTotalRecords()).isEqualTo(1); + assertThat(dataBuffer.numTotalRecords()).isOne(); assertThat(dataBuffer.numTotalBytes()).isEqualTo(recordSize); // should release all data and resources dataBuffer.release(); - assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(0); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isZero(); assertThat(dataBuffer.hasRemaining()).isTrue(); - assertThat(dataBuffer.numTotalRecords()).isEqualTo(1); + assertThat(dataBuffer.numTotalRecords()).isOne(); assertThat(dataBuffer.numTotalBytes()).isEqualTo(recordSize); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index e0b4f3fd5a1a5..42726ef8900bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -37,7 +37,6 @@ import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.testutils.junit.extensions.parameterized.NoOpTestExtension; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.CheckedSupplier; @@ -50,6 +49,7 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -96,12 +96,10 @@ void testIllegalReadViewRequest() throws Exception { // Successful request assertThat(subpartition.createReadView(new NoOpBufferAvailablityListener())).isNotNull(); - try { - subpartition.createReadView(new NoOpBufferAvailablityListener()); - - fail("Did not throw expected exception after duplicate notifyNonEmpty view request."); - } catch (IllegalStateException expected) { - } + assertThatThrownBy(() -> subpartition.createReadView(new NoOpBufferAvailablityListener())) + .withFailMessage( + "Did not throw expected exception after duplicate notifyNonEmpty view request.") + .isInstanceOf(IllegalStateException.class); } /** Verifies that the isReleased() check of the view checks the parent subpartition. */ @@ -263,7 +261,7 @@ private void testCleanupReleasedPartition(boolean createView) throws Exception { } partition.release(); - assertThat(partition.getNumberOfQueuedBuffers()).isEqualTo(0); + assertThat(partition.getNumberOfQueuedBuffers()).isZero(); assertThat(partition.isReleased()).isTrue(); if (createView) { @@ -288,7 +286,7 @@ private void testCleanupReleasedPartition(boolean createView) throws Exception { } assertThat(partition.getTotalNumberOfBuffersUnsafe()).isEqualTo(2); assertThat(partition.getTotalNumberOfBytesUnsafe()) - .isEqualTo(0); // buffer data is never consumed + .isZero(); // buffer data is never consumed } @TestTemplate @@ -302,14 +300,14 @@ void testNumberOfQueueBuffers() throws Exception { final PipelinedSubpartition subpartition = createSubpartition(); subpartition.add(createFilledFinishedBufferConsumer(4096)); - assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(subpartition.getNumberOfQueuedBuffers()).isOne(); subpartition.add(createFilledFinishedBufferConsumer(4096)); assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(2); subpartition.getNextBuffer(); - assertThat(subpartition.getNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(subpartition.getNumberOfQueuedBuffers()).isOne(); } @TestTemplate @@ -463,14 +461,11 @@ void testConcurrentTimeoutableCheckpointBarrier() throws Exception { CompletableFuture> checkpointFuture10 = subpartition.getChannelStateFuture(); assertThat(checkpointFuture10).isNotNull(); - try { - // It should fail due to currently does not support concurrent unaligned checkpoints. - subpartition.add(getTimeoutableBarrierBuffer(11L)); - checkpointFuture10.get(); - fail("Should fail with an IllegalStateException."); - } catch (Throwable e) { - ExceptionUtils.assertThrowable(e, IllegalStateException.class); - } + // It should fail due to currently does not support concurrent unaligned checkpoints. + subpartition.add(getTimeoutableBarrierBuffer(11L)); + assertThatThrownBy(checkpointFuture10::get) + .hasCauseInstanceOf(IllegalStateException.class) + .isInstanceOf(ExecutionException.class); } private BufferConsumer getTimeoutableBarrierBuffer(long checkpointId) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index f80035435c8e7..a5854684293d0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -113,34 +113,34 @@ void testRelease() { @TestTemplate void testAddEmptyNonFinishedBuffer() throws IOException { - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); BufferBuilder bufferBuilder = createBufferBuilder(); subpartition.add(bufferBuilder.createBufferConsumer()); - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); assertThat(readView.getNextBuffer()).isNull(); bufferBuilder.finish(); bufferBuilder = createBufferBuilder(); subpartition.add(bufferBuilder.createBufferConsumer()); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isOne(); // notification from finishing previous buffer. - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); assertThat(readView.getNextBuffer()).isNull(); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); } @TestTemplate void testAddNonEmptyNotFinishedBuffer() throws Exception { - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // note that since the buffer builder is not finished, there is still a retained instance! - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertNextBuffer(readView, 1024, false, 0, false, false); } @@ -153,7 +153,7 @@ void testUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isOne(); assertThat(availablityListener.getNumNotifications()).isGreaterThan(0L); assertNextBuffer(readView, 1025, false, 0, false, true); // not notified, but we could still access the unfinished buffer @@ -171,7 +171,7 @@ void testFlushWithUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished long oldNumNotifications = availablityListener.getNumNotifications(); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isOne(); subpartition.flush(); // buffer queue is > 1, should already be notified, no further notification necessary @@ -191,12 +191,12 @@ void testFlushWithUnfinishedBufferBehindFinished() throws Exception { void testFlushWithUnfinishedBufferBehindFinished2() throws Exception { // no buffers -> no notification or any other effects subpartition.flush(); - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); subpartition.add(createFilledFinishedBufferConsumer(1025)); // finished subpartition.add(createFilledUnfinishedBufferConsumer(1024)); // not finished - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isOne(); assertNextBuffer(readView, 1025, false, 0, false, true); long oldNumNotifications = availablityListener.getNumNotifications(); @@ -207,27 +207,27 @@ void testFlushWithUnfinishedBufferBehindFinished2() throws Exception { // calling again should not flush again assertThat(availablityListener.getNumNotifications()).isEqualTo(oldNumNotifications + 1); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(1); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isOne(); assertNextBuffer(readView, 1024, false, 0, false, false); assertNoNextBuffer(readView); } @TestTemplate void testMultipleEmptyBuffers() throws Exception { - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); subpartition.add(createFilledFinishedBufferConsumer(0)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(2); subpartition.add(createFilledFinishedBufferConsumer(1024)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); assertNextBuffer(readView, 1024, false, 0, false, true); } @@ -235,7 +235,7 @@ void testMultipleEmptyBuffers() throws Exception { @TestTemplate void testEmptyFlush() { subpartition.flush(); - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); } @TestTemplate @@ -245,43 +245,43 @@ void testBasicPipelinedProduceConsumeLogic() throws Exception { assertNoNextBuffer(readView); assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()) .isFalse(); // also after getNextBuffer() - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); // Add data to the queue... subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); - assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isOne(); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) - .isEqualTo(0); // only updated when getting the buffer + .isZero(); // only updated when getting the buffer - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); // ...and one available result assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(BUFFER_SIZE); // only updated when getting the buffer - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertNoNextBuffer(readView); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); // Add data to the queue... subpartition.add(createFilledFinishedBufferConsumer(BUFFER_SIZE)); assertThat(readView.getAvailabilityAndBacklog(0).isAvailable()).isFalse(); assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(2); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(BUFFER_SIZE); // only updated when getting the buffer - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(2 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertNoNextBuffer(readView); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); // some tests with events @@ -295,37 +295,37 @@ void testBasicPipelinedProduceConsumeLogic() throws Exception { assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(5); assertThat(subpartition.getBuffersInBacklogUnsafe()) - .isEqualTo(1); // two buffers (events don't count) + .isOne(); // two buffers (events don't count) assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(2 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); // the first buffer assertNextBuffer(readView, BUFFER_SIZE, true, 0, true, true); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(3 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); // the event assertNextEvent(readView, BUFFER_SIZE, null, false, 0, false, true); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(4 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); // the remaining buffer assertNextBuffer(readView, BUFFER_SIZE, false, 0, false, true); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(5 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); // nothing more assertNoNextBuffer(readView); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(5); assertThat(subpartition.getTotalNumberOfBytesUnsafe()) .isEqualTo(5 * BUFFER_SIZE); // only updated when getting the buffer - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); } @TestTemplate @@ -334,22 +334,22 @@ void testBarrierOvertaking() throws Exception { subpartition.setChannelStateWriter(channelStateWriter); subpartition.add(createFilledFinishedBufferConsumer(1)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(0); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isZero(); + assertThat(availablityListener.getNumPriorityEvents()).isZero(); subpartition.add(createFilledFinishedBufferConsumer(2)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isOne(); + assertThat(availablityListener.getNumPriorityEvents()).isZero(); BufferConsumer eventBuffer = EventSerializer.toBufferConsumer(EndOfSuperstepEvent.INSTANCE, false); subpartition.add(eventBuffer); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isOne(); + assertThat(availablityListener.getNumPriorityEvents()).isZero(); subpartition.add(createFilledFinishedBufferConsumer(4)); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(0); + assertThat(availablityListener.getNumNotifications()).isOne(); + assertThat(availablityListener.getNumPriorityEvents()).isZero(); CheckpointOptions options = CheckpointOptions.unaligned( @@ -359,8 +359,8 @@ void testBarrierOvertaking() throws Exception { BufferConsumer barrierBuffer = EventSerializer.toBufferConsumer(new CheckpointBarrier(0, 0, options), true); subpartition.add(barrierBuffer); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); + assertThat(availablityListener.getNumPriorityEvents()).isOne(); final List inflight = channelStateWriter.getAddedOutput().get(subpartition.getSubpartitionInfo()); @@ -401,16 +401,16 @@ void testAvailabilityAfterPriority() throws Exception { BufferConsumer barrierBuffer = EventSerializer.toBufferConsumer(new CheckpointBarrier(0, 0, options), true); subpartition.add(barrierBuffer); - assertThat(availablityListener.getNumNotifications()).isEqualTo(1); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); + assertThat(availablityListener.getNumNotifications()).isOne(); + assertThat(availablityListener.getNumPriorityEvents()).isOne(); subpartition.add(createFilledFinishedBufferConsumer(1)); assertThat(availablityListener.getNumNotifications()).isEqualTo(2); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isOne(); subpartition.add(createFilledFinishedBufferConsumer(2)); assertThat(availablityListener.getNumNotifications()).isEqualTo(2); - assertThat(availablityListener.getNumPriorityEvents()).isEqualTo(1); + assertThat(availablityListener.getNumPriorityEvents()).isOne(); assertNextEvent( readView, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java index b6ba4fab4d3e8..83d870b237b77 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PrioritizedDequeTest.java @@ -68,9 +68,9 @@ void testGetAndRemove() { deque.add(1); deque.add(3); - assertThat(deque.getAndRemove(v -> v == 1).intValue()).isEqualTo(1); + assertThat(deque.getAndRemove(v -> v == 1).intValue()).isOne(); assertThat(deque.asUnmodifiableCollection()).containsExactly(0, 2, 1, 3); - assertThat(deque.getAndRemove(v -> v == 1).intValue()).isEqualTo(1); + assertThat(deque.getAndRemove(v -> v == 1).intValue()).isOne(); assertThat(deque.asUnmodifiableCollection()).containsExactly(0, 2, 3); try { int removed = deque.getAndRemove(v -> v == 1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java index 06de23dc50bfa..e9945c667fc84 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java @@ -228,7 +228,7 @@ private static ResultPartition createResultPartition( 1); // guard our test assumptions - assertThat(descriptor.getNumberOfSubpartitions()).isEqualTo(1); + assertThat(descriptor.getNumberOfSubpartitions()).isOne(); final ResultPartition partition = factory.create("test", 0, descriptor); manager.registerResultPartition(partition); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 4422d40cb6b53..5b0efe37c29e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -180,7 +180,7 @@ private void testAddOnReleasedPartition(ResultPartitionType partitionType) throw // partitionWriter.emitRecord() should silently drop the given record bufferWritingResultPartition.emitRecord(ByteBuffer.allocate(bufferSize), 0); } finally { - assertThat(bufferWritingResultPartition.numBuffersOut.getCount()).isEqualTo(1); + assertThat(bufferWritingResultPartition.numBuffersOut.getCount()).isOne(); assertThat(bufferWritingResultPartition.numBytesOut.getCount()).isEqualTo(bufferSize); // the buffer should be recycled for the result partition has already been released assertThat(bufferWritingResultPartition.getBufferPool().bestEffortGetNumOfUsedBuffers()) @@ -230,10 +230,10 @@ private void testAddOnPartition(final ResultPartitionType partitionType) throws // partitionWriter.emitRecord() will allocate a new buffer and copies the record to it bufferWritingResultPartition.emitRecord(ByteBuffer.allocate(bufferSize), 0); } finally { - assertThat(bufferWritingResultPartition.numBuffersOut.getCount()).isEqualTo(1); + assertThat(bufferWritingResultPartition.numBuffersOut.getCount()).isOne(); assertThat(bufferWritingResultPartition.numBytesOut.getCount()).isEqualTo(bufferSize); assertThat(bufferWritingResultPartition.getBufferPool().bestEffortGetNumOfUsedBuffers()) - .isEqualTo(1); + .isOne(); } } @@ -533,7 +533,7 @@ public void testWaitForAllRecordProcessed() throws IOException { bufferWritingResultPartition.getAllDataProcessedFuture(); assertThat(allRecordsProcessedFuture).isNotDone(); for (ResultSubpartition resultSubpartition : bufferWritingResultPartition.subpartitions) { - assertThat(resultSubpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); + assertThat(resultSubpartition.getTotalNumberOfBuffersUnsafe()).isOne(); Buffer nextBuffer = ((PipelinedSubpartition) resultSubpartition).pollBuffer().buffer(); assertThat(nextBuffer.isBuffer()).isFalse(); assertThat(EventSerializer.fromBuffer(nextBuffer, getClass().getClassLoader())) @@ -870,7 +870,7 @@ void testSizeOfQueuedBuffers() throws Exception { assertThat(resultPartition.getSizeOfQueuedBuffersUnsafe()).isEqualTo(7); assertThat(subpartition1.pollBuffer().buffer().getSize()).isEqualTo(7); - assertThat(resultPartition.getSizeOfQueuedBuffersUnsafe()).isEqualTo(0); + assertThat(resultPartition.getSizeOfQueuedBuffersUnsafe()).isZero(); } @NotNull diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java index 6a0528152bbf6..0971e986e1047 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java @@ -130,7 +130,7 @@ void testCreateSubpartitionReader() throws Exception { assertThat(readScheduler.isRunning()).isTrue(); assertThat(readScheduler.getDataFileChannel().isOpen()).isTrue(); assertThat(readScheduler.getIndexFileChannel().isOpen()).isTrue(); - assertThat(ioExecutor.numQueuedRunnables()).isEqualTo(1); + assertThat(ioExecutor.numQueuedRunnables()).isOne(); int numBuffersRead = 0; while (numBuffersRead < numBuffersPerSubpartition) { @@ -171,7 +171,7 @@ void testReleaseWhileReading() throws Exception { assertThat(subpartitionReader.getFailureCause()).isNotNull(); assertThat(subpartitionReader.isReleased()).isTrue(); - assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(0); + assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isZero(); assertThat(subpartitionReader.getAvailabilityAndBacklog(0).isAvailable()).isTrue(); readScheduler.getReleaseFuture().get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java index c202669e9f328..6a1bc591e3a47 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java @@ -340,7 +340,7 @@ void testReleaseWhileWriting() throws Exception { assertThatThrownBy( () -> partition.emitRecord(ByteBuffer.allocate(bufferSize * numBuffers), 2)) .isInstanceOf(IllegalStateException.class); - assertThat(fileChannelManager.getPaths()[0].list().length).isEqualTo(0); + assertThat(fileChannelManager.getPaths()[0].list().length).isZero(); } @TestTemplate @@ -373,7 +373,7 @@ void testRelease() throws Exception { while (partition.getResultFile() != null) { Thread.sleep(100); } - assertThat(checkNotNull(fileChannelManager.getPaths()[0].list()).length).isEqualTo(0); + assertThat(checkNotNull(fileChannelManager.getPaths()[0].list()).length).isZero(); } @TestTemplate diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java index 9552bb457c52d..2d15672dff2e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java @@ -94,14 +94,14 @@ void testReadBuffers() throws Exception { Queue segments = createsMemorySegments(2); subpartitionReader.readBuffers(segments, FreeingBufferRecycler.INSTANCE); - assertThat(listener.numNotifications).isEqualTo(1); - assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); + assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isOne(); assertThat(segments).isEmpty(); segments = createsMemorySegments(2); subpartitionReader.readBuffers(segments, FreeingBufferRecycler.INSTANCE); - assertThat(listener.numNotifications).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(2); assertThat(segments).isEmpty(); @@ -115,7 +115,7 @@ void testReadBuffers() throws Exception { assertThat(listener.numNotifications).isEqualTo(2); assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()) .isEqualTo(numBuffersPerSubpartition - 2); - assertThat(segments.size()).isEqualTo(1); + assertThat(segments.size()).isOne(); } @Test @@ -159,7 +159,7 @@ void testFail() throws Exception { createSortMergeSubpartitionReader(listener); subpartitionReader.readBuffers(segments, segments::add); - assertThat(listener.numNotifications).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(4); subpartitionReader.fail(new RuntimeException("Test exception.")); @@ -186,7 +186,7 @@ void testReleaseAllResources() throws Exception { createSortMergeSubpartitionReader(listener); subpartitionReader.readBuffers(segments, segments::add); - assertThat(listener.numNotifications).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()).isEqualTo(4); subpartitionReader.releaseAllResources(); @@ -195,7 +195,7 @@ void testReleaseAllResources() throws Exception { assertThat(subpartitionReader.getAvailabilityAndBacklog(0).isAvailable()).isTrue(); assertThat(subpartitionReader.isReleased()).isTrue(); - assertThat(listener.numNotifications).isEqualTo(1); + assertThat(listener.numNotifications).isOne(); assertThat(subpartitionReader.getFailureCause()).isNull(); } finally { assertThat(segments).hasSize(numSegments); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index f6bb1b58691f5..29df29abcbdc6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -25,7 +25,7 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Basic subpartition behaviour tests. */ abstract class SubpartitionTestBase { @@ -43,12 +43,8 @@ void createReaderAfterDispose() throws Exception { final ResultSubpartition subpartition = createSubpartition(); subpartition.release(); - try { - subpartition.createReadView(() -> {}); - fail("expected an exception"); - } catch (IllegalStateException e) { - // expected - } + assertThatThrownBy(() -> subpartition.createReadView(() -> {})) + .isInstanceOf(IllegalStateException.class); } @TestTemplate @@ -57,16 +53,16 @@ void testAddAfterFinish() throws Exception { try { subpartition.finish(); - assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isOne(); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); BufferConsumer bufferConsumer = createFilledFinishedBufferConsumer(4096); assertThat(subpartition.add(bufferConsumer)).isEqualTo(-1); assertThat(bufferConsumer.isRecycled()).isTrue(); - assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isEqualTo(1); - assertThat(subpartition.getBuffersInBacklogUnsafe()).isEqualTo(0); + assertThat(subpartition.getTotalNumberOfBuffersUnsafe()).isOne(); + assertThat(subpartition.getBuffersInBacklogUnsafe()).isZero(); } finally { if (subpartition != null) { subpartition.release(); @@ -149,14 +145,8 @@ void testRecycleBufferAndConsumerOnFailure() throws Exception { final BufferConsumer consumer = BufferBuilderTestUtils.createFilledFinishedBufferConsumer(100); - try { - subpartition.add(consumer); - subpartition.flush(); - fail("should fail with an exception"); - } catch (Exception ignored) { - // expected - } - + subpartition.add(consumer); + assertThatThrownBy(subpartition::flush); assertThat(consumer.isRecycled()).isTrue(); } finally { subpartition.release(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java index 7f6b9d48940cc..cbb7e2ac4024b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java @@ -225,7 +225,7 @@ void testFloatingBuffersUsage() throws Exception { drainBuffer(totalRequestedBuffers, remoteInputChannel1); - assertThat(remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()).isEqualTo(0); + assertThat(remoteInputChannel1.unsynchronizedGetFloatingBuffersAvailable()).isZero(); assertThat((double) inputBuffersUsageGauge.getValue()) .isEqualTo( (double) (buffersPerChannel + totalRequestedBuffers) / totalBuffers, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index 6972209263010..1fe6f49b48175 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -37,7 +37,7 @@ class InputChannelTest { void testExponentialBackoff() throws Exception { InputChannel ch = createInputChannel(500, 4000); - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); assertThat(ch.increaseBackoff()).isTrue(); assertThat(ch.getCurrentBackoff()).isEqualTo(500); @@ -59,7 +59,7 @@ void testExponentialBackoff() throws Exception { void testExponentialBackoffCappedAtMax() throws Exception { InputChannel ch = createInputChannel(500, 3000); - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); assertThat(ch.increaseBackoff()).isTrue(); assertThat(ch.getCurrentBackoff()).isEqualTo(500); @@ -81,7 +81,7 @@ void testExponentialBackoffCappedAtMax() throws Exception { void testExponentialBackoffSingle() throws Exception { InputChannel ch = createInputChannel(500, 500); - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); assertThat(ch.increaseBackoff()).isTrue(); assertThat(ch.getCurrentBackoff()).isEqualTo(500); @@ -94,10 +94,10 @@ void testExponentialBackoffSingle() throws Exception { void testExponentialNoBackoff() throws Exception { InputChannel ch = createInputChannel(0, 0); - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); assertThat(ch.increaseBackoff()).isFalse(); - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); } private InputChannel createInputChannel(int initialBackoff, int maxBackoff) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 0f04a4a3247b7..35aa9cdcabdc5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -270,12 +270,8 @@ void testPartitionRequestExponentialBackoff() throws Exception { } // Exception after backoff is greater than the maximum backoff. - try { - ch.retriggerSubpartitionRequest(timer); - ch.getNextBuffer(); - fail("Did not throw expected exception."); - } catch (Exception expected) { - } + ch.retriggerSubpartitionRequest(timer); + assertThatThrownBy(ch::getNextBuffer); } @Test @@ -463,11 +459,7 @@ void testGetNextAfterPartitionReleased() throws Exception { // release the subpartition view subpartitionView.releaseAllResources(); - try { - channel.getNextBuffer(); - fail("Did not throw expected CancelTaskException"); - } catch (CancelTaskException ignored) { - } + assertThatThrownBy(channel::getNextBuffer).isInstanceOf(CancelTaskException.class); channel.releaseAllResources(); assertThat(channel.getNextBuffer()).isEmpty(); @@ -662,7 +654,7 @@ void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() throws Exce inputGate.setInputChannels(localChannel); // then: Buffers in use should be equal to 0 until subpartition view initialization. - assertThat(localChannel.getBuffersInUseCount()).isEqualTo(0); + assertThat(localChannel.getBuffersInUseCount()).isZero(); // when: The subpartition view is initialized. localChannel.requestSubpartition(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index c04696cbbf79a..77535782759f0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -174,16 +174,14 @@ void testExceptionOnReordering() throws Exception { // This does not yet throw the exception, but sets the error at the channel. inputChannel.onBuffer(buffer, 29, -1); - try { - inputChannel.getNextBuffer(); + assertThatThrownBy(inputChannel::getNextBuffer) + .withFailMessage( + "Did not throw expected exception after enqueuing an out-of-order buffer."); - fail("Did not throw expected exception after enqueuing an out-of-order buffer."); - } catch (Exception expected) { - assertThat(buffer.isRecycled()).isFalse(); - // free remaining buffer instances - inputChannel.releaseAllResources(); - assertThat(buffer.isRecycled()).isTrue(); - } + assertThat(buffer.isRecycled()).isFalse(); + // free remaining buffer instances + inputChannel.releaseAllResources(); + assertThat(buffer.isRecycled()).isTrue(); } @Test @@ -219,12 +217,9 @@ public void addInputData( final Buffer buffer = createBuffer(TestBufferFactory.BUFFER_SIZE); assertThat(buffer.isRecycled()).isFalse(); - try { - inputChannel.onBuffer(buffer, 0, -1); - fail("This should have failed"); - } catch (ExpectedTestException ex) { - // ignore - } + assertThatThrownBy(() -> inputChannel.onBuffer(buffer, 0, -1)) + .isInstanceOf(ExpectedTestException.class); + // This check is not strictly speaking necessary. Generally speaking if exception happens // during persisting, there are two potentially correct outcomes: // 1. buffer is recycled only once, in #onBuffer call when handling exception @@ -326,7 +321,7 @@ private void testConcurrentReleaseAndSomething( assertThat(inputChannel.getNumberOfQueuedBuffers()) .withFailMessage( "Resource leak during concurrent release and notifyBufferAvailable.") - .isEqualTo(0); + .isZero(); } } finally { executor.shutdown(); @@ -371,12 +366,8 @@ void testPartitionRequestExponentialBackoff() throws Exception { } // Exception after backoff is greater than the maximum backoff. - try { - ch.retriggerSubpartitionRequest(); - ch.getNextBuffer(); - fail("Did not throw expected exception."); - } catch (Exception expected) { - } + ch.retriggerSubpartitionRequest(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(IOException.class); } @Test @@ -398,12 +389,8 @@ void testPartitionRequestSingleBackoff() throws Exception { client.verifyResult(partitionId, 0, 500); // Exception after backoff is greater than the maximum backoff. - try { - ch.retriggerSubpartitionRequest(); - ch.getNextBuffer(); - fail("Did not throw expected exception."); - } catch (Exception expected) { - } + ch.retriggerSubpartitionRequest(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(IOException.class); } @Test @@ -421,12 +408,8 @@ void testPartitionRequestNoBackoff() throws Exception { client.verifyResult(partitionId, 0, 0); // Exception, because backoff is disabled. - try { - ch.retriggerSubpartitionRequest(); - ch.getNextBuffer(); - fail("Did not throw expected exception."); - } catch (Exception expected) { - } + ch.retriggerSubpartitionRequest(); + assertThatThrownBy(ch::getNextBuffer).isInstanceOf(IOException.class); } @Test @@ -536,7 +519,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(16); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Increase the backlog @@ -554,7 +537,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(18); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one exclusive buffer @@ -571,7 +554,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(18); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one floating buffer @@ -589,7 +572,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(18); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Decrease the backlog @@ -606,7 +589,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(15); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); // Recycle one more floating buffer @@ -624,7 +607,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(15); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 1 buffers available in local pool") - .isEqualTo(1); + .isOne(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isFalse(); // Increase the backlog again @@ -642,7 +625,7 @@ void testAvailableBuffersLessThanRequiredBuffers() throws Exception { .isEqualTo(17); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); assertThat(inputChannel.isWaitingForFloatingBuffers()).isTrue(); } catch (Throwable t) { thrown = t; @@ -694,7 +677,7 @@ void testAvailableBuffersEqualToRequiredBuffers() throws Exception { .isEqualTo(14); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); // Recycle one floating buffer floatingBuffer.recycleBuffer(); @@ -712,7 +695,7 @@ void testAvailableBuffersEqualToRequiredBuffers() throws Exception { .isEqualTo(14); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 1 buffers available in local pool") - .isEqualTo(1); + .isOne(); // Recycle one exclusive buffer exclusiveBuffer.recycleBuffer(); @@ -782,7 +765,7 @@ void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { .isEqualTo(14); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); // Decrease the backlog to make the number of available buffers more than required // buffers @@ -799,7 +782,7 @@ void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { .isEqualTo(12); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 0 buffers available in local pool") - .isEqualTo(0); + .isZero(); // Recycle one exclusive buffer exclusiveBuffer.recycleBuffer(); @@ -817,7 +800,7 @@ void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { .isEqualTo(12); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be 1 buffers available in local pool") - .isEqualTo(1); + .isOne(); // Recycle one floating buffer floatingBuffer.recycleBuffer(); @@ -902,7 +885,7 @@ void testFairDistributionFloatingBuffers() throws Exception { .isEqualTo(3); assertThat(inputChannel.getUnannouncedCredit()) .withFailMessage("There should be 1 unannounced credits in the channel") - .isEqualTo(1); + .isOne(); } } catch (Throwable t) { thrown = t; @@ -955,7 +938,7 @@ void testFailureInNotifyBufferAvailable() throws Exception { .hasCauseInstanceOf(IllegalStateException.class); // currently, the buffer is still enqueued in the bufferQueue of failingRemoteIC - assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isZero(); buffer = successfulRemoteIC.requestBuffer(); assertThat(buffer) .withFailMessage("buffer should still remain in failingRemoteIC") @@ -964,7 +947,7 @@ void testFailureInNotifyBufferAvailable() throws Exception { // releasing resources in failingRemoteIC should free the buffer again and immediately // recycle it into successfulRemoteIC failingRemoteIC.releaseAllResources(); - assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(0); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isZero(); buffer = successfulRemoteIC.requestBuffer(); assertThat(buffer).withFailMessage("no buffer given to successfulRemoteIC").isNotNull(); } catch (Throwable t) { @@ -1028,7 +1011,7 @@ public Void call() throws Exception { assertThat(inputChannel.getNumberOfAvailableBuffers()) .withFailMessage("There should be no buffers available in the channel.") - .isEqualTo(0); + .isZero(); assertThat( bufferPool.getNumberOfAvailableMemorySegments() + networkBufferPool.getNumberOfAvailableMemorySegments()) @@ -1096,7 +1079,7 @@ public Void call() throws Exception { .isEqualTo(inputChannel.getNumberOfRequiredBuffers()); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage("There should be no buffers available in local pool.") - .isEqualTo(0); + .isZero(); } catch (Throwable t) { thrown = t; } finally { @@ -1151,7 +1134,7 @@ public Void call() throws Exception { assertThat(inputChannel.getNumberOfAvailableBuffers()) .withFailMessage("There should be no buffers available in the channel.") - .isEqualTo(0); + .isZero(); assertThat(bufferPool.getNumberOfAvailableMemorySegments()) .withFailMessage( "There should be %d buffers available in local pool.", @@ -1409,13 +1392,13 @@ void testOnUpstreamBlockedAndResumed() throws Exception { remoteChannel2.onBuffer(barrier, 0, 0); assertThat(remoteChannel1.getNumberOfAvailableBuffers()).isEqualTo(4); - assertThat(remoteChannel2.getNumberOfAvailableBuffers()).isEqualTo(0); + assertThat(remoteChannel2.getNumberOfAvailableBuffers()).isZero(); remoteChannel1.resumeConsumption(); remoteChannel2.resumeConsumption(); assertThat(remoteChannel1.getUnannouncedCredit()).isEqualTo(4); - assertThat(remoteChannel2.getUnannouncedCredit()).isEqualTo(0); + assertThat(remoteChannel2.getUnannouncedCredit()).isZero(); remoteChannel1.onSenderBacklog(4); remoteChannel2.onSenderBacklog(4); @@ -1604,7 +1587,7 @@ void testSizeOfQueuedBuffers() throws Exception { int bufferSize = 1; int queueSize = 0; final RemoteInputChannel channel = buildInputGateAndGetChannel(sequenceNumber); - assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(0); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isZero(); // Receive a couple of buffers. for (int i = 0; i < 2; i++) { @@ -1627,7 +1610,7 @@ void testSizeOfQueuedBuffers() throws Exception { assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(queueSize); } - assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isEqualTo(0); + assertThat(channel.unsynchronizedGetSizeOfQueuedBuffers()).isZero(); } private void sendBarrier( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 0f03b3d223d83..28b3d3223ed21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -161,7 +161,7 @@ void testSetupLogic() throws Exception { assertThat( ((RecoveredInputChannel) inputChannel) .bufferManager.getNumberOfAvailableBuffers()) - .isEqualTo(0); + .isZero(); } } @@ -169,24 +169,24 @@ void testSetupLogic() throws Exception { // after setup assertThat(inputGate.getBufferPool()).isNotNull(); - assertThat(inputGate.getBufferPool().getNumberOfRequiredMemorySegments()).isEqualTo(1); + assertThat(inputGate.getBufferPool().getNumberOfRequiredMemorySegments()).isOne(); for (InputChannel inputChannel : inputGate.getInputChannels().values()) { if (inputChannel instanceof RemoteRecoveredInputChannel) { assertThat( ((RemoteRecoveredInputChannel) inputChannel) .bufferManager.getNumberOfAvailableBuffers()) - .isEqualTo(0); + .isZero(); } else if (inputChannel instanceof LocalRecoveredInputChannel) { assertThat( ((LocalRecoveredInputChannel) inputChannel) .bufferManager.getNumberOfAvailableBuffers()) - .isEqualTo(0); + .isZero(); } } inputGate.convertRecoveredInputChannels(); assertThat(inputGate.getBufferPool()).isNotNull(); - assertThat(inputGate.getBufferPool().getNumberOfRequiredMemorySegments()).isEqualTo(1); + assertThat(inputGate.getBufferPool().getNumberOfRequiredMemorySegments()).isOne(); for (InputChannel inputChannel : inputGate.getInputChannels().values()) { if (inputChannel instanceof RemoteInputChannel) { assertThat(((RemoteInputChannel) inputChannel).getNumberOfAvailableBuffers()) @@ -473,14 +473,14 @@ void testBackwardsEventWithUninitializedChannel() throws Exception { setupInputGate(inputGate, inputChannels); // Only the local channel can request - assertThat(partitionManager.counter).isEqualTo(1); + assertThat(partitionManager.counter).isOne(); // Send event backwards and initialize unknown channel afterwards final TaskEvent event = new TestTaskEvent(); inputGate.sendTaskEvent(event); // Only the local channel can send out the event - assertThat(taskEventPublisher.counter).isEqualTo(1); + assertThat(taskEventPublisher.counter).isOne(); // After the update, the pending event should be send to local channel @@ -520,7 +520,7 @@ void testUpdateChannelBeforeRequest() throws Exception { location, createRemoteWithIdAndLocation(resultPartitionID.getPartitionId(), location)); - assertThat(partitionManager.counter).isEqualTo(0); + assertThat(partitionManager.counter).isZero(); } /** @@ -684,7 +684,7 @@ void testRequestBackoffConfiguration() throws Exception { InputChannel[] channels = new InputChannel[] {localChannel, remoteChannel, unknownChannel}; for (InputChannel ch : channels) { - assertThat(ch.getCurrentBackoff()).isEqualTo(0); + assertThat(ch.getCurrentBackoff()).isZero(); assertThat(ch.increaseBackoff()).isTrue(); assertThat(ch.getCurrentBackoff()).isEqualTo(initialBackoff); @@ -933,12 +933,12 @@ void testSingleInputGateWithSubpartitionIndexRange() throws IOException, Interru SubpartitionInfo info6 = createSubpartitionInfo(partitionIds[2], 1); assertThat(gate.getInputChannels().size()).isEqualTo(6); - assertThat(gate.getInputChannels().get(info1).getConsumedSubpartitionIndex()).isEqualTo(0); - assertThat(gate.getInputChannels().get(info2).getConsumedSubpartitionIndex()).isEqualTo(1); - assertThat(gate.getInputChannels().get(info3).getConsumedSubpartitionIndex()).isEqualTo(0); - assertThat(gate.getInputChannels().get(info4).getConsumedSubpartitionIndex()).isEqualTo(1); - assertThat(gate.getInputChannels().get(info5).getConsumedSubpartitionIndex()).isEqualTo(0); - assertThat(gate.getInputChannels().get(info6).getConsumedSubpartitionIndex()).isEqualTo(1); + assertThat(gate.getInputChannels().get(info1).getConsumedSubpartitionIndex()).isZero(); + assertThat(gate.getInputChannels().get(info2).getConsumedSubpartitionIndex()).isOne(); + assertThat(gate.getInputChannels().get(info3).getConsumedSubpartitionIndex()).isZero(); + assertThat(gate.getInputChannels().get(info4).getConsumedSubpartitionIndex()).isOne(); + assertThat(gate.getInputChannels().get(info5).getConsumedSubpartitionIndex()).isZero(); + assertThat(gate.getInputChannels().get(info6).getConsumedSubpartitionIndex()).isOne(); assertChannelsType(gate, LocalRecoveredInputChannel.class, Arrays.asList(info1, info2)); assertChannelsType(gate, RemoteRecoveredInputChannel.class, Arrays.asList(info3, info4)); @@ -947,7 +947,7 @@ void testSingleInputGateWithSubpartitionIndexRange() throws IOException, Interru // test setup gate.setup(); assertThat(gate.getBufferPool()).isNotNull(); - assertThat(gate.getBufferPool().getNumberOfRequiredMemorySegments()).isEqualTo(1); + assertThat(gate.getBufferPool().getNumberOfRequiredMemorySegments()).isOne(); gate.finishReadRecoveredState(); while (!gate.getStateConsumedFuture().isDone()) { @@ -1027,7 +1027,7 @@ void testQueuedBuffers() throws Exception { setupInputGate(inputGate, inputChannels); remoteInputChannel.onBuffer(createBuffer(1), 0, 0); - assertThat(inputGate.getNumberOfQueuedBuffers()).isEqualTo(1); + assertThat(inputGate.getNumberOfQueuedBuffers()).isOne(); resultPartition.emitRecord(ByteBuffer.allocate(1), 0); assertThat(inputGate.getNumberOfQueuedBuffers()).isEqualTo(2); @@ -1186,11 +1186,11 @@ void testBufferInUseCount() throws Exception { inputGate.setInputChannels(inputChannels); // It should be no buffers when all channels are empty. - assertThat(inputGate.getBuffersInUseCount()).isEqualTo(0); + assertThat(inputGate.getBuffersInUseCount()).isZero(); // Add buffers into channels. inputChannels[0].readBuffer(); - assertThat(inputGate.getBuffersInUseCount()).isEqualTo(1); + assertThat(inputGate.getBuffersInUseCount()).isOne(); inputChannels[0].readBuffer(); assertThat(inputGate.getBuffersInUseCount()).isEqualTo(2); @@ -1244,7 +1244,7 @@ private static void verifyBuffersInBufferPool(boolean isPipeline, int subpartiti for (InputChannel inputChannel : gate.getInputChannels().values()) { if (inputChannel instanceof RemoteInputChannel) { - assertThat(((RemoteInputChannel) inputChannel).getInitialCredit()).isEqualTo(0); + assertThat(((RemoteInputChannel) inputChannel).getInitialCredit()).isZero(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 0e16a2e0ba926..5724060dd1cc6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -206,7 +206,7 @@ void testAvailability() throws IOException, InterruptedException { assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); inputChannel1.read(BufferBuilderTestUtils.buildSomeBuffer(2)); assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); - assertThat(inputGate.getNext().get().getBuffer().getSize()).isEqualTo(1); + assertThat(inputGate.getNext().get().getBuffer().getSize()).isOne(); assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsBufferContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsBufferContextTest.java index f0419d114dfca..065bca39162ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsBufferContextTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsBufferContextTest.java @@ -53,7 +53,7 @@ void testBufferStartSpillingRefCount() { assertThat(bufferContext.isSpillStarted()).isTrue(); assertThat(buffer.refCnt()).isEqualTo(2); spilledFuture.complete(null); - assertThat(buffer.refCnt()).isEqualTo(1); + assertThat(buffer.refCnt()).isOne(); } @Test @@ -65,7 +65,7 @@ void testBufferStartSpillingRepeatedly() { @Test void testBufferReleaseRefCount() { Buffer buffer = bufferContext.getBuffer(); - assertThat(buffer.refCnt()).isEqualTo(1); + assertThat(buffer.refCnt()).isOne(); bufferContext.release(); assertThat(bufferContext.isReleased()).isTrue(); assertThat(buffer.isRecycled()).isTrue(); @@ -134,7 +134,7 @@ void testBufferConsumedThenRelease() { Buffer buffer = bufferContext.getBuffer(); bufferContext.consumed(HsConsumerId.DEFAULT); bufferContext.release(); - assertThat(buffer.refCnt()).isEqualTo(1); + assertThat(buffer.refCnt()).isOne(); } private static HsBufferContext createBufferContext() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java index c3bf1ad875d8e..e830ccc757338 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataIndexImplTest.java @@ -102,7 +102,7 @@ void testGetReadableRegion() { readableRegion -> { assertRegionStartWithTargetBufferIndex(readableRegion, 1); // Readable region will not include discontinuous buffer. - assertThat(readableRegion.numReadable).isEqualTo(1); + assertThat(readableRegion.numReadable).isOne(); }); assertThat(hsDataIndex.getReadableRegion(subpartitionId, 3, 0)) .hasValueSatisfying( @@ -116,7 +116,7 @@ void testGetReadableRegion() { .hasValueSatisfying( readableRegion -> { assertRegionStartWithTargetBufferIndex(readableRegion, 4); - assertThat(readableRegion.numReadable).isEqualTo(1); + assertThat(readableRegion.numReadable).isOne(); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java index b1c7345b33d84..e88b27dd960b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java @@ -168,7 +168,7 @@ void testRunReleaseUnusedBuffers() throws Exception { (requestedBuffers, readBuffers) -> { assertThat(prepareForSchedulingFinished).isCompleted(); assertThat(requestedBuffers).hasSize(BUFFER_POOL_SIZE); - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(0); + assertThat(bufferPool.getAvailableBuffers()).isZero(); // read one buffer, return another buffer to data manager. readBuffers.add(requestedBuffers.poll()); }); @@ -179,7 +179,7 @@ void testRunReleaseUnusedBuffers() throws Exception { ioExecutor.trigger(); // not used buffer should be recycled. - assertThat(bufferPool.getAvailableBuffers()).isEqualTo(1); + assertThat(bufferPool.getAvailableBuffers()).isOne(); } /** Test file data manager will schedule readers in order. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManagerTest.java index 5be52e7d4716c..f2f8cd89a06d3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManagerTest.java @@ -155,7 +155,7 @@ void testHandleDecision() throws Exception { assertThatFuture(spilledFuture).eventuallySucceeds(); assertThatFuture(readableFuture).eventuallySucceeds(); assertThat(readableFuture).isCompletedWithValue(2); - assertThat(memoryDataManager.getNumTotalUnSpillBuffers()).isEqualTo(1); + assertThat(memoryDataManager.getNumTotalUnSpillBuffers()).isOne(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java index b7b3ee8627975..0751d40abdd50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java @@ -203,7 +203,7 @@ void testBroadcastEvent() throws Exception { try (HsResultPartition resultPartition = createHsResultPartition(2, bufferPool)) { resultPartition.broadcastEvent(EndOfPartitionEvent.INSTANCE, false); // broadcast event does not request buffer - assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(1); + assertThat(bufferPool.getNumberOfAvailableMemorySegments()).isOne(); Tuple2[] viewAndListeners = createSubpartitionViews(resultPartition, 2); @@ -502,7 +502,7 @@ void testMetricsUpdateForBroadcastOnlyResultPartition() throws Exception { BufferPool bufferPool = globalPool.createBufferPool(3, 3); try (HsResultPartition partition = createHsResultPartition(2, bufferPool, true)) { partition.broadcastRecord(ByteBuffer.allocate(bufferSize)); - assertThat(taskIOMetricGroup.getNumBuffersOutCounter().getCount()).isEqualTo(1); + assertThat(taskIOMetricGroup.getNumBuffersOutCounter().getCount()).isOne(); assertThat(taskIOMetricGroup.getNumBytesOutCounter().getCount()).isEqualTo(bufferSize); IOMetrics ioMetrics = taskIOMetricGroup.createSnapshot(); assertThat(ioMetrics.getResultPartitionBytes()).hasSize(1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionConsumerMemoryDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionConsumerMemoryDataManagerTest.java index 00d89b086c2e9..7b00a70cbf054 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionConsumerMemoryDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionConsumerMemoryDataManagerTest.java @@ -142,14 +142,14 @@ void testAddBuffer() { assertThat(subpartitionConsumerMemoryDataManager.consumeBuffer(0, Collections.emptyList())) .hasValueSatisfying( bufferAndBacklog -> { - assertThat(bufferAndBacklog.getSequenceNumber()).isEqualTo(0); + assertThat(bufferAndBacklog.getSequenceNumber()).isZero(); assertThat(bufferAndBacklog.buffer().getDataType()) .isEqualTo(Buffer.DataType.DATA_BUFFER); }); assertThat(subpartitionConsumerMemoryDataManager.consumeBuffer(1, Collections.emptyList())) .hasValueSatisfying( bufferAndBacklog -> { - assertThat(bufferAndBacklog.getSequenceNumber()).isEqualTo(1); + assertThat(bufferAndBacklog.getSequenceNumber()).isOne(); assertThat(bufferAndBacklog.buffer().getDataType()) .isEqualTo(Buffer.DataType.DATA_BUFFER); }); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java index 1050d3156af85..6120e92a240fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java @@ -182,7 +182,7 @@ void testReadBufferSkip() throws Exception { subpartitionOperation.advanceConsumptionProgress(); subpartitionOperation.advanceConsumptionProgress(); - assertThat(subpartitionOperation.getConsumingOffset(true)).isEqualTo(1); + assertThat(subpartitionOperation.getConsumingOffset(true)).isOne(); // update consumptionProgress subpartitionFileReader.prepareForScheduling(); // read buffer, expected buffer with index: 2 @@ -456,7 +456,7 @@ void testConsumeBuffer() throws Throwable { (bufferAndBacklog -> { assertThat(bufferAndBacklog.getNextDataType()) .isEqualTo(DataType.DATA_BUFFER); - assertThat(bufferAndBacklog.getSequenceNumber()).isEqualTo(0); + assertThat(bufferAndBacklog.getSequenceNumber()).isZero(); // first buffer's data is 0. assertThat( bufferAndBacklog @@ -464,7 +464,7 @@ void testConsumeBuffer() throws Throwable { .getNioBufferReadable() .order(ByteOrder.nativeOrder()) .getInt()) - .isEqualTo(0); + .isZero(); })); // if nextBufferToConsume is less than peek elements index, return Optional.empty. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java index 176d02e649b24..976e8e12af987 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java @@ -144,7 +144,7 @@ void testGetNextBufferFromDiskNextDataTypeIsNone() { TestingHsDataView.builder() .setPeekNextToConsumeDataTypeFunction( (bufferToConsume) -> { - assertThat(bufferToConsume).isEqualTo(1); + assertThat(bufferToConsume).isOne(); return DataType.EVENT_BUFFER; }) .build(); @@ -400,9 +400,9 @@ void testGetConsumingOffset() { assertThat(subpartitionView.getConsumingOffset(true)).isEqualTo(-1); subpartitionView.getNextBuffer(); - assertThat(subpartitionView.getConsumingOffset(true)).isEqualTo(0); + assertThat(subpartitionView.getConsumingOffset(true)).isZero(); subpartitionView.getNextBuffer(); - assertThat(subpartitionView.getConsumingOffset(true)).isEqualTo(1); + assertThat(subpartitionView.getConsumingOffset(true)).isOne(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/index/FileDataIndexCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/index/FileDataIndexCacheTest.java index 76ef63838afc3..6b6e2f8848276 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/index/FileDataIndexCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/index/FileDataIndexCacheTest.java @@ -67,8 +67,8 @@ void testPutAndGet() { assertThat(regionOpt) .hasValueSatisfying( (region) -> { - assertThat(region.getFirstBufferIndex()).isEqualTo(0); - assertThat(region.getRegionStartOffset()).isEqualTo(0); + assertThat(region.getFirstBufferIndex()).isZero(); + assertThat(region.getRegionStartOffset()).isZero(); assertThat(region.getNumBuffers()).isEqualTo(3); }); } @@ -95,7 +95,7 @@ void testCachedRegionRemovedWhenExceedsRetainedEntry(@TempDir Path tmpPath) thro // number of regions exceeds numRetainedIndexEntry, trigger cache purge. indexCache.put(0, createTestRegions(9, 9L, 3, 1)); - assertThat(spilledRegionManager.getSpilledRegionSize(0)).isEqualTo(1); + assertThat(spilledRegionManager.getSpilledRegionSize(0)).isOne(); TestingFileDataIndexRegion region = spilledRegionManager.getRegion(0, 0); assertThat(region).isNotNull(); assertRegionEquals(region, regionList.get(0)); @@ -130,15 +130,15 @@ void testCacheLoadSpilledRegion(@TempDir Path tmpPath) throws Exception { spilledRegionManager = testingSpilledRegionManagerFactory.getLastSpilledRegionManager(); indexCache.put(0, createTestRegions(0, 0L, 1, 2)); - assertThat(spilledRegionManager.getSpilledRegionSize(0)).isEqualTo(1); + assertThat(spilledRegionManager.getSpilledRegionSize(0)).isOne(); assertThat(spilledRegionManager.getFindRegionInvoked()).isZero(); Optional regionOpt = indexCache.get(0, 0); assertThat(spilledRegionManager.getSpilledRegionSize(0)).isEqualTo(2); assertThat(regionOpt).isPresent(); - assertThat(spilledRegionManager.getFindRegionInvoked()).isEqualTo(1); + assertThat(spilledRegionManager.getFindRegionInvoked()).isOne(); // previously get should already load this region to cache. assertThat(indexCache.get(0, 0)).isPresent(); - assertThat(spilledRegionManager.getFindRegionInvoked()).isEqualTo(1); + assertThat(spilledRegionManager.getFindRegionInvoked()).isOne(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java index 5e4507dc521b9..c913f24f61339 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java @@ -84,7 +84,7 @@ void testReadDifferentSegments() throws ExecutionException, InterruptedException reader.readBuffer(0); assertThat(requiredSegmentIdFuture).isNotDone(); reader.readBuffer(1); - assertThat(requiredSegmentIdFuture.get()).isEqualTo(1); + assertThat(requiredSegmentIdFuture.get()).isOne(); } private static Supplier createInputChannelSupplier( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java index 3a15071c91691..c6ca6e919a8a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java @@ -145,8 +145,7 @@ void testCloseWithUnFinishedBuffers() throws IOException { new TieredStorageSubpartitionId(0), Buffer.DataType.DATA_BUFFER, false); - assertThat(tieredStorageMemoryManager.numOwnerRequestedBuffer(bufferAccumulator)) - .isEqualTo(1); + assertThat(tieredStorageMemoryManager.numOwnerRequestedBuffer(bufferAccumulator)).isOne(); bufferAccumulator.close(); assertThat(tieredStorageMemoryManager.numOwnerRequestedBuffer(this)).isZero(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java index 0568832c7e2f2..2702a31506a6b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java @@ -80,11 +80,11 @@ void testRequestAndRecycleBuffers() throws IOException { createStorageMemoryManager( bufferPool, Collections.singletonList(new TieredStorageMemorySpec(this, 0))); - assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(0); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isZero(); BufferBuilder builder = storageMemoryManager.requestBufferBlocking(this); - assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(1); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isOne(); recycleBufferBuilder(builder); - assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(0); + assertThat(bufferPool.bestEffortGetNumOfUsedBuffers()).isZero(); storageMemoryManager.release(); } @@ -198,12 +198,12 @@ void testTriggerReclaimBuffers() throws IOException { requestedBuffers.add(storageMemoryManager.requestBufferBlocking(this)); } - assertThat(reclaimBufferCounter).isEqualTo(0); + assertThat(reclaimBufferCounter).isZero(); assertThat(requestedBuffers.size()).isEqualTo(numBuffersBeforeTriggerReclaim); requestedBuffers.add(storageMemoryManager.requestBufferBlocking(this)); assertThatFuture(hasReclaimBufferFinished).eventuallySucceeds(); - assertThat(reclaimBufferCounter).isEqualTo(1); - assertThat(requestedBuffers.size()).isEqualTo(1); + assertThat(reclaimBufferCounter).isOne(); + assertThat(requestedBuffers.size()).isOne(); recycleRequestedBuffers(); storageMemoryManager.release(); @@ -215,7 +215,7 @@ void testTransferBufferOwnership() throws IOException { createStorageMemoryManager( 1, Collections.singletonList(new TieredStorageMemorySpec(this, 0))); BufferBuilder bufferBuilder = memoryManager.requestBufferBlocking(this); - assertThat(memoryManager.numOwnerRequestedBuffer(this)).isEqualTo(1); + assertThat(memoryManager.numOwnerRequestedBuffer(this)).isOne(); BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumerFromBeginning(); Buffer buffer = bufferConsumer.build(); @@ -223,10 +223,10 @@ void testTransferBufferOwnership() throws IOException { bufferConsumer.close(); Object newOwner = new Object(); memoryManager.transferBufferOwnership(this, newOwner, buffer); - assertThat(memoryManager.numOwnerRequestedBuffer(this)).isEqualTo(0); - assertThat(memoryManager.numOwnerRequestedBuffer(newOwner)).isEqualTo(1); + assertThat(memoryManager.numOwnerRequestedBuffer(this)).isZero(); + assertThat(memoryManager.numOwnerRequestedBuffer(newOwner)).isOne(); buffer.recycleBuffer(); - assertThat(memoryManager.numOwnerRequestedBuffer(newOwner)).isEqualTo(0); + assertThat(memoryManager.numOwnerRequestedBuffer(newOwner)).isZero(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java index 476bbbd58cc78..6ab1e43b9ffb1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java @@ -152,7 +152,7 @@ void testWriteRecords() throws IOException { isBroadcast ? numSubpartitions * numToWriteRecords * bufferSize : numToWriteRecords * bufferSize; - assertThat(numReceivedBuffersInTier1.get()).isEqualTo(1); + assertThat(numReceivedBuffersInTier1.get()).isOne(); assertThat(numReceivedBuffers.get()) .isEqualTo(numReceivedBuffersInTier1.get() + numReceivedBuffersInTier2.get()); assertThat(numReceivedBytes.get()).isEqualTo(numExpectedBytes); From 0317dd9af4144137c2a98eb16d8765f7d4a2f115 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Wed, 16 Aug 2023 20:54:20 +0800 Subject: [PATCH 3/7] Address Rui's comments --- .../io/disk/FileChannelStreamsTest.java | 4 +- .../AsynchronousFileIOChannelTest.java | 4 +- .../io/disk/iomanager/IOManagerAsyncTest.java | 4 +- .../io/disk/iomanager/IOManagerITCase.java | 2 +- .../io/disk/iomanager/IOManagerTest.java | 4 +- .../serialization/EventSerializerTest.java | 3 +- .../api/writer/RecordWriterDelegateTest.java | 8 +-- .../network/buffer/NetworkBufferPoolTest.java | 52 +++++++++---------- .../PartitionRequestServerHandlerTest.java | 6 +-- .../network/partition/AvailabilityUtil.java | 2 +- .../partition/BoundedDataTestBase.java | 5 +- ...sourceManagerPartitionTrackerImplTest.java | 2 +- .../partition/consumer/InputGateTestBase.java | 14 ++--- .../consumer/RemoteInputChannelTest.java | 2 +- .../consumer/UnionInputGateTest.java | 10 ++-- ...ProducerMergedPartitionFileReaderTest.java | 4 +- .../remote/RemoteTierMasterAgentTest.java | 6 +-- 17 files changed, 65 insertions(+), 67 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java index 7e897da163946..711ddbdee832a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java @@ -64,7 +64,7 @@ void testCloseAndDeleteOutputView() { // delete, make sure file is removed out.closeAndDelete(); - assertThat(new File(channel.getPath()).exists()).isFalse(); + assertThat(new File(channel.getPath())).doesNotExist(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -100,7 +100,7 @@ void testCloseAndDeleteInputView() { // delete, make sure file is removed in.closeAndDelete(); - assertThat(new File(channel.getPath()).exists()).isFalse(); + assertThat(new File(channel.getPath())).doesNotExist(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java index 4be054ec474a6..7f5a0984c7580 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java @@ -308,8 +308,8 @@ public void requestFailed(MemorySegment buffer, IOException e) { writer.close(); - assertThat(callbackCounter.get()).isEqualTo(NUM_BLOCKS); - assertThat(exceptionOccurred.get()).isFalse(); + assertThat(callbackCounter).hasValue(NUM_BLOCKS); + assertThat(exceptionOccurred).isFalse(); } finally { writer.closeAndDelete(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java index e1c9de9b46207..ebc3d80c70001 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java @@ -199,7 +199,7 @@ public void read() throws IOException { } } - assertThat(exceptionForwarded.get()).isTrue(); + assertThat(exceptionForwarded).isTrue(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -246,7 +246,7 @@ public void write() throws IOException { } } - assertThat(exceptionForwarded.get()).isTrue(); + assertThat(exceptionForwarded).isTrue(); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index 991d879d43aa9..b757d0dff4827 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -220,7 +220,7 @@ void parallelChannelsTest() throws Exception { // check that files are deleted for (int i = 0; i < NUM_CHANNELS; i++) { File f = new File(ids[i].getPath()); - assertThat(f.exists()).withFailMessage("Channel file has not been deleted.").isFalse(); + assertThat(f).withFailMessage("Channel file has not been deleted.").doesNotExist(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java index c971eacea5181..33dd285a7d455 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java @@ -54,9 +54,9 @@ void channelEnumerator(@TempDir File tempPath) throws Exception { File path = id.getPathFile(); - assertThat(path.isAbsolute()) + assertThat(path) .withFailMessage("Channel IDs must name an absolute path.") - .isTrue(); + .isAbsolute(); assertThat(path.isDirectory()) .withFailMessage("Channel IDs must name a file, not a directory.") .isFalse(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java index 18d20f1b1b119..4a0378b0aafce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java @@ -124,8 +124,7 @@ void testSerializeDeserializeEvent() throws Exception { AbstractEvent deserialized = EventSerializer.fromSerializedEvent( serializedEvent, getClass().getClassLoader()); - assertThat(deserialized).isNotNull(); - assertThat(deserialized).isEqualTo(evt); + assertThat(deserialized).isNotNull().isEqualTo(evt); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java index c41e6824d49ff..2b828a25b7731 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java @@ -135,7 +135,7 @@ private RecordWriter createRecordWriter(NetworkBufferPool globalPool) throws Exc private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exception { // writer is available at the beginning assertThat(writerDelegate.isAvailable()).isTrue(); - assertThat(writerDelegate.getAvailableFuture().isDone()).isTrue(); + assertThat(writerDelegate.getAvailableFuture()).isDone(); // request one buffer from the local pool to make it unavailable RecordWriter recordWriter = writerDelegate.getRecordWriter(0); @@ -144,7 +144,7 @@ private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exce } assertThat(writerDelegate.isAvailable()).isFalse(); CompletableFuture future = writerDelegate.getAvailableFuture(); - assertThat(future.isDone()).isFalse(); + assertThat(future).isNotDone(); // recycle the buffer to make the local pool available again ResultSubpartitionView readView = @@ -154,9 +154,9 @@ private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exce Buffer buffer = readView.getNextBuffer().buffer(); buffer.recycleBuffer(); - assertThat(future.isDone()).isTrue(); + assertThat(future).isDone(); assertThat(writerDelegate.isAvailable()).isTrue(); - assertThat(writerDelegate.getAvailableFuture().isDone()).isTrue(); + assertThat(writerDelegate.getAvailableFuture()).isDone(); } private void verifyBroadcastEvent( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 0590c3cd56739..b42dda7aafd14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -262,12 +262,12 @@ void testInsufficientNumberOfBuffers() throws Exception { try { // the global pool should be in available state initially - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); // request 5 segments List segments1 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); assertThat(segments1).hasSize(numberOfSegmentsToRequest); // request only 1 segment @@ -278,11 +278,11 @@ void testInsufficientNumberOfBuffers() throws Exception { // recycle 5 segments CompletableFuture availableFuture = globalPool.getAvailableFuture(); globalPool.recycleUnpooledMemorySegments(segments1); - assertThat(availableFuture.isDone()).isTrue(); + assertThat(availableFuture).isDone(); List segments2 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); assertThat(segments2).hasSize(numberOfSegmentsToRequest); } finally { globalPool.destroy(); @@ -571,26 +571,26 @@ void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() { try { // the global pool should be in available state initially - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); // request the first segment final MemorySegment segment1 = checkNotNull(globalPool.requestPooledMemorySegment()); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); // request the second segment final MemorySegment segment2 = checkNotNull(globalPool.requestPooledMemorySegment()); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); final CompletableFuture availableFuture = globalPool.getAvailableFuture(); // recycle the first segment globalPool.recyclePooledMemorySegment(segment1); - assertThat(availableFuture.isDone()).isTrue(); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(availableFuture).isDone(); + assertThat(globalPool.getAvailableFuture()).isDone(); // recycle the second segment globalPool.recyclePooledMemorySegment(segment2); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); } finally { globalPool.destroy(); @@ -612,38 +612,38 @@ void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception try { // the global pool should be in available state initially - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); // request 5 segments List segments1 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); assertThat(segments1).hasSize(numberOfSegmentsToRequest); // request another 5 segments List segments2 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); assertThat(segments2).hasSize(numberOfSegmentsToRequest); // recycle 5 segments CompletableFuture availableFuture = globalPool.getAvailableFuture(); globalPool.recycleUnpooledMemorySegments(segments1); - assertThat(availableFuture.isDone()).isTrue(); + assertThat(availableFuture).isDone(); // request another 5 segments final List segments3 = globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); assertThat(segments3).hasSize(numberOfSegmentsToRequest); // recycle another 5 segments globalPool.recycleUnpooledMemorySegments(segments2); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); // recycle the last 5 segments globalPool.recycleUnpooledMemorySegments(segments3); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); } finally { globalPool.destroy(); @@ -672,7 +672,7 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt final BufferPool localPool = globalPool.createBufferPool(localPoolRequiredSize, localPoolMaxSize); localBufferPools.add(localPool); - assertThat(localPool.getAvailableFuture().isDone()).isTrue(); + assertThat(localPool.getAvailableFuture()).isDone(); } // request some segments from the global pool in two different ways @@ -683,9 +683,9 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt final List exclusiveSegments = globalPool.requestUnpooledMemorySegments( globalPool.getNumberOfAvailableMemorySegments() - 1); - assertThat(globalPool.getAvailableFuture().isDone()).isTrue(); + assertThat(globalPool.getAvailableFuture()).isDone(); for (final BufferPool localPool : localBufferPools) { - assertThat(localPool.getAvailableFuture().isDone()).isTrue(); + assertThat(localPool.getAvailableFuture()).isDone(); } // blocking request buffers form local buffer pools @@ -716,14 +716,14 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt } final CompletableFuture globalPoolAvailableFuture = globalPool.getAvailableFuture(); - assertThat(globalPoolAvailableFuture.isDone()).isFalse(); + assertThat(globalPoolAvailableFuture).isNotDone(); final List> localPoolAvailableFutures = new ArrayList<>(numLocalBufferPool); for (BufferPool localPool : localBufferPools) { CompletableFuture localPoolAvailableFuture = localPool.getAvailableFuture(); localPoolAvailableFutures.add(localPoolAvailableFuture); - assertThat(localPoolAvailableFuture.isDone()).isFalse(); + assertThat(localPoolAvailableFuture).isNotDone(); } // recycle the previously requested segments @@ -732,9 +732,9 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt } globalPool.recycleUnpooledMemorySegments(exclusiveSegments); - assertThat(globalPoolAvailableFuture.isDone()).isTrue(); + assertThat(globalPoolAvailableFuture).isDone(); for (CompletableFuture localPoolAvailableFuture : localPoolAvailableFutures) { - assertThat(localPoolAvailableFuture.isDone()).isTrue(); + assertThat(localPoolAvailableFuture).isDone(); } // wait until all blocking buffer requests finish @@ -742,9 +742,9 @@ void testBlockingRequestFromMultiLocalBufferPool() throws IOException, Interrupt assertThat(cause.get()).isNull(); assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero(); - assertThat(globalPool.getAvailableFuture().isDone()).isFalse(); + assertThat(globalPool.getAvailableFuture()).isNotDone(); for (BufferPool localPool : localBufferPools) { - assertThat(localPool.getAvailableFuture().isDone()).isFalse(); + assertThat(localPool.getAvailableFuture()).isNotDone(); assertThat(localPool.bestEffortGetNumOfUsedBuffers()).isEqualTo(localPoolMaxSize); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java index e1075b9766f5b..e262bba99f8fb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java @@ -126,12 +126,12 @@ void testAcknowledgeAllRecordsProcessed() throws IOException { resultPartition.notifyEndOfData(StopMode.DRAIN); CompletableFuture allRecordsProcessedFuture = resultPartition.getAllDataProcessedFuture(); - assertThat(allRecordsProcessedFuture.isDone()).isFalse(); + assertThat(allRecordsProcessedFuture).isNotDone(); channel.writeInbound(new NettyMessage.AckAllUserRecordsProcessed(inputChannelID)); channel.runPendingTasks(); - assertThat(allRecordsProcessedFuture.isDone()).isTrue(); - assertThat(allRecordsProcessedFuture.isCompletedExceptionally()).isFalse(); + assertThat(allRecordsProcessedFuture).isDone(); + assertThat(allRecordsProcessedFuture).isNotCompletedExceptionally(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java index fce757a10dfb4..4ff73f7561bd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AvailabilityUtil.java @@ -40,7 +40,7 @@ public static void assertFutureCompletion( runnable.run(); assertThat(futureSupplier.get().isDone()).isEqualTo(after); if (after) { - assertThat(availableFuture.isDone()).isTrue(); + assertThat(availableFuture).isDone(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java index 72ed97cdf31be..62628b9641080 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java @@ -32,7 +32,6 @@ import org.junit.jupiter.api.io.TempDir; import java.io.IOException; -import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.List; @@ -137,11 +136,11 @@ public void returnNullAfterEmpty() throws Exception { @TestTemplate public void testDeleteFileOnClose() throws Exception { final BoundedData bd = createBoundedData(subpartitionDataPath); - assertThat(Files.exists(subpartitionDataPath)).isTrue(); + assertThat(subpartitionDataPath).exists(); bd.close(); - assertThat(Files.exists(subpartitionDataPath)).isFalse(); + assertThat(subpartitionDataPath).doesNotExist(); } @TestTemplate diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java index 5ab44fe82b16a..db8ad8aa6a91a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java @@ -153,7 +153,7 @@ void testReleasePartition() { // ack the partition release reportEmpty(tracker, TASK_EXECUTOR_ID_1, TASK_EXECUTOR_ID_2); - assertThat(partitionReleaseFuture.isDone()).isTrue(); + assertThat(partitionReleaseFuture).isDone(); assertThat(tracker.areAllMapsEmpty()).isTrue(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java index 9415d0b756785..98a334d94a02e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java @@ -44,12 +44,12 @@ protected void testIsAvailable( TestInputChannel inputChannelWithNewData) throws Exception { - assertThat(inputGateToTest.getAvailableFuture().isDone()).isFalse(); + assertThat(inputGateToTest.getAvailableFuture()).isNotDone(); assertThat(inputGateToTest.pollNext()).isEmpty(); CompletableFuture future = inputGateToTest.getAvailableFuture(); - assertThat(inputGateToTest.getAvailableFuture().isDone()).isFalse(); + assertThat(inputGateToTest.getAvailableFuture()).isNotDone(); assertThat(inputGateToTest.pollNext()).isEmpty(); assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(future); @@ -57,8 +57,8 @@ protected void testIsAvailable( inputChannelWithNewData.readBuffer(); inputGateToNotify.notifyChannelNonEmpty(inputChannelWithNewData); - assertThat(future.isDone()).isTrue(); - assertThat(inputGateToTest.getAvailableFuture().isDone()).isTrue(); + assertThat(future).isDone(); + assertThat(inputGateToTest.getAvailableFuture()).isDone(); assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(PullingAsyncDataInput.AVAILABLE); } @@ -66,15 +66,15 @@ protected void testIsAvailableAfterFinished( InputGate inputGateToTest, Runnable endOfPartitionEvent) throws Exception { CompletableFuture available = inputGateToTest.getAvailableFuture(); - assertThat(available.isDone()).isFalse(); + assertThat(available).isNotDone(); assertThat(inputGateToTest.pollNext()).isEmpty(); endOfPartitionEvent.run(); assertThat(inputGateToTest.pollNext()).isNotEmpty(); // EndOfPartitionEvent - assertThat(available.isDone()).isTrue(); - assertThat(inputGateToTest.getAvailableFuture().isDone()).isTrue(); + assertThat(available).isDone(); + assertThat(inputGateToTest.getAvailableFuture()).isDone(); assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(PullingAsyncDataInput.AVAILABLE); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 77535782759f0..164d33368bd68 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -154,7 +154,7 @@ void testGateNotifiedOnBarrierConversion() throws IOException, InterruptedExcept inputGate.pollNext(); // process announcement to allow the gate remember the SQN channel.convertToPriorityEvent(sequenceNumber); - assertThat(inputGate.getPriorityEventAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getPriorityEventAvailableFuture()).isDone(); } finally { networkBufferPool.destroy(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 5724060dd1cc6..2fccb1b19397b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -203,11 +203,11 @@ void testAvailability() throws IOException, InterruptedException { UnionInputGate inputGate = new UnionInputGate(inputGate1, inputGate2); inputChannel1.read(BufferBuilderTestUtils.buildSomeBuffer(1)); - assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getAvailableFuture()).isDone(); inputChannel1.read(BufferBuilderTestUtils.buildSomeBuffer(2)); - assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getAvailableFuture()).isDone(); assertThat(inputGate.getNext().get().getBuffer().getSize()).isOne(); - assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getAvailableFuture()).isDone(); } @Test @@ -288,8 +288,8 @@ void testEmptyPull() throws IOException, InterruptedException { UnionInputGate inputGate = new UnionInputGate(inputGate1, inputGate2); inputChannel1.notifyChannelNonEmpty(); - assertThat(inputGate.getAvailableFuture().isDone()).isTrue(); + assertThat(inputGate.getAvailableFuture()).isDone(); assertThat(inputGate.pollNext()).isEmpty(); - assertThat(inputGate.getAvailableFuture().isDone()).isFalse(); + assertThat(inputGate.getAvailableFuture()).isNotDone(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileReaderTest.java index 04e808586fc74..e4a72062c6565 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileReaderTest.java @@ -196,9 +196,9 @@ void testReadProgress() throws IOException { @Test void testRelease() { - assertThat(testFilePath.toFile().exists()).isTrue(); + assertThat(testFilePath.toFile()).exists(); partitionFileReader.release(); - assertThat(testFilePath.toFile().exists()).isFalse(); + assertThat(testFilePath.toFile()).doesNotExist(); } private List readBuffer(int bufferIndex, TieredStorageSubpartitionId subpartitionId) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierMasterAgentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierMasterAgentTest.java index 289daaf8385bb..a6825198456e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierMasterAgentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/remote/RemoteTierMasterAgentTest.java @@ -43,15 +43,15 @@ void testAddAndReleasePartition() throws IOException { TieredStorageIdMappingUtils.convertId(new ResultPartitionID()); File partitionFile = new File(getPartitionPath(partitionId, tempFolder.getAbsolutePath())); assertThat(partitionFile.createNewFile()).isTrue(); - assertThat(partitionFile.exists()).isTrue(); + assertThat(partitionFile).exists(); TieredStorageResourceRegistry resourceRegistry = new TieredStorageResourceRegistry(); RemoteTierMasterAgent masterAgent = new RemoteTierMasterAgent(tempFolder.getAbsolutePath(), resourceRegistry); masterAgent.addPartition(partitionId); - assertThat(partitionFile.exists()).isTrue(); + assertThat(partitionFile).exists(); masterAgent.releasePartition(partitionId); - assertThat(partitionFile.exists()).isFalse(); + assertThat(partitionFile).doesNotExist(); } } From aef534b4b92834951171869cc5fbc3f9cfd8f0e1 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sat, 19 Aug 2023 01:17:51 +0800 Subject: [PATCH 4/7] Address Rui's comments --- .../io/disk/FileChannelManagerImplTest.java | 2 +- .../io/disk/iomanager/IOManagerTest.java | 6 ++-- .../network/buffer/AbstractByteBufTest.java | 12 +++---- .../network/buffer/NetworkBufferPoolTest.java | 4 +-- ...NettyMessageClientDecoderDelegateTest.java | 2 +- .../PartitionRequestClientFactoryTest.java | 2 +- .../io/network/partition/DataBufferTest.java | 2 +- .../partition/InputGateFairnessTest.java | 4 +-- .../partition/PipelinedSubpartitionTest.java | 3 +- .../SortMergeSubpartitionReaderTest.java | 2 +- .../TaskExecutorPartitionTrackerImplTest.java | 23 +++++++++--- .../consumer/RemoteInputChannelTest.java | 2 +- .../consumer/SingleInputGateTest.java | 6 ++-- .../hybrid/HsFileDataManagerTest.java | 36 ++++++++++++------- .../hybrid/HsResultPartitionTest.java | 2 +- .../tiered/file/SegmentPartitionFileTest.java | 2 +- .../netty/NettyConnectionReaderTest.java | 3 +- .../storage/HashBufferAccumulatorTest.java | 2 +- .../TieredStorageMemoryManagerImplTest.java | 4 +-- .../TieredStorageProducerClientTest.java | 16 ++++----- .../storage/TieredStorageSortBufferTest.java | 2 +- 21 files changed, 82 insertions(+), 55 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java index 0454e52b89c9b..7715b46d7fed4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelManagerImplTest.java @@ -90,7 +90,7 @@ void testFairness() throws Exception { threads[i].join(); } - assertThat(counter2.get()).isEqualTo(counter1.get()); + assertThat(counter2).hasValue(counter1.get()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java index 33dd285a7d455..3ff821494b643 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java @@ -27,6 +27,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; @@ -53,13 +54,14 @@ void channelEnumerator(@TempDir File tempPath) throws Exception { FileIOChannel.ID id = enumerator.next(); File path = id.getPathFile(); + Files.createFile(id.getPathFile().toPath()); assertThat(path) .withFailMessage("Channel IDs must name an absolute path.") .isAbsolute(); - assertThat(path.isDirectory()) + assertThat(path) .withFailMessage("Channel IDs must name a file, not a directory.") - .isFalse(); + .isFile(); assertThat(path.getParentFile().getParentFile().getParentFile()) .withFailMessage("Path is not in the temp directory.") diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index 9fcffa5361656..bf5d38c305400 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -2239,7 +2239,7 @@ void testHashCode() { set.add(elemA); set.add(elemB); - assertThat(set.size()).isEqualTo(2); + assertThat(set).hasSize(2); ByteBuf elemACopy = elemA.copy(); assertThat(set.contains(elemACopy)).isTrue(); @@ -2250,14 +2250,14 @@ void testHashCode() { buffer.writeBytes(elemA.duplicate()); assertThat(set.remove(buffer)).isTrue(); - assertThat(set.contains(elemA)).isFalse(); - assertThat(set.size()).isOne(); + assertThat(set).doesNotContainSequence(elemA); + assertThat(set).hasSize(1); buffer.clear(); buffer.writeBytes(elemB.duplicate()); assertThat(set.remove(buffer)).isTrue(); - assertThat(set.contains(elemB)).isFalse(); - assertThat(set.size()).isZero(); + assertThat(set).doesNotContainSequence(elemB); + assertThat(set).isEmpty(); elemA.release(); elemB.release(); elemACopy.release(); @@ -4900,7 +4900,7 @@ public void run() { t1.start(); latch.await(); - assertThat(cnt.get()).isZero(); + assertThat(cnt).hasValue(0); innerLatch.countDown(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index b42dda7aafd14..5cea208cdaae6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -163,7 +163,7 @@ void testDestroyAll() throws IOException { } } - assertThat(buffers.size()).isEqualTo(globalPool.getTotalNumberOfMemorySegments()); + assertThat(buffers).hasSize(globalPool.getTotalNumberOfMemorySegments()); assertThat(fixedPool.requestBuffer()).isNull(); assertThat(boundedPool.requestBuffer()).isNull(); @@ -220,7 +220,7 @@ void testRequestMemorySegmentsLessThanTotalBuffers() throws IOException { List memorySegments = Collections.emptyList(); try { memorySegments = globalPool.requestUnpooledMemorySegments(numBuffers / 2); - assertThat(memorySegments.size()).isEqualTo(numBuffers / 2); + assertThat(memorySegments).hasSize(numBuffers / 2); globalPool.recycleUnpooledMemorySegments(memorySegments); memorySegments.clear(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java index ceb4b63bfa6bc..0b008d6300f6f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java @@ -324,7 +324,7 @@ private List decodeMessages(EmbeddedChannel channel, List private void verifyDecodedMessages( List expectedMessages, List decodedMessages) { - assertThat(decodedMessages.size()).isEqualTo(expectedMessages.size()); + assertThat(decodedMessages).hasSameSizeAs(expectedMessages); for (int i = 0; i < expectedMessages.size(); ++i) { assertThat(decodedMessages.get(i)).isInstanceOf(expectedMessages.get(i).getClass()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java index 0b3e206576ecb..05dd16aeb1a78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java @@ -162,7 +162,7 @@ private void checkReuseNettyPartitionRequestClient( RESOURCE_ID, (int) (Math.random() * Integer.MAX_VALUE)); set.add(factory.createPartitionRequestClient(connectionID)); } - assertThat(set.size()).isLessThanOrEqualTo(maxNumberOfConnections); + assertThat(set).hasSizeLessThanOrEqualTo(maxNumberOfConnections); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java index 009b543e49d5a..e936f2d277763 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java @@ -204,7 +204,7 @@ public static void checkWriteReadResult( subpartitionDataRead.flip(); assertThat(subpartitionDataRead).isEqualTo(subpartitionDataWritten); - assertThat(eventsRead.size()).isEqualTo(eventsWritten.size()); + assertThat(eventsRead).hasSameSizeAs(eventsWritten); for (int i = 0; i < eventsWritten.size(); ++i) { assertThat(eventsRead.get(i).getDataType()) .isEqualTo(eventsWritten.get(i).dataType); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index 4f3e154f891ad..5b8a062c3a012 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -411,9 +411,9 @@ private void ensureUnique(Collection channels) { } } - assertThat(uniquenessChecker.size()) + assertThat(uniquenessChecker) .withFailMessage("found duplicate input channels") - .isEqualTo(channels.size()); + .hasSameSizeAs(channels); uniquenessChecker.clear(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 42726ef8900bf..b3e7a30e708f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -54,6 +54,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledFinishedBufferConsumer; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -441,7 +442,7 @@ private void testTimeoutWithNDataBuffers( pollBufferAndCheckType(subpartition, Buffer.DataType.DATA_BUFFER); } - assertThat(channelStateFuture.get()).isEqualTo(expectedBuffers); + assertThatFuture(channelStateFuture).eventuallySucceeds().isEqualTo(expectedBuffers); } private void pollBufferAndCheckType( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java index 2d15672dff2e6..93a2e95ba90f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeSubpartitionReaderTest.java @@ -115,7 +115,7 @@ void testReadBuffers() throws Exception { assertThat(listener.numNotifications).isEqualTo(2); assertThat(subpartitionReader.unsynchronizedGetNumberOfQueuedBuffers()) .isEqualTo(numBuffersPerSubpartition - 2); - assertThat(segments.size()).isOne(); + assertThat(segments).hasSize(1); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java index 2c25f3cad96f1..225062e221e48 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImplTest.java @@ -43,6 +43,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link TaskExecutorPartitionTrackerImpl}. */ @@ -113,7 +114,9 @@ void testStopTrackingAndReleaseJobPartitions() throws Exception { partitionTracker.stopTrackingAndReleaseJobPartitions( Collections.singleton(resultPartitionId1)); - assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); + assertThatFuture(shuffleReleaseFuture) + .eventuallySucceeds() + .satisfies(actual -> assertThat(actual).containsExactly(resultPartitionId1)); } @Test @@ -144,7 +147,10 @@ void testStopTrackingAndReleaseJobPartitionsFor() throws Exception { 1)); partitionTracker.stopTrackingAndReleaseJobPartitionsFor(jobId1); - assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); + assertThatFuture(shuffleReleaseFuture) + .eventuallySucceeds() + .asList() + .contains(resultPartitionId1); } @Test @@ -175,7 +181,10 @@ void promoteJobPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseJobPartitionsFor(jobId); - assertThat(shuffleReleaseFuture.get()).doesNotContain(resultPartitionId1); + assertThatFuture(shuffleReleaseFuture) + .eventuallySucceeds() + .asList() + .doesNotContain(resultPartitionId1); } @Test @@ -205,7 +214,9 @@ void stopTrackingAndReleaseAllClusterPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseAllClusterPartitions(); - assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); + assertThatFuture(shuffleReleaseFuture) + .eventuallySucceeds() + .satisfies(actual -> assertThat(actual).contains(resultPartitionId1)); } @Test @@ -234,7 +245,9 @@ void stopTrackingAndReleaseClusterPartitions() throws Exception { partitionTracker.promoteJobPartitions(Collections.singleton(resultPartitionId1)); partitionTracker.stopTrackingAndReleaseClusterPartitions(Collections.singleton(dataSetId1)); - assertThat(shuffleReleaseFuture.get()).contains(resultPartitionId1); + assertThatFuture(shuffleReleaseFuture) + .eventuallySucceeds() + .satisfies(actual -> assertThat(actual).contains(resultPartitionId1)); } private static class TestingShuffleEnvironment diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 164d33368bd68..6f91470798ff9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -1790,7 +1790,7 @@ public PartitionRequestClient createPartitionRequestClient( simulatedNetworkThread.join(); - assertThat(timedOutOrInterrupted.get()) + assertThat(timedOutOrInterrupted) .withFailMessage( "Test ended by timeout or interruption - this indicates that the network thread was blocked.") .isFalse(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 28b3d3223ed21..1c49e3b65c346 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -661,7 +661,7 @@ void testRequestBackoffConfiguration() throws Exception { Map channelMap = gate.getInputChannels(); - assertThat(channelMap.size()).isEqualTo(3); + assertThat(channelMap).hasSize(3); channelMap .values() .forEach( @@ -932,7 +932,7 @@ void testSingleInputGateWithSubpartitionIndexRange() throws IOException, Interru SubpartitionInfo info5 = createSubpartitionInfo(partitionIds[2], 0); SubpartitionInfo info6 = createSubpartitionInfo(partitionIds[2], 1); - assertThat(gate.getInputChannels().size()).isEqualTo(6); + assertThat(gate.getInputChannels()).hasSize(6); assertThat(gate.getInputChannels().get(info1).getConsumedSubpartitionIndex()).isZero(); assertThat(gate.getInputChannels().get(info2).getConsumedSubpartitionIndex()).isOne(); assertThat(gate.getInputChannels().get(info3).getConsumedSubpartitionIndex()).isZero(); @@ -1098,7 +1098,7 @@ void testInputGateRemovalFromNettyShuffleEnvironment() throws Exception { Map createdInputGatesById = createInputGateWithLocalChannels(network, numberOfGates, 1); - assertThat(createdInputGatesById.size()).isEqualTo(numberOfGates); + assertThat(createdInputGatesById).hasSize(numberOfGates); for (InputGateID id : createdInputGatesById.keySet()) { assertThat(network.getInputGate(id).isPresent()).isTrue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java index e88b27dd960b1..9c0f1ab597e5d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java @@ -51,6 +51,7 @@ import java.util.concurrent.TimeoutException; import java.util.function.Consumer; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; import static org.apache.flink.runtime.io.network.partition.hybrid.HsConsumerId.DEFAULT; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -256,11 +257,13 @@ void testRunRequestBufferTimeout(@TempDir Path tempDir) throws Exception { // second round run will trigger timeout. fileDataManager.run(); - assertThat(prepareForSchedulingFinished).isCompleted(); - assertThat(cause).isCompleted(); - assertThat(cause.get()) - .isInstanceOf(TimeoutException.class) - .hasMessageContaining("Buffer request timeout"); + assertThatFuture(cause) + .eventuallySucceeds() + .satisfies( + e -> + assertThat(e) + .isInstanceOf(TimeoutException.class) + .hasMessageContaining("Buffer request timeout")); } /** @@ -282,10 +285,13 @@ void testRunReadBuffersThrowException() throws Exception { ioExecutor.trigger(); - assertThat(cause).isCompleted(); - assertThat(cause.get()) - .isInstanceOf(IOException.class) - .hasMessageContaining("expected exception."); + assertThatFuture(cause) + .eventuallySucceeds() + .satisfies( + e -> + assertThat(e) + .isInstanceOf(IOException.class) + .hasMessageContaining("expected exception.")); } // ----------------------- test release --------------------------------------- @@ -329,10 +335,14 @@ public void go() throws Exception { releaseThread.sync(); - assertThat(cause).isCompleted(); - assertThat(cause.get()) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("Result partition has been already released."); + assertThatFuture(cause) + .eventuallySucceeds() + .satisfies( + e -> + assertThat(e) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "Result partition has been already released.")); } /** Test file data manager was released, but receive new subpartition reader registration. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java index 0751d40abdd50..6d5a7f57bb1b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartitionTest.java @@ -669,7 +669,7 @@ private static void checkWriteReadResult( subpartitionDataRead.flip(); assertThat(subpartitionDataWritten).isEqualTo(subpartitionDataRead); - assertThat(eventsWritten.size()).isEqualTo(eventsRead.size()); + assertThat(eventsWritten).hasSameSizeAs(eventsRead); for (int i = 0; i < eventsWritten.size(); i++) { assertThat(eventsWritten.get(i).f1).isEqualTo(eventsRead.get(i).getDataType()); assertThat(eventsWritten.get(i).f0) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SegmentPartitionFileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SegmentPartitionFileTest.java index 889289d079ed5..ef0989b2afc06 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SegmentPartitionFileTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/SegmentPartitionFileTest.java @@ -174,7 +174,7 @@ private static void writeAndCheckSegmentFinishFile( SegmentPartitionFile.writeSegmentFinishFile( baseDir, partitionId, subpartitionId, segmentId); File segmentFinishDir = getSegmentFinishDir(baseDir, partitionId, subpartitionId); - assertThat(segmentFinishDir.isDirectory()).isTrue(); + assertThat(segmentFinishDir).isDirectory(); File[] segmentFinishFiles = segmentFinishDir.listFiles(); assertThat(segmentFinishFiles).hasSize(1); assertThat(segmentFinishFiles[0]).isFile(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java index c913f24f61339..bf9f88f2c896e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java @@ -36,6 +36,7 @@ import java.util.concurrent.ExecutionException; import java.util.function.Supplier; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType.DATA_BUFFER; import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType.NONE; import static org.assertj.core.api.Assertions.assertThat; @@ -84,7 +85,7 @@ void testReadDifferentSegments() throws ExecutionException, InterruptedException reader.readBuffer(0); assertThat(requiredSegmentIdFuture).isNotDone(); reader.readBuffer(1); - assertThat(requiredSegmentIdFuture.get()).isOne(); + assertThatFuture(requiredSegmentIdFuture).eventuallySucceeds().isEqualTo(1); } private static Supplier createInputChannelSupplier( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java index c6ca6e919a8a5..b188a2b21c5d3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/HashBufferAccumulatorTest.java @@ -103,7 +103,7 @@ record = EventSerializer.toSerializedEvent(EndOfPartitionEvent.INSTANCE); bufferAccumulator.receive(record, subpartitionId, dataType, false); } - assertThat(numReceivedFinishedBuffer.get()).isEqualTo(numExpectBuffers); + assertThat(numReceivedFinishedBuffer).hasValue(numExpectBuffers); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java index 2702a31506a6b..1f03f24050a70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java @@ -199,11 +199,11 @@ void testTriggerReclaimBuffers() throws IOException { } assertThat(reclaimBufferCounter).isZero(); - assertThat(requestedBuffers.size()).isEqualTo(numBuffersBeforeTriggerReclaim); + assertThat(requestedBuffers).hasSize(numBuffersBeforeTriggerReclaim); requestedBuffers.add(storageMemoryManager.requestBufferBlocking(this)); assertThatFuture(hasReclaimBufferFinished).eventuallySucceeds(); assertThat(reclaimBufferCounter).isOne(); - assertThat(requestedBuffers.size()).isOne(); + assertThat(requestedBuffers).hasSize(1); recycleRequestedBuffers(); storageMemoryManager.release(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java index 6ab1e43b9ffb1..fd91b53d8eaa6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java @@ -152,10 +152,10 @@ void testWriteRecords() throws IOException { isBroadcast ? numSubpartitions * numToWriteRecords * bufferSize : numToWriteRecords * bufferSize; - assertThat(numReceivedBuffersInTier1.get()).isOne(); - assertThat(numReceivedBuffers.get()) - .isEqualTo(numReceivedBuffersInTier1.get() + numReceivedBuffersInTier2.get()); - assertThat(numReceivedBytes.get()).isEqualTo(numExpectedBytes); + assertThat(numReceivedBuffersInTier1).hasValue(1); + assertThat(numReceivedBuffers) + .hasValue(numReceivedBuffersInTier1.get() + numReceivedBuffersInTier2.get()); + assertThat(numReceivedBytes).hasValue(numExpectedBytes); } @TestTemplate @@ -215,8 +215,8 @@ void testUpdateMetrics() throws IOException { int numExpectedBuffers = isBroadcast ? numSubpartitions : 1; int numExpectedBytes = isBroadcast ? bufferSize * numSubpartitions : bufferSize; - assertThat(numWriteBuffers.get()).isEqualTo(numExpectedBuffers); - assertThat(numWriteBytes.get()).isEqualTo(numExpectedBytes); + assertThat(numWriteBuffers).hasValue(numExpectedBuffers); + assertThat(numWriteBytes).hasValue(numExpectedBytes); } @TestTemplate @@ -233,9 +233,9 @@ void testClose() { createTieredStorageProducerClient( numSubpartitions, Collections.singletonList(tierProducerAgent)); - assertThat(isClosed.get()).isFalse(); + assertThat(isClosed).isFalse(); tieredStorageProducerClient.close(); - assertThat(isClosed.get()).isTrue(); + assertThat(isClosed).isTrue(); } private static TieredStorageProducerClient createTieredStorageProducerClient( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageSortBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageSortBufferTest.java index 346875cd1654a..340ef466142c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageSortBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageSortBufferTest.java @@ -239,7 +239,7 @@ private static void checkWriteReadResult( subpartitionDataRead.flip(); assertThat(subpartitionDataWritten).isEqualTo(subpartitionDataRead); - assertThat(eventsWritten.size()).isEqualTo(eventsRead.size()); + assertThat(eventsWritten).hasSameSizeAs(eventsRead); for (int i = 0; i < eventsWritten.size(); ++i) { assertThat(eventsWritten.get(i).dataType) .isEqualTo(eventsRead.get(i).getDataType()); From 78d4a02b453445abad096bbdf98e7048c97a88b0 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sun, 20 Aug 2023 14:29:03 +0800 Subject: [PATCH 5/7] Address Zhanghao's comments --- .../io/network/buffer/AbstractByteBufTest.java | 2 +- .../io/network/buffer/LocalBufferPoolTest.java | 2 +- .../netty/ClientTransportErrorHandlingTest.java | 13 ++++--------- .../partition/PipelinedSubpartitionTest.java | 2 +- .../partition/ProducerFailedExceptionTest.java | 3 +-- .../io/network/partition/ResultPartitionTest.java | 2 +- .../partition/SortMergeResultPartitionTest.java | 10 +++++----- .../file/ProducerMergedPartitionFileWriterTest.java | 2 +- 8 files changed, 15 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index bf5d38c305400..1d8875215a6f4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -4658,7 +4658,7 @@ void testEmptyNioBuffers() throws Exception { buffer.clear(); assertThat(buffer.isReadable()).isFalse(); ByteBuffer[] nioBuffers = buffer.nioBuffers(); - assertThat(nioBuffers.length).isOne(); + assertThat(nioBuffers).hasSize(1); assertThat(nioBuffers[0].hasRemaining()).isFalse(); buffer.release(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index c161a6fff3758..ec9f7e5bef75b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -65,7 +65,7 @@ class LocalBufferPoolTest { private BufferPool localBufferPool; @RegisterExtension - static final TestExecutorExtension EXECUTOR_EXTENSION = + private static final TestExecutorExtension EXECUTOR_EXTENSION = new TestExecutorExtension<>(Executors::newCachedThreadPool); @BeforeEach diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java index 47d312f5ffaf0..0d59bad550096 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/ClientTransportErrorHandlingTest.java @@ -325,15 +325,10 @@ void testConnectionResetByPeer() throws Throwable { try { assertThat(cause) - .isInstanceOf(RemoteTransportException.class); - assertThat(cause) - .hasMessageNotContaining( - "Connection reset by peer"); - - assertThat(cause.getCause()) - .isInstanceOf(IOException.class); - assertThat(cause.getCause()) - .hasMessage("Connection reset by peer"); + .hasMessageNotContaining("Connection reset by peer") + .isInstanceOf(RemoteTransportException.class) + .hasRootCauseInstanceOf(IOException.class) + .hasRootCauseMessage("Connection reset by peer"); } catch (Throwable t) { error[0] = t; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index b3e7a30e708f6..afe458e4157fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -75,7 +75,7 @@ public class PipelinedSubpartitionTest extends SubpartitionTestBase { /** Executor service for concurrent produce/consume tests. */ @RegisterExtension - static final TestExecutorExtension EXECUTOR_EXTENSION = + private static final TestExecutorExtension EXECUTOR_EXTENSION = new TestExecutorExtension<>(Executors::newCachedThreadPool); @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java index 1ccc6d7ffbf95..16d20fa800868 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java @@ -40,7 +40,6 @@ void testCauseIsSerialized() throws Exception { // of a user level Exception, which can not be deserialized by the // remote receiver's system class loader. ProducerFailedException e = new ProducerFailedException(new Exception()); - assertThat(e.getCause()).isNotNull(); - assertThat(e.getCause()).isInstanceOf(SerializedThrowable.class); + assertThat(e).hasCauseInstanceOf(SerializedThrowable.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 5b0efe37c29e6..65df6880f4128 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -560,7 +560,7 @@ void testDifferentBufferSizeForSubpartitions() throws IOException { createResultPartition(ResultPartitionType.PIPELINED_BOUNDED); ResultSubpartition[] subpartitions = bufferWritingResultPartition.subpartitions; - assertThat(subpartitions.length).isEqualTo(2); + assertThat(subpartitions).hasSize(2); PipelinedSubpartition subpartition0 = (PipelinedSubpartition) subpartitions[0]; PipelinedSubpartition subpartition1 = (PipelinedSubpartition) subpartitions[1]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java index 6a1bc591e3a47..d2e9be9b27ba9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java @@ -303,7 +303,7 @@ void testDataBroadcast() throws Exception { int eventSize = EventSerializer.toSerializedEvent(EndOfPartitionEvent.INSTANCE).remaining(); long dataSize = numSubpartitions * numRecords * bufferSize + numSubpartitions * eventSize; assertThat(partition.getResultFile()).isNotNull(); - assertThat(checkNotNull(fileChannelManager.getPaths()[0].list()).length).isEqualTo(2); + assertThat(checkNotNull(fileChannelManager.getPaths()[0].list())).hasSize(2); for (File file : checkNotNull(fileChannelManager.getPaths()[0].listFiles())) { if (file.getName().endsWith(PartitionedFile.DATA_FILE_SUFFIX)) { assertThat(file.length()).isLessThan(numSubpartitions * numRecords * bufferSize); @@ -333,14 +333,14 @@ void testReleaseWhileWriting() throws Exception { partition.emitRecord(ByteBuffer.allocate(bufferSize), 2); assertThat(partition.getResultFile()).isNull(); - assertThat(fileChannelManager.getPaths()[0].list().length).isEqualTo(2); + assertThat(fileChannelManager.getPaths()[0].list()).hasSize(2); partition.release(); assertThatThrownBy( () -> partition.emitRecord(ByteBuffer.allocate(bufferSize * numBuffers), 2)) .isInstanceOf(IllegalStateException.class); - assertThat(fileChannelManager.getPaths()[0].list().length).isZero(); + assertThat(fileChannelManager.getPaths()[0].list()).isEmpty(); } @TestTemplate @@ -357,7 +357,7 @@ void testRelease() throws Exception { partition.close(); assertThat(partition.getResultFile().getNumRegions()).isEqualTo(3); - assertThat(checkNotNull(fileChannelManager.getPaths()[0].list()).length).isEqualTo(2); + assertThat(checkNotNull(fileChannelManager.getPaths()[0].list())).hasSize(2); ResultSubpartitionView view = partition.createSubpartitionView(0, listener); partition.release(); @@ -373,7 +373,7 @@ void testRelease() throws Exception { while (partition.getResultFile() != null) { Thread.sleep(100); } - assertThat(checkNotNull(fileChannelManager.getPaths()[0].list()).length).isZero(); + assertThat(checkNotNull(fileChannelManager.getPaths()[0].list())).isEmpty(); } @TestTemplate diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileWriterTest.java index be93b80b72b3d..ce45f65587991 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/file/ProducerMergedPartitionFileWriterTest.java @@ -76,7 +76,7 @@ void testWrite() throws IOException { int numExpectedBytes = numExpectedBuffers * (BufferReaderWriterUtil.HEADER_LENGTH + bufferSizeBytes); assertThat(receivedBuffers).hasValue(numExpectedBuffers); - assertThat(bytesRead.length).isEqualTo(numExpectedBytes); + assertThat(bytesRead).hasSize(numExpectedBytes); } @Test From 557e0cf3a4e82b1ec10e0c796de8c63dc09c5800 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Mon, 28 Aug 2023 12:17:30 +0800 Subject: [PATCH 6/7] Address Sergey's comments --- .../disk/BatchShuffleReadBufferPoolTest.java | 3 ++- .../io/network/buffer/AbstractByteBufTest.java | 6 +++--- .../network/buffer/BufferCompressionTest.java | 2 +- .../io/network/buffer/LocalBufferPoolTest.java | 7 ++++--- ...BasedPartitionRequestClientHandlerTest.java | 3 ++- .../netty/NettyClientServerSslTest.java | 4 ++-- .../io/network/netty/NettyClientTest.java | 2 +- .../netty/NettyPartitionRequestClientTest.java | 6 +++--- .../PartitionRequestClientFactoryTest.java | 6 +++--- .../BoundedBlockingSubpartitionTest.java | 4 ++-- ...undedBlockingSubpartitionWriteReadTest.java | 4 ++-- .../network/partition/BoundedDataTestBase.java | 18 +++++++++--------- .../io/network/partition/DataBufferTest.java | 4 ++-- .../partition/FileBufferReaderITCase.java | 6 +++--- .../partition/FileChannelBoundedDataTest.java | 11 +++++------ ...FileChannelMemoryMappedBoundedDataTest.java | 2 +- .../partition/MemoryMappedBoundedDataTest.java | 2 +- .../PipelinedApproximateSubpartitionTest.java | 4 ++-- ...pproximateSubpartitionWithReadViewTest.java | 4 ++-- .../PipelinedSubpartitionWithReadViewTest.java | 6 +++--- ...esourceManagerPartitionTrackerImplTest.java | 2 +- .../network/partition/ResultPartitionTest.java | 2 +- ...tMergeResultPartitionReadSchedulerTest.java | 3 ++- .../SortMergeResultPartitionTest.java | 6 +++--- .../partition/consumer/InputChannelIDTest.java | 2 +- .../consumer/SingleInputGateTest.java | 2 +- .../partition/consumer/UnionInputGateTest.java | 3 ++- .../hybrid/HsFileDataManagerTest.java | 3 ++- .../hybrid/HsSubpartitionViewTest.java | 3 ++- .../TieredStorageIdMappingUtilsTest.java | 2 +- .../tiered/common/TieredStorageUtilsTest.java | 2 +- .../netty/NettyConnectionWriterTest.java | 2 +- ...ieredStorageResultSubpartitionViewTest.java | 2 +- .../TieredStorageMemoryManagerImplTest.java | 5 +++-- .../TieredStorageProducerClientTest.java | 6 +++--- .../tier/disk/DiskTierProducerAgentTest.java | 2 +- 36 files changed, 79 insertions(+), 72 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java index 710cd8a47e47f..7d20c78a1a202 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/BatchShuffleReadBufferPoolTest.java @@ -26,13 +26,14 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link BatchShuffleReadBufferPool}. */ -@Timeout(60) +@Timeout(value = 60, unit = TimeUnit.SECONDS) class BatchShuffleReadBufferPoolTest { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java index 1d8875215a6f4..567d4808d3e17 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java @@ -2112,7 +2112,7 @@ void testToString() { } @Test - @Timeout(10) + @Timeout(value = 10, unit = TimeUnit.SECONDS) void testToStringMultipleThreads() throws Throwable { buffer.clear(); buffer.writeBytes("Hello, World!".getBytes(CharsetUtil.ISO_8859_1)); @@ -2241,10 +2241,10 @@ void testHashCode() { assertThat(set).hasSize(2); ByteBuf elemACopy = elemA.copy(); - assertThat(set.contains(elemACopy)).isTrue(); + assertThat(set).contains(elemACopy); ByteBuf elemBCopy = elemB.copy(); - assertThat(set.contains(elemBCopy)).isTrue(); + assertThat(set).contains(elemBCopy); buffer.clear(); buffer.writeBytes(elemA.duplicate()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java index f51103ee2a547..67292727d3aa9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferCompressionTest.java @@ -38,7 +38,7 @@ /** Tests for {@link BufferCompressor} and {@link BufferDecompressor}. */ @ExtendWith(ParameterizedTestExtension.class) -public class BufferCompressionTest { +class BufferCompressionTest { private static final int BUFFER_SIZE = 4 * 1024 * 1024; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index ec9f7e5bef75b..207b25d4bec33 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -46,6 +46,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.util.Preconditions.checkArgument; @@ -129,7 +130,7 @@ void testReserveSegments() throws Exception { } @Test - @Timeout(10) // timeout can indicate a potential deadlock + @Timeout(value = 10, unit = TimeUnit.SECONDS) // timeout can indicate a potential deadlock void testReserveSegmentsAndCancel() throws Exception { int totalSegments = 4; int segmentsToReserve = 2; @@ -415,13 +416,13 @@ void testIncreasePoolSizeInternal( } @Test - @Timeout(30) + @Timeout(value = 30, unit = TimeUnit.SECONDS) void testRequestBufferOnRecycleWithOverdraft() throws Exception { testRequestBuffersOnRecycle(true); } @Test - @Timeout(30) + @Timeout(value = 30, unit = TimeUnit.SECONDS) void testRequestBufferOnRecycleWithoutOverdraft() throws Exception { testRequestBuffersOnRecycle(false); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index a3c134189863c..c686de9ae402b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -64,6 +64,7 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.netty.PartitionRequestQueueTest.blockChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; @@ -94,7 +95,7 @@ class CreditBasedPartitionRequestClientHandlerTest { * @see FLINK-1627 */ @Test - @Timeout(60) + @Timeout(value = 60, unit = TimeUnit.SECONDS) @SuppressWarnings("unchecked") void testReleaseInputChannelDuringDecode() throws Exception { // Mocks an input channel in a state as it was released during a decode. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java index 4210aa7e41e32..b0e63e15cae8b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientServerSslTest.java @@ -53,9 +53,9 @@ /** Tests for the SSL connection between Netty Server and Client used for the data plane. */ @ExtendWith(ParameterizedTestExtension.class) -public class NettyClientServerSslTest { +class NettyClientServerSslTest { - @Parameter public String sslProvider; + @Parameter private String sslProvider; @Parameters(name = "SSL provider = {0}") public static List parameters() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientTest.java index 6e29197ede7bc..dd3d0cff568c2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyClientTest.java @@ -37,7 +37,7 @@ import static org.assertj.core.api.Assumptions.assumeThat; /** Tests for {@link NettyClient}. */ -public class NettyClientTest { +class NettyClientTest { @Test void testSetKeepaliveOptionWithNioConfigurable() throws Exception { assumeThat(keepaliveForNioConfigurable()).isTrue(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index fd532cd8996b9..3a5464c88e0bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -50,11 +50,11 @@ /** Tests for {@link NettyPartitionRequestClient}. */ @ExtendWith(ParameterizedTestExtension.class) -public class NettyPartitionRequestClientTest { - @Parameter public boolean connectionReuseEnabled; +class NettyPartitionRequestClientTest { + @Parameter private boolean connectionReuseEnabled; @Parameters(name = "connection reuse enabled = {0}") - public static Object[] parameters() { + private static Object[] parameters() { return new Object[][] {new Object[] {true}, new Object[] {false}}; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java index 05dd16aeb1a78..8f1776b9f697e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java @@ -59,13 +59,13 @@ /** {@link PartitionRequestClientFactory} test. */ @ExtendWith(ParameterizedTestExtension.class) -public class PartitionRequestClientFactoryTest { +class PartitionRequestClientFactoryTest { private static final ResourceID RESOURCE_ID = ResourceID.generate(); - @Parameter public boolean connectionReuseEnabled; + @Parameter private boolean connectionReuseEnabled; @Parameters(name = "connectionReuseEnabled={0}") - public static Collection parameters() { + private static Collection parameters() { return Arrays.asList(false, true); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java index 28bd5be1e2847..6fb3727d7f587 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionTest.java @@ -55,7 +55,7 @@ * BoundedBlockingSubpartitionWriteReadTest}. */ @ExtendWith(ParameterizedTestExtension.class) -public class BoundedBlockingSubpartitionTest extends SubpartitionTestBase { +class BoundedBlockingSubpartitionTest extends SubpartitionTestBase { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); @@ -66,7 +66,7 @@ public class BoundedBlockingSubpartitionTest extends SubpartitionTestBase { private final boolean sslEnabled; @Parameters(name = "type = {0}, sslEnabled = {1}") - public static List parameters() { + private static List parameters() { return Arrays.stream(BoundedBlockingSubpartitionType.values()) .map((type) -> new Object[][] {{type, true}, {type, false}}) .flatMap(Arrays::stream) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java index beea4b40d2428..71fbf22445f0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedBlockingSubpartitionWriteReadTest.java @@ -52,7 +52,7 @@ /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ @ExtendWith(ParameterizedTestExtension.class) -public class BoundedBlockingSubpartitionWriteReadTest { +class BoundedBlockingSubpartitionWriteReadTest { private static final String tempDir = EnvironmentInformation.getTemporaryFileDirectory(); @@ -78,7 +78,7 @@ public class BoundedBlockingSubpartitionWriteReadTest { private final boolean sslEnabled; @Parameters(name = "type = {0}, compressionEnabled = {1}") - public static Collection parameters() { + private static Collection parameters() { return Arrays.stream(BoundedBlockingSubpartitionType.values()) .map((type) -> new Object[][] {{type, true}, {type, false}}) .flatMap(Arrays::stream) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java index 62628b9641080..ce52f2dbc98a3 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/BoundedDataTestBase.java @@ -40,7 +40,7 @@ /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ @ExtendWith(ParameterizedTestExtension.class) -public abstract class BoundedDataTestBase { +abstract class BoundedDataTestBase { private Path subpartitionDataPath; @@ -58,10 +58,10 @@ public abstract class BoundedDataTestBase { private static final BufferDecompressor DECOMPRESSOR = new BufferDecompressor(BUFFER_SIZE, COMPRESSION_CODEC); - @Parameter public static boolean compressionEnabled; + @Parameter private static boolean compressionEnabled; @Parameters(name = "compressionEnabled = {0}") - public static List compressionEnabled() { + private static List compressionEnabled() { return Arrays.asList(false, true); } @@ -94,14 +94,14 @@ private BoundedData createBoundedDataWithRegion(int regionSize) throws IOExcepti // ------------------------------------------------------------------------ @TestTemplate - public void testWriteAndReadData() throws Exception { + void testWriteAndReadData() throws Exception { try (BoundedData bd = createBoundedData()) { testWriteAndReadData(bd); } } @TestTemplate - public void testWriteAndReadDataAcrossRegions() throws Exception { + void testWriteAndReadDataAcrossRegions() throws Exception { if (!isRegionBased()) { return; } @@ -120,7 +120,7 @@ private void testWriteAndReadData(BoundedData bd) throws Exception { } @TestTemplate - public void returnNullAfterEmpty() throws Exception { + void returnNullAfterEmpty() throws Exception { try (BoundedData bd = createBoundedData()) { bd.finishWrite(); @@ -134,7 +134,7 @@ public void returnNullAfterEmpty() throws Exception { } @TestTemplate - public void testDeleteFileOnClose() throws Exception { + void testDeleteFileOnClose() throws Exception { final BoundedData bd = createBoundedData(subpartitionDataPath); assertThat(subpartitionDataPath).exists(); @@ -144,14 +144,14 @@ public void testDeleteFileOnClose() throws Exception { } @TestTemplate - public void testGetSizeSingleRegion() throws Exception { + void testGetSizeSingleRegion() throws Exception { try (BoundedData bd = createBoundedData()) { testGetSize(bd, 60_787, 76_687); } } @TestTemplate - public void testGetSizeMultipleRegions() throws Exception { + void testGetSizeMultipleRegions() throws Exception { if (!isRegionBased()) { return; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java index e936f2d277763..4cd5387df8103 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/DataBufferTest.java @@ -46,12 +46,12 @@ /** Tests for {@link SortBasedDataBuffer} and {@link HashBasedDataBuffer}. */ @ExtendWith(ParameterizedTestExtension.class) -public class DataBufferTest { +class DataBufferTest { private final boolean useHashBuffer; @Parameters(name = "UseHashBuffer = {0}") - public static List parameters() { + private static List parameters() { return Arrays.asList(true, false); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java index 340a6fc3c99af..6fbebc9d517c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileBufferReaderITCase.java @@ -68,7 +68,7 @@ * assumption. */ @ExtendWith(ParameterizedTestExtension.class) -public class FileBufferReaderITCase { +class FileBufferReaderITCase { private static final int parallelism = 8; @@ -83,11 +83,11 @@ public class FileBufferReaderITCase { private static final byte[] dataSource = new byte[recordSize]; @Parameters(name = "SSL Enabled = {0}") - public static List paras() { + private static List paras() { return Arrays.asList(true, false); } - @Parameter public boolean sslEnabled; + @Parameter private boolean sslEnabled; @BeforeAll static void setup() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java index 511fff4f975a2..59db7dbea9d9e 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelBoundedDataTest.java @@ -26,7 +26,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; import java.io.IOException; import java.nio.file.Path; @@ -64,12 +64,11 @@ protected BoundedData createBoundedData(Path tempFilePath) throws IOException { } @Override - protected BoundedData createBoundedDataWithRegion(Path tempFilePath, int regionSize) - throws IOException { + protected BoundedData createBoundedDataWithRegion(Path tempFilePath, int regionSize) { throw new UnsupportedOperationException(); } - @Test + @TestTemplate void testReadNextBuffer() throws Exception { final int numberOfBuffers = 3; try (final BoundedData data = createBoundedData()) { @@ -90,7 +89,7 @@ void testReadNextBuffer() throws Exception { } } - @Test + @TestTemplate void testRecycleBufferForNotifyingSubpartitionView() throws Exception { final int numberOfBuffers = 2; try (final BoundedData data = createBoundedData()) { @@ -121,7 +120,7 @@ void testRecycleBufferForNotifyingSubpartitionView() throws Exception { } } - @Test + @TestTemplate void testRecycleBufferForNotifyingBufferAvailabilityListener() throws Exception { final ResultSubpartition subpartition = createFileBoundedBlockingSubpartition(); final int numberOfBuffers = 2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedDataTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedDataTest.java index f0f068722fcb1..6fcc238bf30ff 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedDataTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/FileChannelMemoryMappedBoundedDataTest.java @@ -22,7 +22,7 @@ import java.nio.file.Path; /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ -public class FileChannelMemoryMappedBoundedDataTest extends BoundedDataTestBase { +class FileChannelMemoryMappedBoundedDataTest extends BoundedDataTestBase { @Override protected boolean isRegionBased() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBoundedDataTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBoundedDataTest.java index 0c5b6269abccf..85e233b650c91 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBoundedDataTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MemoryMappedBoundedDataTest.java @@ -22,7 +22,7 @@ import java.nio.file.Path; /** Tests that read the BoundedBlockingSubpartition with multiple threads in parallel. */ -public class MemoryMappedBoundedDataTest extends BoundedDataTestBase { +class MemoryMappedBoundedDataTest extends BoundedDataTestBase { @Override protected boolean isRegionBased() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java index 2f5ac36ce556b..4aa4f1f18a43d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java @@ -145,7 +145,7 @@ void testSkipPartialDataLongRecordOccupyEntireBuffer() throws Exception { } @TestTemplate - public void testSkipPartialDataLongRecordOccupyEntireBufferWithMoreData() throws Exception { + void testSkipPartialDataLongRecordOccupyEntireBufferWithMoreData() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); @@ -169,7 +169,7 @@ public void testSkipPartialDataLongRecordOccupyEntireBufferWithMoreData() throws } @TestTemplate - public void testSkipPartialDataLongRecordEndWithBuffer() throws Exception { + void testSkipPartialDataLongRecordEndWithBuffer() throws Exception { final BufferWritingResultPartition writer = createResultPartition(); final PipelinedApproximateSubpartition subpartition = getPipelinedApproximateSubpartition(writer); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java index 34b7139c4b3b8..6c313ad149a66 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.partition; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; import java.io.IOException; @@ -43,7 +43,7 @@ void before() throws IOException { readView = subpartition.createReadView(availablityListener); } - @Test + @TestTemplate @Override void testRelease() { readView.releaseAllResources(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index a5854684293d0..ee494156ba337 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -64,17 +64,17 @@ * @see PipelinedSubpartitionTest */ @ExtendWith(ParameterizedTestExtension.class) -public class PipelinedSubpartitionWithReadViewTest { +class PipelinedSubpartitionWithReadViewTest { ResultPartition resultPartition; PipelinedSubpartition subpartition; AwaitableBufferAvailablityListener availablityListener; PipelinedSubpartitionView readView; - @Parameter public boolean compressionEnabled; + @Parameter private boolean compressionEnabled; @Parameters(name = "compressionEnabled = {0}") - public static List parameters() { + private static List parameters() { return Arrays.asList(false, true); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java index db8ad8aa6a91a..271ee486e0e50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResourceManagerPartitionTrackerImplTest.java @@ -123,7 +123,7 @@ void testListDataSetsBasics() { private static void checkListedDataSets( ResourceManagerPartitionTracker tracker, int expectedRegistered, int expectedTotal) { final Map listing = tracker.listDataSets(); - assertThat(listing.containsKey(DATA_SET_ID)).isTrue(); + assertThat(listing).containsKey(DATA_SET_ID); DataSetMetaInfo metaInfo = listing.get(DATA_SET_ID); assertThat(metaInfo.getNumRegisteredPartitions().orElse(-1)).isEqualTo(expectedRegistered); assertThat(metaInfo.getNumTotalPartitions()).isEqualTo(expectedTotal); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 65df6880f4128..ce58e1a0e852e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -523,7 +523,7 @@ void testBroadcastRecordWithRecordSpanningMultipleBuffers() throws Exception { } @Test - public void testWaitForAllRecordProcessed() throws IOException { + void testWaitForAllRecordProcessed() throws IOException { // Creates a result partition with 2 channels. BufferWritingResultPartition bufferWritingResultPartition = createResultPartition(ResultPartitionType.PIPELINED_BOUNDED); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java index 0971e986e1047..6f49640abe873 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java @@ -43,6 +43,7 @@ import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import static org.apache.flink.runtime.io.network.partition.PartitionedFileWriteReadTest.createAndConfigIndexEntryBuffer; @@ -116,7 +117,7 @@ void after() throws Exception { } @Test - @Timeout(60) + @Timeout(value = 60, unit = TimeUnit.SECONDS) void testCreateSubpartitionReader() throws Exception { ManuallyTriggeredScheduledExecutorService ioExecutor = new ManuallyTriggeredScheduledExecutorService(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java index d2e9be9b27ba9..6340bf93dec93 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionTest.java @@ -62,7 +62,7 @@ /** Tests for {@link SortMergeResultPartition}. */ @ExtendWith(ParameterizedTestExtension.class) -public class SortMergeResultPartitionTest { +class SortMergeResultPartitionTest { private static final int bufferSize = 1024; @@ -72,7 +72,7 @@ public class SortMergeResultPartitionTest { private static final int numThreads = 4; - @Parameter public boolean useHashDataBuffer; + @Parameter private boolean useHashDataBuffer; private final TestBufferAvailabilityListener listener = new TestBufferAvailabilityListener(); @@ -105,7 +105,7 @@ void shutdown() throws Exception { } @Parameters(name = "useHashDataBuffer={0}") - public static Collection parameters() { + private static Collection parameters() { return Arrays.asList(false, true); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java index b5a2ad893567a..873b943226a71 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelIDTest.java @@ -31,7 +31,7 @@ class InputChannelIDTest { private static final NettyBufferPool ALLOCATOR = new NettyBufferPool(1); @Test - public void testByteBufWriteAndRead() { + void testByteBufWriteAndRead() { final InputChannelID inputChannelID = new InputChannelID(); final int byteBufLen = InputChannelID.getByteBufLength(); final ByteBuf byteBuf = ALLOCATOR.directBuffer(byteBufLen, byteBufLen); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 1c49e3b65c346..ad7935b3ade21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -106,7 +106,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link SingleInputGate}. */ -public class SingleInputGateTest extends InputGateTestBase { +class SingleInputGateTest extends InputGateTestBase { @Test void testCheckpointsDeclinedUnlessAllChannelsAreKnown() throws CheckpointException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 2fccb1b19397b..8431ac53ac75d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -30,6 +30,7 @@ import org.junit.jupiter.api.Timeout; import java.io.IOException; +import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.verifyBufferOrEvent; import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; @@ -46,7 +47,7 @@ class UnionInputGateTest extends InputGateTestBase { * the correct logical index. */ @Test - @Timeout(120) + @Timeout(value = 120, unit = TimeUnit.SECONDS) void testBasicGetNextLogic() throws Exception { // Setup final SingleInputGate ig1 = createInputGate(3); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java index 9c0f1ab597e5d..1e6d1a89f5c03 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManagerTest.java @@ -48,6 +48,7 @@ import java.util.Queue; import java.util.Random; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; @@ -298,7 +299,7 @@ void testRunReadBuffersThrowException() throws Exception { /** Test file data manager release when reader is reading buffers. */ @Test - @Timeout(10) + @Timeout(value = 10, unit = TimeUnit.SECONDS) void testReleasedWhenReading() throws Exception { TestingHsSubpartitionFileReader reader = new TestingHsSubpartitionFileReader(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java index 976e8e12af987..6a71595aa2882 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java @@ -36,6 +36,7 @@ import java.nio.file.Path; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.runtime.io.network.partition.hybrid.HybridShuffleTestUtils.createTestingOutputMetrics; @@ -72,7 +73,7 @@ void testGetNextBufferFromDisk() { } @Test - @Timeout(60) + @Timeout(value = 60, unit = TimeUnit.SECONDS) void testDeadLock(@TempDir Path dataFilePath) throws Exception { final int bufferSize = 16; NetworkBufferPool networkBufferPool = new NetworkBufferPool(10, bufferSize); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageIdMappingUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageIdMappingUtilsTest.java index f8363969af575..9851c8a0b4822 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageIdMappingUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageIdMappingUtilsTest.java @@ -26,7 +26,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TieredStorageIdMappingUtils}. */ -public class TieredStorageIdMappingUtilsTest { +class TieredStorageIdMappingUtilsTest { @Test void testConvertDataSetId() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageUtilsTest.java index b470ea4807810..a6ba37beed6b2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/common/TieredStorageUtilsTest.java @@ -31,7 +31,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TieredStorageUtils}. */ -public class TieredStorageUtilsTest { +class TieredStorageUtilsTest { @Test void testGenerateBufferWithHeaders() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionWriterTest.java index 5d6552184721a..2943024c355fa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionWriterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionWriterTest.java @@ -28,7 +28,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link NettyConnectionWriter}. */ -public class NettyConnectionWriterTest { +class NettyConnectionWriterTest { private static final int SUBPARTITION_ID = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageResultSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageResultSubpartitionViewTest.java index afd84848cef47..0a1cd7a0d2d5f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageResultSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/TieredStorageResultSubpartitionViewTest.java @@ -40,7 +40,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link TieredStorageResultSubpartitionView}. */ -public class TieredStorageResultSubpartitionViewTest { +class TieredStorageResultSubpartitionViewTest { private static final int TIER_NUMBER = 2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java index 1f03f24050a70..54802c0efc9ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageMemoryManagerImplTest.java @@ -36,13 +36,14 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; /** Tests for {@link TieredStorageMemoryManagerImpl}. */ -public class TieredStorageMemoryManagerImplTest { +class TieredStorageMemoryManagerImplTest { private static final int NETWORK_BUFFER_SIZE = 1024; @@ -183,7 +184,7 @@ void testNumMaxNonReclaimableWhenOtherUseMoreThanGuaranteed() throws IOException } @Test - @Timeout(60) + @Timeout(value = 60, unit = TimeUnit.SECONDS) void testTriggerReclaimBuffers() throws IOException { int numBuffers = 5; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java index fd91b53d8eaa6..d4b5e39867c75 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/storage/TieredStorageProducerClientTest.java @@ -49,18 +49,18 @@ /** Tests for {@link TieredStorageProducerClient}. */ @ExtendWith(ParameterizedTestExtension.class) -public class TieredStorageProducerClientTest { +class TieredStorageProducerClientTest { private static final int NUM_TOTAL_BUFFERS = 1000; private static final int NETWORK_BUFFER_SIZE = 1024; - @Parameter public boolean isBroadcast; + @Parameter private boolean isBroadcast; private NetworkBufferPool globalPool; @Parameters(name = "isBroadcast={0}") - public static Collection parameters() { + private static Collection parameters() { return Arrays.asList(false, true); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskTierProducerAgentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskTierProducerAgentTest.java index 3daf34c4a97b7..d3a19a882baf9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskTierProducerAgentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/disk/DiskTierProducerAgentTest.java @@ -48,7 +48,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link DiskTierProducerAgent}. */ -public class DiskTierProducerAgentTest { +class DiskTierProducerAgentTest { @TempDir private Path tempFolder; From 84036ae75eb306b67f73d7fe27bb2a4d2eb7ee4c Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Sun, 10 Sep 2023 22:40:01 +0800 Subject: [PATCH 7/7] Rebase and optimize --- .../io/network/buffer/NetworkBufferTest.java | 5 +++-- .../netty/PartitionRequestQueueTest.java | 4 ++-- .../PartitionRequestServerHandlerTest.java | 4 ++-- .../partition/InputGateFairnessTest.java | 4 ++-- .../JobMasterPartitionTrackerImplTest.java | 3 +-- ...proximateSubpartitionWithReadViewTest.java | 8 ++----- .../partition/PipelinedSubpartitionTest.java | 2 +- ...PipelinedSubpartitionWithReadViewTest.java | 8 ++----- .../consumer/ChannelStatePersisterTest.java | 2 +- .../partition/consumer/InputGateTestBase.java | 6 ++--- .../consumer/LocalInputChannelTest.java | 10 ++++----- .../consumer/SingleInputGateTest.java | 22 ++++++++++++------- .../consumer/UnionInputGateTest.java | 4 ++-- .../HsSubpartitionFileReaderImplTest.java | 1 + .../netty/NettyConnectionReaderTest.java | 3 +-- .../hybrid/tiered/netty/NettyPayloadTest.java | 2 -- ...moryTierSubpartitionProducerAgentTest.java | 3 +-- 17 files changed, 43 insertions(+), 48 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java index 38071903547a8..262611e682180 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferTest.java @@ -256,8 +256,9 @@ private static void testGetMaxCapacity(boolean isBuffer) { NetworkBuffer buffer = newBuffer(100, 1024, isBuffer); assertThat(buffer.getMaxCapacity()).isEqualTo(1024); MemorySegment segment = buffer.getMemorySegment(); - assertThat(segment.size()).isEqualTo(buffer.getMaxCapacity()); - assertThat(segment.size()).isEqualTo(buffer.maxCapacity()); + assertThat(segment.size()) + .isEqualTo(buffer.getMaxCapacity()) + .isEqualTo(buffer.maxCapacity()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java index a224fcf89bb1f..b6243ce1895ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java @@ -460,14 +460,14 @@ private void testCancelPartitionRequest(boolean isAvailableView) throws Exceptio // add credit to make this reader available for adding into availableReaders queue if (isAvailableView) { queue.addCreditOrResumeConsumption(receiverId, viewReader -> viewReader.addCredit(1)); - assertThat(queue.getAvailableReaders().contains(reader)).isTrue(); + assertThat(queue.getAvailableReaders()).contains(reader); } // cancel this subpartition view queue.cancel(receiverId); channel.runPendingTasks(); - assertThat(queue.getAvailableReaders().contains(reader)).isFalse(); + assertThat(queue.getAvailableReaders()).doesNotContain(reader); // the reader view should be released (the partition is not, though, blocking partitions // support multiple successive readers for recovery and caching) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java index e262bba99f8fb..8a9e3c9f2f301 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandlerTest.java @@ -173,9 +173,9 @@ void testReceivingNewBufferSizeBeforeReaderIsCreated() { channel.runPendingTasks(); // If error happens outbound messages would be not empty. - assertThat(channel.outboundMessages().isEmpty()) + assertThat(channel.outboundMessages()) .withFailMessage(channel.outboundMessages().toString()) - .isTrue(); + .isEmpty(); // New buffer size should be silently ignored because it is possible situation. assertThat(testViewReader.bufferSize).isEqualTo(-1); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index 5b8a062c3a012..7160bc7c5a0be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -125,7 +125,7 @@ void testFairConsumptionLocalChannelsPreFilled() throws Exception { assertThat(max == min || max == (min + 1)).isTrue(); } - assertThat(gate.getNext()).isEmpty(); + assertThat(gate.getNext()).isNotPresent(); } @Test @@ -246,7 +246,7 @@ void testFairConsumptionRemoteChannelsPreFilled() throws Exception { assertThat(max == min || max == (min + 1)).isTrue(); } - assertThat(gate.getNext()).isEmpty(); + assertThat(gate.getNext()).isNotPresent(); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java index 2140a37b1bf3a..70aa477bdd571 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java @@ -146,8 +146,7 @@ void testReleaseCallsWithoutLocalResources() { assertThat(releaseCalls).isEmpty(); assertThat(promoteCalls).isEmpty(); - assertThat(shuffleMaster.externallyReleasedPartitions).hasSize(1); - assertThat(shuffleMaster.externallyReleasedPartitions).contains(resultPartitionId); + assertThat(shuffleMaster.externallyReleasedPartitions).containsOnly(resultPartitionId); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java index 6c313ad149a66..457f11a6265eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionWithReadViewTest.java @@ -47,11 +47,7 @@ void before() throws IOException { @Override void testRelease() { readView.releaseAllResources(); - assertThat( - resultPartition - .getPartitionManager() - .getUnreleasedPartitions() - .contains(resultPartition.getPartitionId())) - .isTrue(); + assertThat(resultPartition.getPartitionManager().getUnreleasedPartitions()) + .contains(resultPartition.getPartitionId()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index afe458e4157fd..9b8296f028cff 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -399,7 +399,7 @@ private void testConsumeQuicklyWithNDataBuffers( assertSubpartitionChannelStateFuturesAndQueuedBuffers( subpartition, channelStateFuture, true, 0, true); - assertThat(channelStateFuture.get().isEmpty()).isTrue(); + assertThatFuture(channelStateFuture).eventuallySucceeds().asList().isEmpty(); subpartition.resumeConsumption(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java index ee494156ba337..ca4b62b49ecb9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionWithReadViewTest.java @@ -103,12 +103,8 @@ void testAddTwoNonFinishedBuffer() throws IOException { void testRelease() { readView.releaseAllResources(); resultPartition.close(); - assertThat( - resultPartition - .getPartitionManager() - .getUnreleasedPartitions() - .contains(resultPartition.getPartitionId())) - .isFalse(); + assertThat(resultPartition.getPartitionManager().getUnreleasedPartitions()) + .doesNotContain(resultPartition.getPartitionId()); } @TestTemplate diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java index 109a542dfa651..6ea9bf8712cfb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersisterTest.java @@ -78,7 +78,7 @@ void testNewBarrierNotOverwrittenByCheckForBarrier() throws Exception { persister.startPersisting(1L, Collections.emptyList()); persister.startPersisting(2L, Collections.emptyList()); - assertThat(persister.checkForBarrier(barrier(1L)).isPresent()).isFalse(); + assertThat(persister.checkForBarrier(barrier(1L))).isNotPresent(); assertThat(persister.hasBarrierReceived()).isFalse(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java index 98a334d94a02e..055187757b0f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java @@ -45,12 +45,12 @@ protected void testIsAvailable( throws Exception { assertThat(inputGateToTest.getAvailableFuture()).isNotDone(); - assertThat(inputGateToTest.pollNext()).isEmpty(); + assertThat(inputGateToTest.pollNext()).isNotPresent(); CompletableFuture future = inputGateToTest.getAvailableFuture(); assertThat(inputGateToTest.getAvailableFuture()).isNotDone(); - assertThat(inputGateToTest.pollNext()).isEmpty(); + assertThat(inputGateToTest.pollNext()).isNotPresent(); assertThat(inputGateToTest.getAvailableFuture()).isEqualTo(future); @@ -67,7 +67,7 @@ protected void testIsAvailableAfterFinished( CompletableFuture available = inputGateToTest.getAvailableFuture(); assertThat(available).isNotDone(); - assertThat(inputGateToTest.pollNext()).isEmpty(); + assertThat(inputGateToTest.pollNext()).isNotPresent(); endOfPartitionEvent.run(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 35aa9cdcabdc5..1d55b015d96b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -479,8 +479,9 @@ void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Exception { channel.requestSubpartition(); Optional bufferAndAvailability = channel.getNextBuffer(); - assertThat(bufferAndAvailability).isPresent(); - assertThat(bufferAndAvailability.get().buffer().isCompressed()).isFalse(); + + assertThat(bufferAndAvailability) + .hasValueSatisfying(value -> assertThat(value.buffer().isCompressed()).isFalse()); } @Test @@ -543,13 +544,12 @@ void testEnqueueAvailableChannelWhenResuming() throws IOException, InterruptedEx subpartition.flush(); // No buffer since the subpartition is blocked. - assertThat(channel.inputGate.pollNext()).isEmpty(); + assertThat(channel.inputGate.pollNext()).isNotPresent(); // Resumption makes the subpartition available. channel.resumeConsumption(); Optional nextBuffer = channel.inputGate.pollNext(); - assertThat(nextBuffer).isPresent(); - assertThat(nextBuffer.get().isBuffer()).isTrue(); + assertThat(nextBuffer).hasValueSatisfying(value -> assertThat(value.isBuffer()).isTrue()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index ad7935b3ade21..b7a9161cb13ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -356,8 +356,9 @@ void testGetCompressedBuffer(final String compressionCodec) throws Exception { inputGate.notifyChannelNonEmpty(inputChannel); Optional bufferOrEvent = inputGate.getNext(); - assertThat(bufferOrEvent.isPresent()).isTrue(); - assertThat(bufferOrEvent.get().isBuffer()).isTrue(); + assertThat(bufferOrEvent) + .hasValueSatisfying(value -> assertThat(value.isBuffer()).isTrue()); + ByteBuffer buffer = bufferOrEvent .get() @@ -378,12 +379,16 @@ void testNotifyAfterEndOfPartition() throws Exception { inputChannel.readEndOfPartitionEvent(); inputChannel.notifyChannelNonEmpty(); - assertThat(inputGate.pollNext().get().getEvent()).isEqualTo(EndOfPartitionEvent.INSTANCE); + assertThat(inputGate.pollNext()) + .hasValueSatisfying( + value -> + assertThat(value.getEvent()) + .isEqualTo(EndOfPartitionEvent.INSTANCE)); // gate is still active because of secondary channel // test if released channel is enqueued inputChannel.notifyChannelNonEmpty(); - assertThat(inputGate.pollNext().isPresent()).isFalse(); + assertThat(inputGate.pollNext()).isNotPresent(); } @Test @@ -1101,9 +1106,9 @@ void testInputGateRemovalFromNettyShuffleEnvironment() throws Exception { assertThat(createdInputGatesById).hasSize(numberOfGates); for (InputGateID id : createdInputGatesById.keySet()) { - assertThat(network.getInputGate(id).isPresent()).isTrue(); + assertThat(network.getInputGate(id)).isPresent(); createdInputGatesById.get(id).close(); - assertThat(network.getInputGate(id).isPresent()).isFalse(); + assertThat(network.getInputGate(id)).isNotPresent(); } } } @@ -1417,13 +1422,14 @@ static void verifyBufferOrEvent( throws IOException, InterruptedException { final Optional bufferOrEvent = inputGate.getNext(); - assertThat(bufferOrEvent.isPresent()).isTrue(); + assertThat(bufferOrEvent).satisfies(value -> assertThat(value).satisfies()); + assertThat(bufferOrEvent.get().isBuffer()).isEqualTo(expectedIsBuffer); assertThat(bufferOrEvent.get().getChannelInfo()) .isEqualTo(inputGate.getChannel(expectedChannelIndex).getChannelInfo()); assertThat(bufferOrEvent.get().moreAvailable()).isEqualTo(expectedMoreAvailable); if (!expectedMoreAvailable) { - assertThat(inputGate.pollNext().isPresent()).isFalse(); + assertThat(inputGate.pollNext()).isNotPresent(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 8431ac53ac75d..27eb4aa5ab519 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -131,7 +131,7 @@ void testBasicGetNextLogic() throws Exception { // Return null when the input gate has received all end-of-partition events assertThat(union.isFinished()).isTrue(); - assertThat(union.getNext()).isEmpty(); + assertThat(union.getNext()).isNotPresent(); } @Test @@ -290,7 +290,7 @@ void testEmptyPull() throws IOException, InterruptedException { inputChannel1.notifyChannelNonEmpty(); assertThat(inputGate.getAvailableFuture()).isDone(); - assertThat(inputGate.pollNext()).isEmpty(); + assertThat(inputGate.pollNext()).isNotPresent(); assertThat(inputGate.getAvailableFuture()).isNotDone(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java index 6120e92a240fb..977c4bf640dcf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImplTest.java @@ -643,6 +643,7 @@ private static void checkData( BufferIndexOrError bufferIndexOrError = fileReader.getLoadedBuffers().poll(); assertThat(bufferIndexOrError).isNotNull(); assertThat(bufferIndexOrError.getBuffer()).isPresent(); + Buffer buffer = bufferIndexOrError.getBuffer().get(); buffer = buffer.isCompressed() && bufferDecompressor != null diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java index bf9f88f2c896e..f3dc14e6368cd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyConnectionReaderTest.java @@ -60,8 +60,7 @@ void testReadBuffer() { createInputChannelSupplier(bufferNumber, requiredSegmentIdFuture); NettyConnectionReader reader = createNettyConnectionReader(inputChannelSupplier); Optional buffer = reader.readBuffer(0); - assertThat(buffer).isPresent(); - assertThat(buffer.get().isBuffer()).isTrue(); + assertThat(buffer).hasValueSatisfying(value -> assertThat(value.isBuffer()).isTrue()); assertThat(requiredSegmentIdFuture).isNotDone(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyPayloadTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyPayloadTest.java index 9d273409f618b..878a8909df90d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyPayloadTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/netty/NettyPayloadTest.java @@ -33,7 +33,6 @@ class NettyPayloadTest { void testGetBuffer() { Buffer buffer = BufferBuilderTestUtils.buildSomeBuffer(0); NettyPayload nettyPayload = NettyPayload.newBuffer(buffer, 0, 0); - assertThat(nettyPayload.getBuffer()).isPresent(); assertThat(nettyPayload.getBuffer()).hasValue(buffer); assertThatThrownBy(() -> NettyPayload.newBuffer(null, 0, 0)) .isInstanceOf(IllegalStateException.class); @@ -47,7 +46,6 @@ void testGetBuffer() { void testGetError() { Throwable error = new RuntimeException("test exception"); NettyPayload nettyPayload = NettyPayload.newError(error); - assertThat(nettyPayload.getError()).isPresent(); assertThat(nettyPayload.getError()).hasValue(error); assertThatThrownBy(() -> NettyPayload.newError(null)) .isInstanceOf(NullPointerException.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgentTest.java index 99a6e58b3b6d0..af3a2d03c249c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/tiered/tier/memory/MemoryTierSubpartitionProducerAgentTest.java @@ -75,8 +75,7 @@ void testAddFinishedBuffer() { subpartitionProducerAgent.addFinishedBuffer(sentBuffer); Optional receivedBuffer = received.get().getBuffer(); - assertThat(receivedBuffer).isPresent(); - assertThat(receivedBuffer.get()).isEqualTo(sentBuffer); + assertThat(receivedBuffer).hasValue(sentBuffer); } @Test