You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by le...@apache.org on 2023/12/19 11:12:48 UTC

(flink) branch master updated: [FLINK-32850][flink-runtime][JUnit5 Migration] The io.network.buffer package of flink-runtime module

This is an automated email from the ASF dual-hosted git repository.

leonard pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 8a8cceda4ce [FLINK-32850][flink-runtime][JUnit5 Migration] The io.network.buffer package of flink-runtime module
8a8cceda4ce is described below

commit 8a8cceda4ced7f71d739e9f5f0bd7dadb094242d
Author: Jiabao Sun <ji...@xtransfer.cn>
AuthorDate: Tue Dec 19 19:12:41 2023 +0800

    [FLINK-32850][flink-runtime][JUnit5 Migration] The io.network.buffer package of flink-runtime module
    
    This closes #23604.
---
 .../io/network/buffer/AbstractByteBufTest.java     | 3342 ++++++++++----------
 .../buffer/BufferBuilderAndConsumerTest.java       |  224 +-
 .../io/network/buffer/BufferBuilderTestUtils.java  |   56 +-
 .../io/network/buffer/BufferCompressionTest.java   |  150 +-
 .../BufferConsumerWithPartialRecordLengthTest.java |   74 +-
 .../io/network/buffer/BufferPoolFactoryTest.java   |  291 +-
 .../network/buffer/LocalBufferPoolDestroyTest.java |   62 +-
 .../io/network/buffer/LocalBufferPoolTest.java     |   17 +-
 .../io/network/buffer/NetworkBufferPoolTest.java   |  391 +--
 .../io/network/buffer/NetworkBufferTest.java       |  229 +-
 .../network/buffer/ReadOnlySlicedBufferTest.java   |  215 +-
 .../PipelinedApproximateSubpartitionTest.java      |    4 +-
 .../io/network/partition/ResultPartitionTest.java  |    2 +-
 .../TaskCancelAsyncProducerConsumerITCase.java     |    2 +-
 14 files changed, 2554 insertions(+), 2505 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 6bf4d96140f..1aa7d8c0a61 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,7 +16,7 @@
  */
 package org.apache.flink.runtime.io.network.buffer;
 
-import org.apache.flink.util.TestLogger;
+import org.apache.flink.core.testutils.CheckedThread;
 
 import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
 import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufUtil;
@@ -25,9 +25,10 @@ import org.apache.flink.shaded.netty4.io.netty.util.CharsetUtil;
 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 +64,17 @@ import static org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled.directBuff
 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.within;
+import static org.assertj.core.api.Assumptions.assumeThat;
 
 /**
  * An abstract test class for channel buffers.
  *
  * <p>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 +94,18 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
 
             try {
                 buffer.release();
@@ -123,97 +117,83 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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).isLessThan(buffer2);
+        assumeThat(buffer2).isGreaterThan(buffer);
         buffer2.release();
     }
 
     @Test
-    public void initialState() {
-        assertEquals(CAPACITY, buffer.capacity());
-        assertEquals(0, buffer.readerIndex());
+    void initialState() {
+        assertThat(buffer.capacity()).isEqualTo(CAPACITY);
+        assertThat(buffer.readerIndex()).isZero();
     }
 
-    @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,203 +201,215 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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]).isZero();
+        assertThat(dst[1]).isZero();
+        assertThat(dst[2]).isZero();
+        assertThat(dst[3]).isZero();
     }
 
     @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]).isZero();
+        assertThat(dst[1]).isZero();
+        assertThat(dst[2]).isZero();
+        assertThat(dst[3]).isZero();
     }
 
-    @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);
 
-        buffer.setByte(0, (byte) 1);
-        buffer.setByte(1, (byte) 2);
-        buffer.setByte(2, (byte) 3);
-        buffer.setByte(3, (byte) 4);
+        buffer.setByte(0, 1);
+        buffer.setByte(1, 2);
+        buffer.setByte(2, 3);
+        buffer.setByte(3, 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)).isZero();
+        assertThat(dst.get(1)).isEqualTo((byte) 2);
+        assertThat(dst.get(2)).isEqualTo((byte) 3);
+        assertThat(dst.get(3)).isZero();
     }
 
-    @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);
 
-        buffer.setByte(0, (byte) 1);
-        buffer.setByte(1, (byte) 2);
-        buffer.setByte(2, (byte) 3);
-        buffer.setByte(3, (byte) 4);
+        buffer.setByte(0, 1);
+        buffer.setByte(1, 2);
+        buffer.setByte(2, 3);
+        buffer.setByte(3, 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)).isZero();
+        assertThat(dst.get(1)).isEqualTo((byte) 2);
+        assertThat(dst.get(2)).isEqualTo((byte) 3);
+        assertThat(dst.get(3)).isZero();
     }
 
     @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 +418,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +431,18 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
         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 +460,15 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +497,22 @@ public abstract class AbstractByteBufTest extends TestLogger {
             javaBuffer.flip();
 
             short javaActual = javaBuffer.getShort();
-            assertEquals(expected, javaActual);
-            assertEquals(
-                    javaActual,
-                    testBigEndian ? buffer.getShort(bufferIndex) : buffer.getShortLE(bufferIndex));
+            assertThat(javaActual)
+                    .isEqualTo(expected)
+                    .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 +530,20 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +561,20 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +592,15 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +629,22 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +662,15 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +699,22 @@ public abstract class AbstractByteBufTest extends TestLogger {
             javaBuffer.flip();
 
             int javaActual = javaBuffer.getInt();
-            assertEquals(expected, javaActual);
-            assertEquals(
-                    javaActual,
-                    testBigEndian ? buffer.getInt(bufferIndex) : buffer.getIntLE(bufferIndex));
+            assertThat(javaActual)
+                    .isEqualTo(expected)
+                    .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 +732,20 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +763,15 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +800,22 @@ public abstract class AbstractByteBufTest extends TestLogger {
             javaBuffer.flip();
 
             long javaActual = javaBuffer.getLong();
-            assertEquals(expected, javaActual);
-            assertEquals(
-                    javaActual,
-                    testBigEndian ? buffer.getLong(bufferIndex) : buffer.getLongLE(bufferIndex));
+            assertThat(javaActual)
+                    .isEqualTo(expected)
+                    .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 +833,17 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +861,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +879,73 @@ public abstract class AbstractByteBufTest extends TestLogger {
         }
 
         for (int i = 0; i < buffer.capacity(); i++) {
-            assertEquals(0, buffer.getByte(i));
+            assertThat(buffer.getByte(i)).isZero();
         }
     }
 
     @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()).isZero();
+        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()).isZero();
+        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 +953,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +962,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
-        assertFalse(buffer.isWritable());
+        assertThat(buffer.readerIndex()).isZero();
+        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 +998,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1007,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
-        assertFalse(buffer.isWritable());
+        assertThat(buffer.readerIndex()).isZero();
+        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 +1043,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1052,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex());
-        assertEquals(buffer.capacity() % 3, buffer.writableBytes());
+        assertThat(buffer.readerIndex()).isZero();
+        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()).isZero();
+        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 +1088,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1097,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity() / 3 * 3, buffer.writerIndex());
-        assertEquals(buffer.capacity() % 3, buffer.writableBytes());
+        assertThat(buffer.readerIndex()).isZero();
+        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()).isZero();
+        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 +1133,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1142,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
-        assertFalse(buffer.isWritable());
+        assertThat(buffer.readerIndex()).isZero();
+        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 +1178,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1187,35 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
-        assertFalse(buffer.isWritable());
+        assertThat(buffer.readerIndex()).isZero();
+        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 +1223,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1232,30 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
-        assertFalse(buffer.isWritable());
+        assertThat(buffer.readerIndex()).isZero();
+        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 +1269,13 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1289,13 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1310,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1334,16 @@ public abstract class AbstractByteBufTest extends TestLogger {
             random.nextBytes(expectedValueContent);
             value.clear();
             buffer.getBytes(i, value);
-            assertEquals(0, value.readerIndex());
-            assertEquals(BLOCK_SIZE, value.writerIndex());
+            assertThat(value.readerIndex()).isZero();
+            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 +1359,13 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1382,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1390,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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 +1406,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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()).isZero();
+            assertThat(buffer.writerIndex()).isEqualTo(i);
             buffer.writeBytes(value);
         }
 
@@ -1430,23 +1428,23 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            assertThat(buffer.writerIndex()).isEqualTo(i);
             int readerIndex = random.nextInt(BLOCK_SIZE);
             buffer.writeBytes(value, readerIndex, BLOCK_SIZE);
         }
@@ -1456,27 +1454,27 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            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()).isZero();
+            assertThat(value.writerIndex()).isEqualTo(valueContent.length);
         }
 
         random.setSeed(seed);
@@ -1485,32 +1483,32 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            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()).isZero();
+            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 +1517,32 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            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()).isZero();
+            assertThat(value.writerIndex()).isZero();
         }
 
         random.setSeed(seed);
@@ -1554,36 +1552,36 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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()).isZero();
+            assertThat(value.writerIndex()).isZero();
         }
         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()).isZero();
+            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 +1591,23 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1615,11 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            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()).isZero();
+            assertThat(value.writerIndex()).isZero();
         }
 
         random.setSeed(seed);
@@ -1631,19 +1629,19 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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()).isZero();
+            assertThat(value.writerIndex()).isZero();
         }
     }
 
     @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 +1649,15 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            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 +1667,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1698,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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()).isZero();
+            assertThat(buffer.writerIndex()).isEqualTo(i);
             buffer.writeBytes(value);
         }
 
@@ -1722,26 +1720,26 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+            assertThat(buffer.writerIndex()).isEqualTo(i);
             buffer.writeBytes(value);
         }
 
@@ -1749,26 +1747,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +1775,16 @@ public abstract class AbstractByteBufTest extends TestLogger {
             i += length;
         }
 
-        assertEquals(0, buffer.readerIndex());
-        assertEquals(buffer.capacity(), buffer.writerIndex());
+        assertThat(buffer.readerIndex()).isZero();
+        assertThat(buffer.writerIndex()).isEqualTo(buffer.capacity());
 
         for (int i = 0; i < buffer.capacity(); i++) {
-            assertEquals(0, buffer.getByte(i));
+            assertThat(buffer.getByte(i)).isZero();
         }
     }
 
     @Test
-    public void testDiscardReadBytes() {
+    void testDiscardReadBytes() {
         buffer.writerIndex(0);
         for (int i = 0; i < buffer.capacity(); i += 4) {
             buffer.writeInt(i);
@@ -1807,40 +1800,40 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+        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()).isZero();
+        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 +1842,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
      * 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 +1852,24 @@ public abstract class AbstractByteBufTest extends TestLogger {
         // 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()).isZero();
+        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 +1877,34 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +1916,24 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
         // 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()).isZero();
+        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 +1945,101 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
         // 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).isEqualTo(duplicate);
 
         // 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() {
+        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() {
+        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();
 
-        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() {
         ByteBuf retainedSlice = buffer.retainedSlice(0, buffer.capacity());
-        assertEquals(0, retainedSlice.readerIndex());
+        assertThat(retainedSlice.readerIndex()).isZero();
         retainedSlice.release();
 
         retainedSlice = buffer.retainedSlice(0, buffer.capacity() - 1);
-        assertEquals(0, retainedSlice.readerIndex());
+        assertThat(retainedSlice.readerIndex()).isZero();
         retainedSlice.release();
 
         retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 1);
-        assertEquals(0, retainedSlice.readerIndex());
+        assertThat(retainedSlice.readerIndex()).isZero();
         retainedSlice.release();
 
         retainedSlice = buffer.retainedSlice(1, buffer.capacity() - 2);
-        assertEquals(0, retainedSlice.readerIndex());
+        assertThat(retainedSlice.readerIndex()).isZero();
         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).isNotEqualTo(null).isNotEqualTo(new Object());
 
         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(wrappedBuffer(value).order(LITTLE_ENDIAN))
+                .isEqualTo(buffer);
 
         value[0]++;
-        assertFalse(buffer.equals(wrappedBuffer(value)));
-        assertFalse(buffer.equals(wrappedBuffer(value).order(LITTLE_ENDIAN)));
+        assertThat(buffer)
+                .isNotEqualTo(wrappedBuffer(value))
+                .isNotEqualTo(wrappedBuffer(value).order(LITTLE_ENDIAN));
     }
 
     @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 +2054,39 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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)
+                .isEqualTo(wrappedBuffer(value))
+                .isEqualTo(wrappedBuffer(value).order(LITTLE_ENDIAN));
 
         value[0]++;
-        assertTrue(buffer.compareTo(wrappedBuffer(value)) < 0);
-        assertTrue(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0);
+        assertThat(buffer)
+                .isLessThan(wrappedBuffer(value))
+                .isLessThan(wrappedBuffer(value).order(LITTLE_ENDIAN));
         value[0] -= 2;
-        assertTrue(buffer.compareTo(wrappedBuffer(value)) > 0);
-        assertTrue(buffer.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) > 0);
+        assertThat(buffer)
+                .isGreaterThan(wrappedBuffer(value))
+                .isGreaterThan(wrappedBuffer(value).order(LITTLE_ENDIAN));
         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)
+                .isGreaterThan(wrappedBuffer(value, 0, 31))
+                .isGreaterThan(wrappedBuffer(value, 0, 31).order(LITTLE_ENDIAN));
+
+        assertThat(buffer.slice(0, 31))
+                .isLessThan(wrappedBuffer(value))
+                .isLessThan(wrappedBuffer(value).order(LITTLE_ENDIAN));
 
         ByteBuf retainedSlice = buffer.retainedSlice(0, 31);
-        assertTrue(retainedSlice.compareTo(wrappedBuffer(value)) < 0);
+        assertThat(retainedSlice).isLessThan(wrappedBuffer(value));
         retainedSlice.release();
 
         retainedSlice = buffer.retainedSlice(0, 31);
-        assertTrue(retainedSlice.compareTo(wrappedBuffer(value).order(LITTLE_ENDIAN)) < 0);
+        assertThat(retainedSlice).isLessThan(wrappedBuffer(value).order(LITTLE_ENDIAN));
         retainedSlice.release();
     }
 
     @Test
-    public void testCompareTo2() {
+    void testCompareTo2() {
         byte[] bytes = {1, 2, 3, 4};
         byte[] bytesReversed = {4, 3, 2, 1};
 
@@ -2098,10 +2096,10 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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,74 +2109,60 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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(value = 10)
+    void testToStringMultipleThreads() throws Throwable {
         buffer.clear();
         buffer.writeBytes("Hello, World!".getBytes(CharsetUtil.ISO_8859_1));
 
         final AtomicInteger counter = new AtomicInteger(30000);
         final AtomicReference<Throwable> errorRef = new AtomicReference<Throwable>();
-        List<Thread> threads = new ArrayList<Thread>();
+        List<CheckedThread> threads = new ArrayList<>();
         for (int i = 0; i < 10; i++) {
-            Thread thread =
-                    new Thread(
-                            new Runnable() {
-                                @Override
-                                public void run() {
-                                    try {
-                                        while (errorRef.get() == null
-                                                && counter.decrementAndGet() > 0) {
-                                            assertEquals(
-                                                    "Hello, World!",
-                                                    buffer.toString(CharsetUtil.ISO_8859_1));
-                                        }
-                                    } catch (Throwable cause) {
-                                        errorRef.compareAndSet(null, cause);
-                                    }
-                                }
-                            });
+            CheckedThread thread =
+                    new CheckedThread() {
+                        @Override
+                        public void go() {
+                            while (errorRef.get() == null && counter.decrementAndGet() > 0) {
+                                assertThat(buffer.toString(CharsetUtil.ISO_8859_1))
+                                        .isEqualTo("Hello, World!");
+                            }
+                        }
+                    };
             threads.add(thread);
-        }
-        for (Thread thread : threads) {
             thread.start();
         }
-
-        for (Thread thread : threads) {
-            thread.join();
-        }
-
-        Throwable error = errorRef.get();
-        if (error != null) {
-            throw error;
+        for (CheckedThread thread : threads) {
+            thread.sync();
         }
     }
 
     @Test
-    public void testIndexOf() {
+    void testIndexOf() {
         buffer.clear();
-        buffer.writeByte((byte) 1);
-        buffer.writeByte((byte) 2);
-        buffer.writeByte((byte) 3);
-        buffer.writeByte((byte) 2);
-        buffer.writeByte((byte) 1);
+        buffer.writeByte(1);
+        buffer.writeByte(2);
+        buffer.writeByte(3);
+        buffer.writeByte(2);
+        buffer.writeByte(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)).isOne();
+        assertThat(buffer.indexOf(4, 1, (byte) 2)).isEqualTo(3);
     }
 
     @Test
-    public void testNioBuffer1() {
-        assumeTrue(buffer.nioBufferCount() == 1);
+    void testNioBuffer1() {
+        assumeThat(buffer.nioBufferCount()).isOne();
 
         byte[] value = new byte[buffer.capacity()];
         random.nextBytes(value);
@@ -2189,8 +2173,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @Test
-    public void testToByteBuffer2() {
-        assumeTrue(buffer.nioBufferCount() == 1);
+    void testToByteBuffer2() {
+        assumeThat(buffer.nioBufferCount()).isOne();
 
         byte[] value = new byte[buffer.capacity()];
         random.nextBytes(value);
@@ -2207,41 +2191,37 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isOne();
 
-        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 +2231,23 @@ public abstract class AbstractByteBufTest extends TestLogger {
         set.add(elemA);
         set.add(elemB);
 
-        assertEquals(2, set.size());
+        assertThat(set).hasSize(2);
         ByteBuf elemACopy = elemA.copy();
-        assertTrue(set.contains(elemACopy));
+        assertThat(set).contains(elemACopy);
 
         ByteBuf elemBCopy = elemB.copy();
-        assertTrue(set.contains(elemBCopy));
+        assertThat(set).contains(elemBCopy);
 
         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).doesNotContainSequence(elemA).hasSize(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).doesNotContainSequence(elemB).isEmpty();
         elemA.release();
         elemB.release();
         elemACopy.release();
@@ -2278,14 +2256,14 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
     // 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 +2272,25 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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) {
+                                        assertThat(value).isEqualTo((byte) (i + 1));
+                                        lastIndex.set(i);
+                                        i++;
+                                        return true;
+                                    }
+                                }))
+                .isEqualTo(-1);
 
-        assertThat(lastIndex.get(), is(CAPACITY * 3 / 4 - 1));
+        assertThat(lastIndex).hasValue(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 +2298,28 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
         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) {
+                                        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 +2327,27 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
         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) {
+                                        assertThat(value).isEqualTo((byte) (i + 1));
+                                        lastIndex.set(i);
+                                        i--;
+                                        return true;
+                                    }
+                                }))
+                .isEqualTo(-1);
 
-        assertThat(lastIndex.get(), is(CAPACITY / 4));
+        assertThat(lastIndex).hasValue(CAPACITY / 4);
     }
 
     @Test
-    public void testInternalNioBuffer() {
+    void testInternalNioBuffer() {
         testInternalNioBuffer(128);
         testInternalNioBuffer(1024);
         testInternalNioBuffer(4 * 1024);
@@ -2381,29 +2359,29 @@ public abstract class AbstractByteBufTest extends TestLogger {
     private void testInternalNioBuffer(int a) {
         ByteBuf buffer = newBuffer(2);
         ByteBuffer buf = buffer.internalNioBuffer(buffer.readerIndex(), 1);
-        assertEquals(1, buf.remaining());
+        assertThat(buf.remaining()).isOne();
 
         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);
     }
 
@@ -2417,51 +2395,48 @@ public abstract class AbstractByteBufTest extends TestLogger {
         final CyclicBarrier barrier = new CyclicBarrier(11);
         for (int i = 0; i < 10; i++) {
             new Thread(
-                            new Runnable() {
-                                @Override
-                                public void run() {
-                                    while (latch.getCount() > 0) {
-                                        ByteBuf buf;
-                                        if (slice) {
-                                            buf = buffer.slice();
-                                        } else {
-                                            buf = buffer.duplicate();
-                                        }
-                                        TestGatheringByteChannel channel =
-                                                new TestGatheringByteChannel();
-
-                                        while (buf.isReadable()) {
-                                            try {
-                                                buf.readBytes(channel, buf.readableBytes());
-                                            } catch (IOException e) {
-                                                // Never happens
-                                                return;
-                                            }
-                                        }
-                                        assertArrayEquals(bytes, channel.writtenBytes());
-                                        latch.countDown();
+                            () -> {
+                                while (latch.getCount() > 0) {
+                                    ByteBuf buf;
+                                    if (slice) {
+                                        buf = buffer.slice();
+                                    } else {
+                                        buf = buffer.duplicate();
                                     }
-                                    try {
-                                        barrier.await();
-                                    } catch (Exception e) {
-                                        // ignore
+                                    TestGatheringByteChannel channel =
+                                            new TestGatheringByteChannel();
+
+                                    while (buf.isReadable()) {
+                                        try {
+                                            buf.readBytes(channel, buf.readableBytes());
+                                        } catch (IOException e) {
+                                            // Never happens
+                                            return;
+                                        }
                                     }
+                                    assertThat(channel.writtenBytes()).isEqualTo(bytes);
+                                    latch.countDown();
+                                }
+                                try {
+                                    barrier.await();
+                                } catch (Exception e) {
+                                    // ignore
                                 }
                             })
                     .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);
     }
 
@@ -2475,50 +2450,47 @@ public abstract class AbstractByteBufTest extends TestLogger {
         final CyclicBarrier barrier = new CyclicBarrier(11);
         for (int i = 0; i < 10; i++) {
             new Thread(
-                            new Runnable() {
-                                @Override
-                                public void run() {
-                                    while (latch.getCount() > 0) {
-                                        ByteBuf buf;
-                                        if (slice) {
-                                            buf = buffer.slice();
-                                        } else {
-                                            buf = buffer.duplicate();
-                                        }
-                                        ByteArrayOutputStream out = new ByteArrayOutputStream();
-
-                                        while (buf.isReadable()) {
-                                            try {
-                                                buf.readBytes(out, buf.readableBytes());
-                                            } catch (IOException e) {
-                                                // Never happens
-                                                return;
-                                            }
-                                        }
-                                        assertArrayEquals(bytes, out.toByteArray());
-                                        latch.countDown();
+                            () -> {
+                                while (latch.getCount() > 0) {
+                                    ByteBuf buf;
+                                    if (slice) {
+                                        buf = buffer.slice();
+                                    } else {
+                                        buf = buffer.duplicate();
                                     }
-                                    try {
-                                        barrier.await();
-                                    } catch (Exception e) {
-                                        // ignore
+                                    ByteArrayOutputStream out = new ByteArrayOutputStream();
+
+                                    while (buf.isReadable()) {
+                                        try {
+                                            buf.readBytes(out, buf.readableBytes());
+                                        } catch (IOException e) {
+                                            // Never happens
+                                            return;
+                                        }
                                     }
+                                    assertThat(out.toByteArray()).isEqualTo(bytes);
+                                    latch.countDown();
+                                }
+                                try {
+                                    barrier.await();
+                                } catch (Exception e) {
+                                    // ignore
                                 }
                             })
                     .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);
     }
 
@@ -2533,50 +2505,47 @@ public abstract class AbstractByteBufTest extends TestLogger {
         final CyclicBarrier barrier = new CyclicBarrier(11);
         for (int i = 0; i < 10; i++) {
             new Thread(
-                            new Runnable() {
-                                @Override
-                                public void run() {
-                                    while (cause.get() == null && latch.getCount() > 0) {
-                                        ByteBuf buf;
-                                        if (slice) {
-                                            buf = buffer.slice();
-                                        } else {
-                                            buf = buffer.duplicate();
-                                        }
+                            () -> {
+                                while (cause.get() == null && latch.getCount() > 0) {
+                                    ByteBuf buf;
+                                    if (slice) {
+                                        buf = buffer.slice();
+                                    } else {
+                                        buf = buffer.duplicate();
+                                    }
 
-                                        byte[] array = new byte[8];
-                                        buf.readBytes(array);
+                                    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);
+                                    Arrays.fill(array, (byte) 0);
+                                    buf.getBytes(0, array);
+                                    assertThat(array).isEqualTo(bytes);
 
-                                        latch.countDown();
-                                    }
-                                    try {
-                                        barrier.await();
-                                    } catch (Exception e) {
-                                        // ignore
-                                    }
+                                    latch.countDown();
+                                }
+                                try {
+                                    barrier.await();
+                                } catch (Exception e) {
+                                    // ignore
                                 }
                             })
                     .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()).isZero();
+            assertThatThrownBy(buffer::readByte).isInstanceOf(IndexOutOfBoundsException.class);
         } finally {
             buffer.release();
         }
@@ -2584,29 +2553,29 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
     @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()).isZero();
+        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 +2590,10 @@ public abstract class AbstractByteBufTest extends TestLogger {
     // - 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 +2603,884 @@ public abstract class AbstractByteBufTest extends TestLogger {
         // 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 +3490,20 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
         for (ByteBuf buf : bufs) {
-            try {
-                assertEquals(0, buf.refCnt());
-                buf.slice();
-                fail();
-            } catch (IllegalReferenceCountException ignored) {
-                // as expected
-            }
+            assertThat(buf.refCnt()).isZero();
+            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 +3511,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @Test
-    public void testSliceAfterReleaseRetainedSliceRetainedDuplicate() {
+    void testSliceAfterReleaseRetainedSliceRetainedDuplicate() {
         ByteBuf buf = newBuffer(1);
         ByteBuf buf2 = buf.retainedSlice(0, 1);
         ByteBuf buf3 = buf2.retainedDuplicate();
@@ -3429,28 +3519,30 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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 +3552,21 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
         for (ByteBuf buf : bufs) {
-            try {
-                assertEquals(0, buf.refCnt());
-                buf.retainedSlice();
-                fail();
-            } catch (IllegalReferenceCountException ignored) {
-                // as expected
-            }
+            assertThat(buf.refCnt()).isZero();
+            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 +3574,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @Test
-    public void testRetainedSliceAfterReleaseRetainedSliceRetainedDuplicate() {
+    void testRetainedSliceAfterReleaseRetainedSliceRetainedDuplicate() {
         ByteBuf buf = newBuffer(1);
         ByteBuf buf2 = buf.retainedSlice(0, 1);
         ByteBuf buf3 = buf2.retainedDuplicate();
@@ -3494,28 +3582,30 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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 +3615,13 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
         for (ByteBuf buf : bufs) {
-            try {
-                assertEquals(0, buf.refCnt());
-                buf.duplicate();
-                fail();
-            } catch (IllegalReferenceCountException ignored) {
-                // as expected
-            }
+            assertThat(buf.refCnt()).isZero();
+            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 +3629,14 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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 +3650,51 @@ public abstract class AbstractByteBufTest extends TestLogger {
             }
         }
         for (ByteBuf buf : bufs) {
-            try {
-                assertEquals(0, buf.refCnt());
-                buf.retainedDuplicate();
-                fail();
-            } catch (IllegalReferenceCountException ignored) {
-                // as expected
-            }
+            assertThat(buf.refCnt()).isZero();
+            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()).isOne();
+        assertThat(buf.slice().release()).isTrue();
+        assertThat(buf.refCnt()).isZero();
     }
 
-    @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(true))
+                .isInstanceOf(IndexOutOfBoundsException.class);
     }
 
     private void testReadSliceOutOfBounds(boolean retainedSlice) {
@@ -3623,29 +3706,28 @@ public abstract class AbstractByteBufTest extends TestLogger {
             } else {
                 buf.readSlice(51);
             }
-            fail();
         } 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 +3737,34 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 +3777,22 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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 +3812,28 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +3844,61 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +3911,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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).isEqualTo(expected1).isEqualTo(buf.slice(buf.readerIndex() + 1, 6));
         // Simulate a handler that releases the original buffer, and propagates a slice.
         buf.release();
 
@@ -3824,49 +3919,47 @@ public abstract class AbstractByteBufTest extends TestLogger {
         slice1.readByte();
 
         ByteBuf dup1 = slice1.retainedDuplicate();
-        assertEquals(0, dup1.compareTo(expected2));
-        assertEquals(0, dup1.compareTo(slice1.duplicate()));
+        assertThat(dup1).isEqualTo(expected2).isEqualTo(slice1.duplicate());
 
         // Advance the reader index on dup1.
         dup1.readByte();
 
         ByteBuf dup2 = dup1.duplicate();
-        assertEquals(0, dup2.compareTo(expected3));
+        assertThat(dup2).isEqualTo(expected3);
 
         // 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).isEqualTo(expected4).isEqualTo(dup2.slice(dup2.readerIndex(), 3));
 
         // 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()).isZero();
+        assertThat(slice1.refCnt()).isZero();
+        assertThat(slice2.refCnt()).isZero();
+        assertThat(dup1.refCnt()).isZero();
+        assertThat(dup2.refCnt()).isZero();
     }
 
     @Test
-    public void testRetainedDuplicateAndRetainedSliceContentIsExpected() {
+    void testRetainedDuplicateAndRetainedSliceContentIsExpected() {
         ByteBuf buf = newBuffer(8).resetWriterIndex();
         ByteBuf expected1 = newBuffer(6).resetWriterIndex();
         ByteBuf expected2 = newBuffer(5).resetWriterIndex();
@@ -3877,8 +3970,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
         expected3.writeBytes(new byte[] {5, 6, 7});
 
         ByteBuf dup1 = buf.retainedDuplicate();
-        assertEquals(0, dup1.compareTo(buf));
-        assertEquals(0, dup1.compareTo(buf.slice()));
+        assertThat(dup1).isEqualTo(buf).isEqualTo(buf.slice());
         // Simulate a handler that releases the original buffer, and propagates a slice.
         buf.release();
 
@@ -3886,243 +3978,243 @@ public abstract class AbstractByteBufTest extends TestLogger {
         dup1.readByte();
 
         ByteBuf slice1 = dup1.retainedSlice(dup1.readerIndex(), 6);
-        assertEquals(0, slice1.compareTo(expected1));
-        assertEquals(0, slice1.compareTo(slice1.duplicate()));
+        assertThat(slice1).isEqualTo(expected1).isEqualTo(slice1.duplicate());
 
         // Advance the reader index on slice1.
         slice1.readByte();
 
         ByteBuf dup2 = slice1.duplicate();
-        assertEquals(0, dup2.compareTo(slice1));
+        assertThat(dup2).isEqualTo(slice1);
 
         // 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).isEqualTo(expected3).isEqualTo(dup2.slice(dup2.readerIndex() + 1, 3));
 
         // 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()).isZero();
+        assertThat(slice1.refCnt()).isZero();
+        assertThat(slice2.refCnt()).isZero();
+        assertThat(dup1.refCnt()).isZero();
+        assertThat(dup2.refCnt()).isZero();
     }
 
     @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 +4224,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+        assertThat(buf.refCnt()).isZero();
     }
 
     private void testReadRetainedSliceUnreleasable(
@@ -4149,12 +4241,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
                 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()).isZero();
+        assertThat(buf.refCnt()).isZero();
     }
 
     private void testRetainedDuplicateUnreleasable(
@@ -4164,12 +4256,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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()).isZero();
+        assertThat(buf.refCnt()).isZero();
     }
 
     private void testDuplicateCapacityChange(boolean retainedDuplicate) {
@@ -4177,9 +4269,9 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 +4304,8 @@ public abstract class AbstractByteBufTest extends TestLogger {
                         ? 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 +4332,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
                         ? 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).isEqualTo(slice.duplicate()).isEqualTo(expected);
             ByteBuf b = slice.retainedDuplicate();
-            assertEquals(0, slice.compareTo(b));
+            assertThat(slice).isEqualTo(b);
+
             b.release();
-            assertEquals(0, slice.compareTo(slice.slice(0, slice.capacity())));
+            assertThat(slice).isEqualTo(slice.slice(0, slice.capacity()));
         } finally {
             if (retainedSlice) {
                 slice.release();
@@ -4266,7 +4358,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
                 retainedSlice1
                         ? buf.retainedSlice(buf.readerIndex() + 5, 3)
                         : buf.slice(buf.readerIndex() + 5, 3).retain();
-        assertEquals(0, slice1.compareTo(expected1));
+        assertThat(slice1).isEqualTo(expected1);
         // Simulate a handler that releases the original buffer, and propagates a slice.
         buf.release();
 
@@ -4274,24 +4366,24 @@ public abstract class AbstractByteBufTest extends TestLogger {
                 retainedSlice2
                         ? slice1.retainedSlice(slice1.readerIndex() + 1, 2)
                         : slice1.slice(slice1.readerIndex() + 1, 2).retain();
-        assertEquals(0, slice2.compareTo(expected2));
+        assertThat(slice2).isEqualTo(expected2);
 
         // 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()).isZero();
+        assertThat(slice1.refCnt()).isZero();
+        assertThat(slice2.refCnt()).isZero();
     }
 
     private void testMultipleLevelRetainedSliceWithNonRetained(boolean doSlice1, boolean doSlice2) {
@@ -4309,60 +4401,60 @@ public abstract class AbstractByteBufTest extends TestLogger {
         expected4DupSlice.writeBytes(new byte[] {4});
 
         ByteBuf slice1 = buf.retainedSlice(buf.readerIndex() + 1, 6);
-        assertEquals(0, slice1.compareTo(expected1));
+        assertThat(slice1).isEqualTo(expected1);
         // 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).isEqualTo(expected2);
+        assertThat(slice2).isEqualTo(slice2.duplicate());
+        assertThat(slice2).isEqualTo(slice2.slice());
 
         ByteBuf tmpBuf = slice2.retainedDuplicate();
-        assertEquals(0, slice2.compareTo(tmpBuf));
+        assertThat(slice2).isEqualTo(tmpBuf);
         tmpBuf.release();
         tmpBuf = slice2.retainedSlice();
-        assertEquals(0, slice2.compareTo(tmpBuf));
+        assertThat(slice2).isEqualTo(tmpBuf);
         tmpBuf.release();
 
         ByteBuf slice3 = doSlice1 ? slice2.slice(slice2.readerIndex() + 1, 2) : slice2.duplicate();
         if (doSlice1) {
-            assertEquals(0, slice3.compareTo(expected3));
+            assertThat(slice3).isEqualTo(expected3);
         } else {
-            assertEquals(0, slice3.compareTo(expected2));
+            assertThat(slice3).isEqualTo(expected2);
         }
 
         ByteBuf slice4 = doSlice2 ? slice3.slice(slice3.readerIndex() + 1, 1) : slice3.duplicate();
         if (doSlice1 && doSlice2) {
-            assertEquals(0, slice4.compareTo(expected4SliceSlice));
+            assertThat(slice4).isEqualTo(expected4SliceSlice);
         } else if (doSlice2) {
-            assertEquals(0, slice4.compareTo(expected4DupSlice));
+            assertThat(slice4).isEqualTo(expected4DupSlice);
         } else {
-            assertEquals(0, slice3.compareTo(slice4));
+            assertThat(slice3).isEqualTo(slice4);
         }
 
         // 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()).isZero();
+        assertThat(slice1.refCnt()).isZero();
+        assertThat(slice2.refCnt()).isZero();
+        assertThat(slice3.refCnt()).isZero();
     }
 
     private void testDuplicateReleaseOriginal(
@@ -4372,28 +4464,28 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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).isEqualTo(expected);
         // 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).isEqualTo(expected);
 
         // 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()).isZero();
+        assertThat(dup1.refCnt()).isZero();
+        assertThat(dup2.refCnt()).isZero();
     }
 
     private void testMultipleRetainedSliceReleaseOriginal(
@@ -4410,7 +4502,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
                 retainedSlice1
                         ? buf.retainedSlice(buf.readerIndex() + 5, 3)
                         : buf.slice(buf.readerIndex() + 5, 3).retain();
-        assertEquals(0, slice1.compareTo(expected1));
+        assertThat(slice1).isEqualTo(expected1);
         // Simulate a handler that releases the original buffer, and propagates a slice.
         buf.release();
 
@@ -4418,32 +4510,32 @@ public abstract class AbstractByteBufTest extends TestLogger {
                 retainedSlice2
                         ? slice1.retainedSlice(slice1.readerIndex() + 1, 2)
                         : slice1.slice(slice1.readerIndex() + 1, 2).retain();
-        assertEquals(0, slice2.compareTo(expected2));
+        assertThat(slice2).isEqualTo(expected2);
 
         // 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).isEqualTo(expected3);
 
         // 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()).isZero();
+        assertThat(slice1.refCnt()).isZero();
+        assertThat(slice2.refCnt()).isZero();
+        assertThat(slice3.refCnt()).isZero();
     }
 
     private void testMultipleRetainedDuplicateReleaseOriginal(
@@ -4453,44 +4545,42 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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).isEqualTo(expected);
         // 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).isEqualTo(expected).isEqualTo(dup2.duplicate()).isEqualTo(dup2.slice());
 
         ByteBuf tmpBuf = dup2.retainedDuplicate();
-        assertEquals(0, dup2.compareTo(tmpBuf));
+        assertThat(dup2).isEqualTo(tmpBuf);
         tmpBuf.release();
         tmpBuf = dup2.retainedSlice();
-        assertEquals(0, dup2.compareTo(tmpBuf));
+        assertThat(dup2).isEqualTo(tmpBuf);
         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).isEqualTo(expected);
 
         // 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()).isZero();
+        assertThat(dup1.refCnt()).isZero();
+        assertThat(dup2.refCnt()).isZero();
+        assertThat(dup3.refCnt()).isZero();
     }
 
     private void testDuplicateContents(boolean retainedDuplicate) {
@@ -4498,12 +4588,11 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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).isEqualTo(buf).isEqualTo(dup.duplicate());
             ByteBuf b = dup.retainedDuplicate();
-            assertEquals(0, dup.compareTo(b));
+            assertThat(dup).isEqualTo(b);
             b.release();
-            assertEquals(0, dup.compareTo(dup.slice(dup.readerIndex(), dup.readableBytes())));
+            assertThat(dup).isEqualTo(dup.slice(dup.readerIndex(), dup.readableBytes()));
         } finally {
             if (retainedDuplicate) {
                 dup.release();
@@ -4513,45 +4602,45 @@ public abstract class AbstractByteBufTest extends TestLogger {
     }
 
     @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()).isOne();
+        assertThat(buf.duplicate().release()).isTrue();
+        assertThat(buf.refCnt()).isZero();
     }
 
     // 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() {
         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).hasSize(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,22 +4655,16 @@ public abstract class AbstractByteBufTest extends TestLogger {
                         ? 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()).isZero();
         buffer.release();
     }
 
     @Test
-    public void testReadBytesAndWriteBytesWithFileChannel() throws IOException {
+    void testReadBytesAndWriteBytesWithFileChannel() throws IOException {
         File file = File.createTempFile("file-channel", ".tmp");
-        RandomAccessFile randomAccessFile = null;
-        try {
-            randomAccessFile = new RandomAccessFile(file, "rw");
+        try (RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw")) {
             FileChannel channel = randomAccessFile.getChannel();
             // channelPosition should never be changed
             long channelPosition = channel.position();
@@ -4594,37 +4677,32 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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 {
-            if (randomAccessFile != null) {
-                randomAccessFile.close();
-            }
             file.delete();
         }
     }
 
     @Test
-    public void testGetBytesAndSetBytesWithFileChannel() throws IOException {
+    void testGetBytesAndSetBytesWithFileChannel() throws IOException {
         File file = File.createTempFile("file-channel", ".tmp");
-        RandomAccessFile randomAccessFile = null;
-        try {
-            randomAccessFile = new RandomAccessFile(file, "rw");
+        try (RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw")) {
             FileChannel channel = randomAccessFile.getChannel();
             // channelPosition should never be changed
             long channelPosition = channel.position();
@@ -4637,50 +4715,47 @@ public abstract class AbstractByteBufTest extends TestLogger {
             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();
         } finally {
-            if (randomAccessFile != null) {
-                randomAccessFile.close();
-            }
             file.delete();
         }
     }
 
     @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()).isZero();
+        assertThat(buffer2.release()).isTrue();
+        assertThat(buffer2.refCnt()).isZero();
     }
 
     @Test
-    public void testForEachByteDesc2() {
+    void testForEachByteDesc2() {
         byte[] expected = {1, 2, 3, 4};
         ByteBuf buf = newBuffer(expected.length);
         try {
@@ -4692,20 +4767,20 @@ public abstract class AbstractByteBufTest extends TestLogger {
                                 private int index = bytes.length - 1;
 
                                 @Override
-                                public boolean process(byte value) throws Exception {
+                                public boolean process(byte value) {
                                     bytes[index--] = value;
                                     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 {
@@ -4717,27 +4792,28 @@ public abstract class AbstractByteBufTest extends TestLogger {
                                 private int index;
 
                                 @Override
-                                public boolean process(byte value) throws Exception {
+                                public boolean process(byte value) {
                                     bytes[index++] = value;
                                     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 +4825,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
             final CountDownLatch innerLatch = new CountDownLatch(1);
 
             final ByteBuf buffer = newBuffer(4);
-            assertEquals(1, buffer.refCnt());
+            assertThat(buffer.refCnt()).isOne();
             final AtomicInteger cnt = new AtomicInteger(Integer.MAX_VALUE);
             Thread t1 =
                     new Thread(
@@ -4762,15 +4838,12 @@ public abstract class AbstractByteBufTest extends TestLogger {
                                     } else {
                                         released = buffer.release();
                                     }
-                                    assertTrue(released);
+                                    assertThat(released).isTrue();
                                     Thread t2 =
                                             new Thread(
-                                                    new Runnable() {
-                                                        @Override
-                                                        public void run() {
-                                                            cnt.set(buffer.refCnt());
-                                                            latch.countDown();
-                                                        }
+                                                    () -> {
+                                                        cnt.set(buffer.refCnt());
+                                                        latch.countDown();
                                                     });
                                     t2.start();
                                     try {
@@ -4785,7 +4858,7 @@ public abstract class AbstractByteBufTest extends TestLogger {
             t1.start();
 
             latch.await();
-            assertEquals(0, cnt.get());
+            assertThat(cnt).hasValue(0);
             innerLatch.countDown();
         }
     }
@@ -4902,65 +4975,67 @@ public abstract class AbstractByteBufTest extends TestLogger {
 
     private static final class TestByteProcessor implements ByteProcessor {
         @Override
-        public boolean process(byte value) throws Exception {
+        public boolean process(byte value) {
             return true;
         }
     }
 
-    @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 +5046,11 @@ public abstract class AbstractByteBufTest extends TestLogger {
         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 b927917e81f..1f4b2d9fa19 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,62 +20,61 @@ package org.apache.flink.runtime.io.network.buffer;
 
 import org.apache.flink.core.memory.MemorySegment;
 
-import org.junit.Test;
-
-import javax.annotation.Nullable;
+import org.junit.jupiter.api.Test;
 
 import java.nio.ByteBuffer;
-import java.nio.IntBuffer;
 import java.util.ArrayList;
 
 import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.assertContent;
 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.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEmptyBufferBuilder;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.toByteBuffer;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
 /** Tests for {@link BufferBuilder}. */
-public class BufferBuilderAndConsumerTest {
+class BufferBuilderAndConsumerTest {
     private static final int BUFFER_INT_SIZE = 10;
     private static final int BUFFER_SIZE = BUFFER_INT_SIZE * Integer.BYTES;
 
     @Test
-    public void referenceCounting() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void referenceCounting() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void append() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void multipleAppends() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer();
 
         bufferBuilder.appendAndCommit(toByteBuffer(0, 1));
@@ -86,8 +85,8 @@ public class BufferBuilderAndConsumerTest {
     }
 
     @Test
-    public void multipleNotCommittedAppends() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void multipleNotCommittedAppends() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer();
 
         bufferBuilder.append(toByteBuffer(0, 1));
@@ -102,34 +101,35 @@ public class BufferBuilderAndConsumerTest {
     }
 
     @Test
-    public void appendOverSize() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void appendOverSize() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         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();
+        bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        bufferBuilder.createBufferConsumer();
+    @Test
+    void creatingBufferConsumerTwice() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         bufferBuilder.createBufferConsumer();
+        assertThatThrownBy(bufferBuilder::createBufferConsumer)
+                .isInstanceOf(IllegalStateException.class);
     }
 
     @Test
-    public void copy() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void copy() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         BufferConsumer bufferConsumer1 = bufferBuilder.createBufferConsumer();
 
         bufferBuilder.appendAndCommit(toByteBuffer(0, 1));
@@ -154,17 +154,17 @@ public class BufferBuilderAndConsumerTest {
     }
 
     @Test
-    public void buildEmptyBuffer() {
-        try (BufferBuilder bufferBuilder = createBufferBuilder()) {
+    void buildEmptyBuffer() {
+        try (BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE)) {
             Buffer buffer = buildSingleBuffer(bufferBuilder);
-            assertEquals(0, buffer.getSize());
+            assertThat(buffer.getSize()).isZero();
             assertContent(buffer, FreeingBufferRecycler.INSTANCE);
         }
     }
 
     @Test
-    public void buildingBufferMultipleTimes() {
-        try (BufferBuilder bufferBuilder = createBufferBuilder()) {
+    void buildingBufferMultipleTimes() {
+        try (BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE)) {
             try (BufferConsumer bufferConsumer = bufferBuilder.createBufferConsumer()) {
                 bufferBuilder.appendAndCommit(toByteBuffer(0, 1));
                 bufferBuilder.appendAndCommit(toByteBuffer(2));
@@ -190,45 +190,43 @@ public class BufferBuilderAndConsumerTest {
     }
 
     @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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        assertEquals(bufferBuilder.getMaxCapacity(), bufferBuilder.getWritableBytes());
+    void testWritableBytes() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
+        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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+    void testWritableBytesWhenFull() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         bufferBuilder.append(toByteBuffer(new int[bufferBuilder.getMaxCapacity()]));
-        assertEquals(0, bufferBuilder.getWritableBytes());
+        assertThat(bufferBuilder.getWritableBytes()).isZero();
     }
 
     @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 class BufferBuilderAndConsumerTest {
         bufferBuilder.close();
 
         // then: Recycling successfully finished.
-        assertEquals(1, recycler.recycleInvocationCounter);
+        assertThat(recycler.recycleInvocationCounter).isOne();
     }
 
     @Test
-    public void recycleConsumerAndBufferBuilder() {
+    void recycleConsumerAndBufferBuilder() {
         // given: Recycler with the counter of recycling invocation.
         CountedRecycler recycler = new CountedRecycler();
         BufferBuilder bufferBuilder =
@@ -255,121 +253,89 @@ public class BufferBuilderAndConsumerTest {
         bufferBuilder.close();
 
         // then: Nothing happened because BufferBuilder has already consumer.
-        assertEquals(0, recycler.recycleInvocationCounter);
+        assertThat(recycler.recycleInvocationCounter).isZero();
 
         // when: Close the consumer.
         bufferConsumer.close();
 
         // then: Recycling successfully finished.
-        assertEquals(1, recycler.recycleInvocationCounter);
+        assertThat(recycler.recycleInvocationCounter).isOne();
     }
 
     @Test
-    public void trimToAvailableSize() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity());
+    void trimToAvailableSize() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
+        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()).isZero();
     }
 
     @Test
-    public void trimToNegativeSize() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity());
+    void trimToNegativeSize() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
+        assertThat(bufferBuilder.getMaxCapacity()).isEqualTo(BUFFER_SIZE);
 
         bufferBuilder.trim(-1);
-        assertEquals(0, bufferBuilder.getMaxCapacity());
+        assertThat(bufferBuilder.getMaxCapacity()).isZero();
     }
 
     @Test
-    public void trimToSizeLessThanWritten() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity());
+    void trimToSizeLessThanWritten() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
+        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() {
-        BufferBuilder bufferBuilder = createBufferBuilder();
-        assertEquals(BUFFER_SIZE, bufferBuilder.getMaxCapacity());
+    void trimToSizeGreaterThanMax() {
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
+        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) {
-        BufferBuilder bufferBuilder = createBufferBuilder();
+        BufferBuilder bufferBuilder = createEmptyBufferBuilder(BUFFER_SIZE);
         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()).isZero();
 
         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());
-
-        assertEquals(0, bufferConsumer.build().getSize());
-        assertTrue(bufferConsumer.isFinished());
-    }
-
-    public static ByteBuffer toByteBuffer(int... data) {
-        ByteBuffer byteBuffer = ByteBuffer.allocate(data.length * Integer.BYTES);
-        byteBuffer.asIntBuffer().put(data);
-        return byteBuffer;
-    }
-
-    private static void assertContent(BufferConsumer actualConsumer, int... expected) {
-        assertFalse(actualConsumer.isFinished());
-        Buffer buffer = actualConsumer.build();
-        assertFalse(buffer.isRecycled());
-        assertContent(buffer, FreeingBufferRecycler.INSTANCE, expected);
-        assertEquals(expected.length * Integer.BYTES, buffer.getSize());
-        buffer.recycleBuffer();
-    }
-
-    public static void assertContent(
-            Buffer actualBuffer, @Nullable BufferRecycler recycler, int... expected) {
-        IntBuffer actualIntBuffer = actualBuffer.getNioBufferReadable().asIntBuffer();
-        int[] actual = new int[actualIntBuffer.limit()];
-        actualIntBuffer.get(actual);
-        assertArrayEquals(expected, actual);
-
-        if (recycler != null) {
-            assertEquals(recycler, actualBuffer.getRecycler());
-        }
-    }
+        assertThat(actualWrittenBytes).isEqualTo(expectedWrittenBytes);
+        assertThat(bufferBuilder.isFinished()).isTrue();
+        assertThat(bufferConsumer.isFinished()).isFalse();
+        assertThat(bufferConsumer.getWrittenBytes()).isEqualTo(expectedWrittenBytes);
 
-    private static BufferBuilder createBufferBuilder() {
-        return new BufferBuilder(
-                allocateUnpooledSegment(BUFFER_SIZE), FreeingBufferRecycler.INSTANCE);
+        assertThat(bufferConsumer.build().getSize()).isZero();
+        assertThat(bufferBuilder.isFinished()).isTrue();
     }
 
     private static class CountedRecycler implements BufferRecycler {
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 a8ef32f5224..62ce872d2e6 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
@@ -21,11 +21,14 @@ package org.apache.flink.runtime.io.network.buffer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 
+import javax.annotation.Nullable;
+
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
+import java.nio.IntBuffer;
 
 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 +123,7 @@ public class BufferBuilderTestUtils {
         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 +143,7 @@ public class BufferBuilderTestUtils {
         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++);
         }
     }
 
@@ -158,4 +161,51 @@ public class BufferBuilderTestUtils {
                 MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
                 FreeingBufferRecycler.INSTANCE);
     }
+
+    public static ByteBuffer toByteBuffer(int... data) {
+        ByteBuffer byteBuffer = ByteBuffer.allocate(data.length * Integer.BYTES);
+        byteBuffer.asIntBuffer().put(data);
+        return byteBuffer;
+    }
+
+    public static void assertContent(BufferConsumer actualConsumer, int... expected) {
+        assertThat(actualConsumer.isFinished()).isFalse();
+        Buffer buffer = actualConsumer.build();
+        assertThat(buffer.isRecycled()).isFalse();
+        assertContent(buffer, FreeingBufferRecycler.INSTANCE, expected);
+        assertThat(buffer.getSize()).isEqualTo(expected.length * Integer.BYTES);
+        buffer.recycleBuffer();
+    }
+
+    public static void assertContent(
+            Buffer actualBuffer, @Nullable BufferRecycler recycler, int... expected) {
+        IntBuffer actualIntBuffer = actualBuffer.getNioBufferReadable().asIntBuffer();
+        int[] actual = new int[actualIntBuffer.limit()];
+        actualIntBuffer.get(actual);
+        assertThat(actual).containsExactly(expected);
+
+        if (recycler != null) {
+            assertThat(actualBuffer.getRecycler()).isEqualTo(recycler);
+        }
+    }
+
+    /**
+     * Returns whether the stack trace represents a Thread in a blocking buffer request.
+     *
+     * @param stackTrace Stack trace of the Thread to check
+     * @return Flag indicating whether the Thread is in a blocking buffer request or not
+     */
+    public static boolean isInBlockingBufferRequest(StackTraceElement[] stackTrace) {
+        if (stackTrace.length >= 8) {
+            for (int x = 0; x < stackTrace.length - 2; x++) {
+                if (stackTrace[x].getMethodName().equals("get")
+                        && stackTrace[x + 2]
+                                .getClassName()
+                                .equals(LocalBufferPool.class.getName())) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
 }
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 d269151205b..67292727d3a 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,25 +22,23 @@ import org.apache.flink.core.memory.MemorySegment;
 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)
-public class BufferCompressionTest {
+@ExtendWith(ParameterizedTestExtension.class)
+class BufferCompressionTest {
 
     private static final int BUFFER_SIZE = 4 * 1024 * 1024;
 
@@ -95,111 +93,143 @@ public class 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 @@ public class BufferCompressionTest {
     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 7122d3045eb..1eae88e9383 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
@@ -18,23 +18,21 @@
 
 package org.apache.flink.runtime.io.network.buffer;
 
-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.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.assertContent;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEmptyBufferBuilder;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.toByteBuffer;
+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<BufferConsumerWithPartialRecordLength> 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()).isZero();
+        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()).isZero();
     }
 
     @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()).isZero();
 
         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()).isZero();
 
         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,14 +115,14 @@ public class BufferConsumerWithPartialRecordLengthTest {
 
         // 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);
     }
 
     private void writeToBuffer(ByteBuffer record) {
         if (builder == null) {
-            builder = createBufferBuilder();
+            builder = createEmptyBufferBuilder(BUFFER_SIZE);
             buffers.add(
                     new BufferConsumerWithPartialRecordLength(
                             builder.createBufferConsumerFromBeginning(), 0));
@@ -133,7 +131,7 @@ public class BufferConsumerWithPartialRecordLengthTest {
 
         while (record.hasRemaining()) {
             builder.finish();
-            builder = createBufferBuilder();
+            builder = createEmptyBufferBuilder(BUFFER_SIZE);
             final int partialRecordBytes = builder.appendAndCommit(record);
             buffers.add(
                     new BufferConsumerWithPartialRecordLength(
@@ -145,10 +143,4 @@ public class BufferConsumerWithPartialRecordLengthTest {
             builder = null;
         }
     }
-
-    private BufferBuilder createBufferBuilder() {
-        return new BufferBuilder(
-                MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE),
-                FreeingBufferRecycler.INSTANCE);
-    }
 }
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 19bd2e7eae3..e60c2806fa7 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 @@ package org.apache.flink.runtime.io.network.buffer;
 
 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.List;
 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() {
-        String msg = "Did not return all buffers to network buffer pool after test.";
+    @AfterEach
+    void verifyAllBuffersReturned() {
         try {
-            assertEquals(msg, numBuffers, networkBufferPool.getNumberOfAvailableMemorySegments());
+            assertThat(networkBufferPool.getNumberOfAvailableMemorySegments())
+                    .withFailMessage(
+                            "Did not return all buffers to network buffer pool after test.")
+                    .isEqualTo(numBuffers);
         } finally {
             // in case buffers have actually been requested, we must release them again
             networkBufferPool.destroyAllBufferPools();
@@ -72,24 +66,28 @@ public class BufferPoolFactoryTest {
 
     /** 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() {
+        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 class BufferPoolFactoryTest {
      * {@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 class BufferPoolFactoryTest {
             // 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()).isZero();
             // 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 class BufferPoolFactoryTest {
     }
 
     @Test
-    public void testBoundedPools() throws IOException {
+    void testBoundedPools() throws IOException {
         BufferPool bufferPool1 = networkBufferPool.createBufferPool(1, 1);
-        assertEquals(1, bufferPool1.getNumBuffers());
+        assertThat(bufferPool1.getNumBuffers()).isOne();
 
         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 class BufferPoolFactoryTest {
      * 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()).isOne();
+            assertThat(second.getNumBuffers()).isOne();
+            assertThat(third.getNumBuffers()).isOne();
 
             // similar to #verifyAllBuffersReturned()
             String msg = "Wrong number of available segments after creating buffer pools.";
-            assertEquals(msg, 0, globalPool.getNumberOfAvailableMemorySegments());
+            assertThat(globalPool.getNumberOfAvailableMemorySegments())
+                    .withFailMessage(msg)
+                    .isZero();
         } finally {
             // in case buffers have actually been requested, we must release them again
             globalPool.destroyAllBufferPools();
@@ -311,52 +319,62 @@ public class BufferPoolFactoryTest {
      * 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<MemorySegment> 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<MemorySegment> 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<MemorySegment> 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 class BufferPoolFactoryTest {
     }
 
     @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 class BufferPoolFactoryTest {
                                         * 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 class BufferPoolFactoryTest {
                                         * 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 class BufferPoolFactoryTest {
     }
 
     @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 class BufferPoolFactoryTest {
                 for (BufferPool pool : pools) {
                     numDistributedBuffers += pool.getNumBuffers();
                 }
-                assertEquals(maxTotalUsed, numDistributedBuffers);
+                assertThat(numDistributedBuffers).isEqualTo(maxTotalUsed);
             }
 
             pools.forEach(BufferPool::lazyDestroy);
@@ -445,21 +461,24 @@ public class BufferPoolFactoryTest {
     }
 
     @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 87f9ebf3abc..901ee79efc3 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,25 @@ package org.apache.flink.runtime.io.network.buffer;
 
 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.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.isInBlockingBufferRequest;
+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 {
+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 +49,7 @@ public class LocalBufferPoolDestroyTest {
      * and we check whether the request Thread threw the expected Exception.
      */
     @Test
-    public void testDestroyWhileBlockingRequest() throws Exception {
+    void testDestroyWhileBlockingRequest() throws Exception {
         AtomicReference<Exception> asyncException = new AtomicReference<>();
 
         NetworkBufferPool networkBufferPool = null;
@@ -65,8 +60,8 @@ public class LocalBufferPoolDestroyTest {
             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 +83,9 @@ public class LocalBufferPoolDestroyTest {
             }
 
             // 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 +94,9 @@ public class LocalBufferPoolDestroyTest {
             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")
+                    .isInstanceOf(CancelTaskException.class);
         } finally {
             if (localBufferPool != null) {
                 localBufferPool.lazyDestroy();
@@ -111,33 +109,13 @@ public class LocalBufferPoolDestroyTest {
         }
     }
 
-    /**
-     * Returns whether the stack trace represents a Thread in a blocking buffer request.
-     *
-     * @param stackTrace Stack trace of the Thread to check
-     * @return Flag indicating whether the Thread is in a blocking buffer request or not
-     */
-    public static boolean isInBlockingBufferRequest(StackTraceElement[] stackTrace) {
-        if (stackTrace.length >= 8) {
-            for (int x = 0; x < stackTrace.length - 2; x++) {
-                if (stackTrace[x].getMethodName().equals("get")
-                        && stackTrace[x + 2]
-                                .getClassName()
-                                .equals(LocalBufferPool.class.getName())) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-
     /** Task triggering a blocking buffer request (the test assumes that no buffer is available). */
     private static class BufferRequestTask implements Runnable {
 
         private final BufferPool bufferPool;
         private final AtomicReference<Exception> asyncException;
 
-        public BufferRequestTask(BufferPool bufferPool, AtomicReference<Exception> asyncException) {
+        BufferRequestTask(BufferPool bufferPool, AtomicReference<Exception> asyncException) {
             this.bufferPool = bufferPool;
             this.asyncException = asyncException;
         }
@@ -146,7 +124,7 @@ public class LocalBufferPoolDestroyTest {
         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 6126d07e4ff..6f407b03a08 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<ExecutorService> EXECUTOR_RESOURCE =
+    private static final TestExecutorExtension<ExecutorService> EXECUTOR_EXTENSION =
             new TestExecutorExtension<>(Executors::newCachedThreadPool);
 
     @BeforeEach
@@ -73,7 +73,7 @@ class LocalBufferPoolTest {
         networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
         localBufferPool = new LocalBufferPool(networkBufferPool, 1);
 
-        assertThat(localBufferPool.getNumberOfAvailableMemorySegments()).isEqualTo(1);
+        assertThat(localBufferPool.getNumberOfAvailableMemorySegments()).isOne();
     }
 
     @AfterEach
@@ -82,9 +82,8 @@ class LocalBufferPoolTest {
             localBufferPool.lazyDestroy();
         }
 
-        String msg = "Did not return all buffers to memory segment pool after test.";
         assertThat(networkBufferPool.getNumberOfAvailableMemorySegments())
-                .withFailMessage(msg)
+                .withFailMessage("Did not return all buffers to memory segment pool after test.")
                 .isEqualTo(numBuffers);
         // no other local buffer pools used than the one above, but call just in case
         networkBufferPool.destroyAllBufferPools();
@@ -116,7 +115,7 @@ class LocalBufferPoolTest {
             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);
 
@@ -129,7 +128,7 @@ class LocalBufferPoolTest {
     }
 
     @Test
-    @Timeout(10) // timeout can indicate a potential deadlock
+    @Timeout(value = 10) // timeout can indicate a potential deadlock
     void testReserveSegmentsAndCancel() throws Exception {
         int totalSegments = 4;
         int segmentsToReserve = 2;
@@ -587,7 +586,7 @@ class LocalBufferPoolTest {
         Future<Boolean>[] taskResults = new Future[numConcurrentTasks];
         for (int i = 0; i < numConcurrentTasks; i++) {
             taskResults[i] =
-                    EXECUTOR_RESOURCE
+                    EXECUTOR_EXTENSION
                             .getExecutor()
                             .submit(
                                     new BufferRequesterTask(
@@ -855,7 +854,7 @@ class LocalBufferPoolTest {
         bufferPool.lazyDestroy();
     }
 
-    private void assertRequestedBufferAndIsAvailable(
+    private static void assertRequestedBufferAndIsAvailable(
             LocalBufferPool bufferPool,
             int numberOfRequestedOverdraftBuffer,
             int numberOfRequestedBuffer,
@@ -920,7 +919,7 @@ class LocalBufferPoolTest {
         }
 
         @Override
-        public Boolean call() throws Exception {
+        public Boolean call() {
             try {
                 for (int i = 0; i < numBuffersToRequest; i++) {
                     Buffer buffer = checkNotNull(bufferProvider.requestBuffer());
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 20fd5964b48..6c3731716fa 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.memory.MemorySegment;
 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;
@@ -33,7 +32,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
-import java.util.Optional;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CompletableFuture;
@@ -43,102 +41,71 @@ import java.util.concurrent.Executors;
 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;
 
 /** Tests for {@link NetworkBufferPool}. */
-public class NetworkBufferPoolTest extends TestLogger {
+class NetworkBufferPoolTest {
 
     @Test
-    public void testCreatePoolAfterDestroy() {
-        try {
-            final int bufferSize = 128;
-            final int numBuffers = 10;
+    void testCreatePoolAfterDestroy() {
+        final int bufferSize = 128;
+        final int numBuffers = 10;
 
-            NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize);
-            assertThat(globalPool.getNumberOfRegisteredBufferPools(), is(0));
+        NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize);
+        assertThat(globalPool.getNumberOfRegisteredBufferPools()).isZero();
 
-            globalPool.destroy();
+        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))
+                .isInstanceOf(IllegalStateException.class);
 
-            try {
-                globalPool.createBufferPool(2, 10);
-                fail("Should throw an IllegalStateException");
-            } catch (IllegalStateException e) {
-                // yippie!
-            }
+        assertThatThrownBy(() -> globalPool.createBufferPool(2, 10))
+                .isInstanceOf(IllegalStateException.class);
 
-            try {
-                globalPool.createBufferPool(2, Integer.MAX_VALUE);
-                fail("Should throw an IllegalStateException");
-            } catch (IllegalStateException e) {
-                // yippie!
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            fail(e.getMessage());
-        }
+        assertThatThrownBy(() -> globalPool.createBufferPool(2, Integer.MAX_VALUE))
+                .isInstanceOf(IllegalStateException.class);
     }
 
     @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()).isZero();
 
-        assertThat(globalPool.getTotalMemory(), is((long) numBuffers * bufferSize));
-        assertThat(globalPool.getAvailableMemory(), is((long) numBuffers * bufferSize));
-        assertThat(globalPool.getUsedMemory(), is(0L));
+        assertThat(globalPool.getTotalMemory()).isEqualTo(numBuffers * bufferSize);
+        assertThat(globalPool.getAvailableMemory()).isEqualTo(numBuffers * bufferSize);
+        assertThat(globalPool.getUsedMemory()).isZero();
     }
 
     @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()).isOne();
 
-        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(numBuffers * bufferSize);
+        assertThat(globalPool.getAvailableMemory()).isEqualTo((numBuffers - 1) * bufferSize);
+        assertThat(globalPool.getUsedMemory()).isEqualTo(bufferSize);
     }
 
     @Test
-    public void testMemoryUsageInTheContextOfMemoryPoolDestruction() {
+    void testMemoryUsageInTheContextOfMemoryPoolDestruction() {
         final int bufferSize = 128;
         final int numBuffers = 10;
 
@@ -146,26 +113,26 @@ public class NetworkBufferPoolTest extends TestLogger {
 
         globalPool.destroy();
 
-        assertThat(globalPool.getTotalNumberOfMemorySegments(), is(0));
-        assertThat(globalPool.getNumberOfAvailableMemorySegments(), is(0));
-        assertThat(globalPool.getNumberOfUsedMemorySegments(), is(0));
+        assertThat(globalPool.getTotalNumberOfMemorySegments()).isZero();
+        assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero();
+        assertThat(globalPool.getNumberOfUsedMemorySegments()).isZero();
 
-        assertThat(globalPool.getTotalMemory(), is(0L));
-        assertThat(globalPool.getAvailableMemory(), is(0L));
-        assertThat(globalPool.getUsedMemory(), is(0L));
+        assertThat(globalPool.getTotalMemory()).isZero();
+        assertThat(globalPool.getAvailableMemory()).isZero();
+        assertThat(globalPool.getUsedMemory()).isZero();
     }
 
     @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()).isOne();
+        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 +146,49 @@ public class NetworkBufferPoolTest extends TestLogger {
             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).hasSize(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()).isZero();
 
         // buffers are not yet recycled
-        assertEquals(0, globalPool.getNumberOfAvailableMemorySegments());
+        assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero();
 
         // 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).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).isInstanceOf(CancelTaskException.class);
 
-        try {
-            nonFixedPool.requestBuffer();
-            fail("Should fail with an CancelTaskException");
-        } catch (CancelTaskException e) {
-            // stayin' alive
-        }
+        assertThatThrownBy(nonFixedPool::requestBuffer).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 +196,7 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +204,11 @@ public class NetworkBufferPoolTest extends TestLogger {
         List<MemorySegment> memorySegments = Collections.emptyList();
         try {
             memorySegments = globalPool.requestUnpooledMemorySegments(numBuffers / 2);
-            assertEquals(memorySegments.size(), numBuffers / 2);
+            assertThat(memorySegments).hasSize(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,16 +220,15 @@ public class NetworkBufferPoolTest extends TestLogger {
      * required buffers exceeding the capacity of {@link NetworkBufferPool}.
      */
     @Test
-    public void testRequestMemorySegmentsMoreThanTotalBuffers() {
+    void testRequestMemorySegmentsMoreThanTotalBuffers() {
         final int numBuffers = 10;
 
         NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128);
 
         try {
-            globalPool.requestUnpooledMemorySegments(numBuffers + 1);
-            fail("Should throw an IOException");
-        } catch (IOException e) {
-            assertEquals(globalPool.getNumberOfAvailableMemorySegments(), numBuffers);
+            assertThatThrownBy(() -> globalPool.requestUnpooledMemorySegments(numBuffers + 1))
+                    .isInstanceOf(IOException.class);
+            assertThat(globalPool.getNumberOfAvailableMemorySegments()).isEqualTo(numBuffers);
         } finally {
             globalPool.destroy();
         }
@@ -289,113 +239,106 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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();
 
             // request 5 segments
             List<MemorySegment> segments1 =
                     globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest);
-            assertFalse(globalPool.getAvailableFuture().isDone());
-            assertEquals(numberOfSegmentsToRequest, segments1.size());
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
+            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();
 
             List<MemorySegment> segments2 =
                     globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest);
-            assertFalse(globalPool.getAvailableFuture().isDone());
-            assertEquals(numberOfSegmentsToRequest, segments2.size());
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
+            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()).isZero();
         }
     }
 
     @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(
-                            Optional.of(
-                                    "Memory usage [120%] is too high to satisfy all of the requests. "
-                                            + "This can severely impact network throughput. "
-                                            + "Please consider increasing available network memory, "
-                                            + "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()));
+            assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(60);
+            assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(120);
+            assertThat(globalPool.getUsageWarning())
+                    .hasValue(
+                            "Memory usage [120%] is too high to satisfy all of the requests. "
+                                    + "This can severely impact network throughput. "
+                                    + "Please consider increasing available network memory, "
+                                    + "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()).isEmpty();
 
             BufferPool bufferPool2 = globalPool.createBufferPool(10, 20);
 
-            assertEquals(80, globalPool.getEstimatedNumberOfRequestedMemorySegments());
-            assertEquals(160, globalPool.getEstimatedRequestedSegmentsUsage());
-            assertThat(
-                    globalPool.getUsageWarning(),
-                    equalTo(
-                            Optional.of(
-                                    "Memory usage [160%] is too high to satisfy all of the requests. "
-                                            + "This can severely impact network throughput. "
-                                            + "Please consider increasing available network memory, "
-                                            + "or decreasing configured size of network buffer pools. ("
-                                            + "totalMemory=6.250kb (6400 bytes), "
-                                            + "requestedMemory=10.000kb (10240 bytes), "
-                                            + "missingMemory=3.750kb (3840 bytes))")));
+            assertThat(globalPool.getEstimatedNumberOfRequestedMemorySegments()).isEqualTo(80);
+            assertThat(globalPool.getEstimatedRequestedSegmentsUsage()).isEqualTo(160);
+            assertThat(globalPool.getUsageWarning())
+                    .hasValue(
+                            "Memory usage [160%] is too high to satisfy all of the requests. "
+                                    + "This can severely impact network throughput. "
+                                    + "Please consider increasing available network memory, "
+                                    + "or decreasing configured size of network buffer pools. ("
+                                    + "totalMemory=6.250kb (6400 bytes), "
+                                    + "requestedMemory=10.000kb (10240 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())
+                    .hasValue("Memory usage [80%] went back to normal");
+            assertThat(globalPool.getUsageWarning()).isEmpty();
         }
     }
 
@@ -403,13 +346,13 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +361,7 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +377,7 @@ public class NetworkBufferPoolTest extends TestLogger {
             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 +401,7 @@ public class NetworkBufferPoolTest extends TestLogger {
             // 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 +419,12 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +447,9 @@ public class NetworkBufferPoolTest extends TestLogger {
         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 +460,12 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +490,7 @@ public class NetworkBufferPoolTest extends TestLogger {
         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 +505,7 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +518,7 @@ public class NetworkBufferPoolTest extends TestLogger {
             localBufferPool.requestBuffer();
         }
 
-        assertEquals(0, globalPool.getNumberOfAvailableMemorySegments());
+        assertThat(globalPool.getNumberOfAvailableMemorySegments()).isZero();
 
         CheckedThread asyncRequest =
                 new CheckedThread() {
@@ -588,8 +531,9 @@ public class NetworkBufferPoolTest extends TestLogger {
         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 +546,33 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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();
 
             // request the first segment
             final MemorySegment segment1 = checkNotNull(globalPool.requestPooledMemorySegment());
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
             // request the second segment
             final MemorySegment segment2 = checkNotNull(globalPool.requestPooledMemorySegment());
-            assertFalse(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
 
             final CompletableFuture<?> availableFuture = globalPool.getAvailableFuture();
 
             // recycle the first segment
             globalPool.recyclePooledMemorySegment(segment1);
-            assertTrue(availableFuture.isDone());
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(availableFuture).isDone();
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
             // recycle the second segment
             globalPool.recyclePooledMemorySegment(segment2);
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
         } finally {
             globalPool.destroy();
@@ -642,7 +586,7 @@ public class NetworkBufferPoolTest extends TestLogger {
      * NetworkBufferPool#recycleUnpooledMemorySegments(Collection)}.
      */
     @Test
-    public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception {
+    void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception {
         final int numberOfSegmentsToRequest = 5;
         final int numBuffers = 2 * numberOfSegmentsToRequest;
 
@@ -650,38 +594,38 @@ public class NetworkBufferPoolTest extends TestLogger {
 
         try {
             // the global pool should be in available state initially
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
             // request 5 segments
             List<MemorySegment> segments1 =
                     globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest);
-            assertTrue(globalPool.getAvailableFuture().isDone());
-            assertEquals(numberOfSegmentsToRequest, segments1.size());
+            assertThat(globalPool.getAvailableFuture()).isDone();
+            assertThat(segments1).hasSize(numberOfSegmentsToRequest);
 
             // request another 5 segments
             List<MemorySegment> segments2 =
                     globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest);
-            assertFalse(globalPool.getAvailableFuture().isDone());
-            assertEquals(numberOfSegmentsToRequest, segments2.size());
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
+            assertThat(segments2).hasSize(numberOfSegmentsToRequest);
 
             // recycle 5 segments
             CompletableFuture<?> availableFuture = globalPool.getAvailableFuture();
             globalPool.recycleUnpooledMemorySegments(segments1);
-            assertTrue(availableFuture.isDone());
+            assertThat(availableFuture).isDone();
 
             // request another 5 segments
             final List<MemorySegment> segments3 =
                     globalPool.requestUnpooledMemorySegments(numberOfSegmentsToRequest);
-            assertFalse(globalPool.getAvailableFuture().isDone());
-            assertEquals(numberOfSegmentsToRequest, segments3.size());
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
+            assertThat(segments3).hasSize(numberOfSegmentsToRequest);
 
             // recycle another 5 segments
             globalPool.recycleUnpooledMemorySegments(segments2);
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
             // recycle the last 5 segments
             globalPool.recycleUnpooledMemorySegments(segments3);
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
 
         } finally {
             globalPool.destroy();
@@ -693,8 +637,7 @@ public class NetworkBufferPoolTest extends TestLogger {
      * 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 +654,7 @@ public class NetworkBufferPoolTest extends TestLogger {
                 final BufferPool localPool =
                         globalPool.createBufferPool(localPoolRequiredSize, localPoolMaxSize);
                 localBufferPools.add(localPool);
-                assertTrue(localPool.getAvailableFuture().isDone());
+                assertThat(localPool.getAvailableFuture()).isDone();
             }
 
             // request some segments from the global pool in two different ways
@@ -722,9 +665,9 @@ public class NetworkBufferPoolTest extends TestLogger {
             final List<MemorySegment> exclusiveSegments =
                     globalPool.requestUnpooledMemorySegments(
                             globalPool.getNumberOfAvailableMemorySegments() - 1);
-            assertTrue(globalPool.getAvailableFuture().isDone());
+            assertThat(globalPool.getAvailableFuture()).isDone();
             for (final BufferPool localPool : localBufferPools) {
-                assertTrue(localPool.getAvailableFuture().isDone());
+                assertThat(localPool.getAvailableFuture()).isDone();
             }
 
             // blocking request buffers form local buffer pools
@@ -751,18 +694,18 @@ public class NetworkBufferPoolTest extends TestLogger {
             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).isNotDone();
 
             final List<CompletableFuture<?>> localPoolAvailableFutures =
                     new ArrayList<>(numLocalBufferPool);
             for (BufferPool localPool : localBufferPools) {
                 CompletableFuture<?> localPoolAvailableFuture = localPool.getAvailableFuture();
                 localPoolAvailableFutures.add(localPoolAvailableFuture);
-                assertFalse(localPoolAvailableFuture.isDone());
+                assertThat(localPoolAvailableFuture).isNotDone();
             }
 
             // recycle the previously requested segments
@@ -771,20 +714,20 @@ public class NetworkBufferPoolTest extends TestLogger {
             }
             globalPool.recycleUnpooledMemorySegments(exclusiveSegments);
 
-            assertTrue(globalPoolAvailableFuture.isDone());
+            assertThat(globalPoolAvailableFuture).isDone();
             for (CompletableFuture<?> localPoolAvailableFuture : localPoolAvailableFutures) {
-                assertTrue(localPoolAvailableFuture.isDone());
+                assertThat(localPoolAvailableFuture).isDone();
             }
 
             // 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()).isZero();
+            assertThat(globalPool.getAvailableFuture()).isNotDone();
             for (BufferPool localPool : localBufferPools) {
-                assertFalse(localPool.getAvailableFuture().isDone());
-                assertEquals(localPoolMaxSize, localPool.bestEffortGetNumOfUsedBuffers());
+                assertThat(localPool.getAvailableFuture()).isNotDone();
+                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 8e796fe0722..262611e6821 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.MemorySegment;
 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 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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()).isZero();
+        assertThat(buffer.writerIndex()).isZero();
         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 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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 class NetworkBufferTest extends AbstractByteBufTest {
      */
     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()).isZero();
     }
 
     @Test
-    public void testDataBufferRetainBuffer() {
+    void testDataBufferRetainBuffer() {
         testRetainBuffer(true);
     }
 
     @Test
-    public void testEventBufferRetainBuffer() {
+    void testEventBufferRetainBuffer() {
         testRetainBuffer(false);
     }
 
@@ -189,19 +184,19 @@ public class NetworkBufferTest extends AbstractByteBufTest {
      */
     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 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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()).isZero();
+        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()).isZero();
+        assertThat(slice.getSize()).isEqualTo(10);
     }
 
     @Test
-    public void testDataBufferCreateSlice2() {
+    void testDataBufferCreateSlice2() {
         testCreateSlice2(true);
     }
 
     @Test
-    public void testEventBufferCreateSlice2() {
+    void testEventBufferCreateSlice2() {
         testCreateSlice2(false);
     }
 
@@ -236,42 +231,43 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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()).isZero();
+        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()).isZero();
+        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())
+                .isEqualTo(buffer.maxCapacity());
     }
 
     @Test
-    public void testDataBufferGetSetReaderIndex() {
+    void testDataBufferGetSetReaderIndex() {
         testGetSetReaderIndex(true);
     }
 
     @Test
-    public void testEventBufferGetSetReaderIndex() {
+    void testEventBufferGetSetReaderIndex() {
         testGetSetReaderIndex(false);
     }
 
@@ -281,67 +277,67 @@ public class NetworkBufferTest extends AbstractByteBufTest {
      */
     private static void testGetSetReaderIndex(boolean isBuffer) {
         NetworkBuffer buffer = newBuffer(100, 1024, isBuffer);
-        assertEquals(0, buffer.getReaderIndex());
+        assertThat(buffer.getReaderIndex()).isZero();
 
         // fake some data
         buffer.setSize(100);
-        assertEquals(0, buffer.getReaderIndex());
+        assertThat(buffer.getReaderIndex()).isZero();
         buffer.setReaderIndex(1);
-        assertEquals(1, buffer.getReaderIndex());
+        assertThat(buffer.getReaderIndex()).isOne();
     }
 
     @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()).isZero(); // initially 0
+        assertThat(buffer.writerIndex()).isEqualTo(buffer.getSize());
+        assertThat(buffer.readerIndex()).isZero(); // 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()).isZero(); // 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()).isZero();
         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()).isZero();
     }
 
     @Test
-    public void testDataBufferGetNioBufferReadable() {
+    void testDataBufferGetNioBufferReadable() {
         testGetNioBufferReadable(true);
     }
 
     @Test
-    public void testEventBufferGetNioBufferReadable() {
+    void testEventBufferGetNioBufferReadable() {
         testGetNioBufferReadable(false);
     }
 
@@ -349,32 +345,32 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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()).isZero();
+        assertThat(byteBuffer.limit()).isZero();
+        assertThat(byteBuffer.capacity()).isZero();
 
         // 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()).isZero();
+        assertThat(byteBuffer.limit()).isZero();
+        assertThat(byteBuffer.capacity()).isZero();
         // 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()).isZero();
+        assertThat(buffer.getSize()).isEqualTo(10);
     }
 
     @Test
-    public void testGetNioBufferReadableThreadSafe() {
+    void testGetNioBufferReadableThreadSafe() {
         NetworkBuffer buffer = newBuffer(1024, 1024);
         testGetNioBufferReadableThreadSafe(buffer);
     }
@@ -383,19 +379,21 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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 +401,32 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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()).isOne();
+        assertThat(byteBuffer.limit()).isOne();
+        assertThat(byteBuffer.capacity()).isOne();
 
         // 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()).isOne();
+        assertThat(byteBuffer.limit()).isOne();
+        assertThat(byteBuffer.capacity()).isOne();
         // 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()).isZero();
+        assertThat(buffer.getSize()).isEqualTo(10);
     }
 
     @Test
-    public void testGetNioBufferThreadSafe() {
+    void testGetNioBufferThreadSafe() {
         NetworkBuffer buffer = newBuffer(1024, 1024);
         testGetNioBufferThreadSafe(buffer, 10);
     }
@@ -437,21 +435,22 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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 +459,6 @@ public class NetworkBufferTest extends AbstractByteBufTest {
         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 3b92d177618..319069c610a 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.runtime.io.network.netty.NettyBufferPool;
 
 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 class ReadOnlySlicedBufferTest {
     }
 
     @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 class ReadOnlySlicedBufferTest {
      * 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 class ReadOnlySlicedBufferTest {
      * 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 class ReadOnlySlicedBufferTest {
      * 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,76 @@ public class ReadOnlySlicedBufferTest {
      * 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(slice1.getMemorySegmentOffset())
+                .isOne();
 
         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()).isOne();
+        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()).isOne();
+        assertThat(slice2.getMemorySegmentOffset()).isOne();
 
         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()).isOne();
+        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 +211,7 @@ public class ReadOnlySlicedBufferTest {
      * ReadOnlySlicedNetworkBuffer#getReaderIndex()}.
      */
     @Test
-    public void testGetSetReaderIndex1() {
+    void testGetSetReaderIndex1() {
         testGetSetReaderIndex(buffer.readOnlySlice());
     }
 
@@ -224,16 +221,16 @@ public class ReadOnlySlicedBufferTest {
      * 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()).isZero();
+        assertThat(slice.getReaderIndex()).isZero();
         slice.setReaderIndex(1);
-        assertEquals(0, buffer.getReaderIndex());
-        assertEquals(1, slice.getReaderIndex());
+        assertThat(buffer.getReaderIndex()).isZero();
+        assertThat(slice.getReaderIndex()).isOne();
     }
 
     /**
@@ -241,7 +238,7 @@ public class ReadOnlySlicedBufferTest {
      * ReadOnlySlicedNetworkBuffer#setSize(int)}, {@link ReadOnlySlicedNetworkBuffer#getSize()}.
      */
     @Test
-    public void testGetSetSize1() {
+    void testGetSetSize1() {
         testGetSetSize(buffer.readOnlySlice(), DATA_SIZE);
     }
 
@@ -250,96 +247,96 @@ public class ReadOnlySlicedBufferTest {
      * 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()).isZero();
+        assertThat(slice.getReaderIndex()).isZero();
+        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()).isOne();
+        assertThat(sliceByteBuffer.limit()).isOne();
+        assertThat(sliceByteBuffer.capacity()).isOne();
 
         // 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()).isZero();
+        assertThat(slice.getReaderIndex()).isZero();
+        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 +344,8 @@ public class ReadOnlySlicedBufferTest {
     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 +354,19 @@ public class ReadOnlySlicedBufferTest {
         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/partition/PipelinedApproximateSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedApproximateSubpartitionTest.java
index 16b3e6df770..494109a78f9 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
@@ -27,8 +27,8 @@ import org.junit.Test;
 import java.io.IOException;
 
 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.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.assertContent;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.toByteBuffer;
 import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
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 4422d40cb6b..103f6247ba9 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
@@ -44,7 +44,7 @@ import java.nio.ByteBuffer;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 
-import static org.apache.flink.runtime.io.network.buffer.LocalBufferPoolDestroyTest.isInBlockingBufferRequest;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.isInBlockingBufferRequest;
 import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.createPartition;
 import static org.apache.flink.runtime.io.network.partition.PartitionTestUtils.verifyCreateSubpartitionViewThrowsException;
 import static org.assertj.core.api.Assertions.assertThat;
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
index 12f92e6e9c1..eda4f667b03 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java
@@ -57,7 +57,7 @@ import java.util.Arrays;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.flink.runtime.io.network.buffer.LocalBufferPoolDestroyTest.isInBlockingBufferRequest;
+import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.isInBlockingBufferRequest;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;