From 618e0e7082d7d27d3d5e3c4fc45ee8f34b5c1ee9 Mon Sep 17 00:00:00 2001 From: Zhanghao Chen Date: Sun, 13 Aug 2023 16:12:37 +0800 Subject: [PATCH] [FLINK-32858][tests][JUnit5 migration] Migrate flink-runtime/utils tests to JUnit5 --- .../runtime/util/AddressResolutionTest.java | 41 +- .../runtime/util/BlockingShutdownTest.java | 33 +- .../runtime/util/BoundedFIFOQueueTest.java | 67 +- .../runtime/util/DualKeyLinkedMapTest.java | 50 +- .../util/EnvironmentInformationTest.java | 50 +- .../util/FlinkSecurityManagerITCase.java | 26 +- .../flink/runtime/util/HardwareTest.java | 15 +- .../runtime/util/JvmExitOnFatalErrorTest.java | 21 +- .../NonReusingKeyGroupedIteratorTest.java | 1087 ++++++++-------- .../runtime/util/ResourceCounterTest.java | 141 +-- .../util/ResourceManagerUtilsTest.java | 36 +- .../util/ReusingKeyGroupedIteratorTest.java | 1109 +++++++++-------- .../flink/runtime/util/RunnablesTest.java | 36 +- .../runtime/util/SerializedThrowableTest.java | 113 +- .../util/SlotSelectionStrategyUtilsTest.java | 20 +- .../util/StateHandleStoreUtilsTest.java | 47 +- .../runtime/util/ZooKeeperUtilsTest.java | 34 +- .../util/ZooKeeperUtilsTreeCacheTest.java | 49 +- ...KeeperUtilsWriteLeaderInformationTest.java | 3 - .../runtime/util/bash/BashJavaUtilsTest.java | 14 +- .../util/bash/FlinkConfigLoaderTest.java | 57 +- .../memory/JobManagerProcessSpecTest.java | 17 +- .../config/memory/ManagedMemoryUtilsTest.java | 63 +- .../memory/ProcessMemoryUtilsTestBase.java | 133 +- .../memory/TaskExecutorProcessSpecTest.java | 17 +- .../util/event/TaskEventHandlerTest.java | 13 +- 26 files changed, 1627 insertions(+), 1665 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java index 53680fa73eec69..d009b80aea896f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/AddressResolutionTest.java @@ -21,19 +21,18 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.rpc.AddressResolution; import org.apache.flink.runtime.rpc.RpcSystem; -import org.apache.flink.util.TestLogger; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.net.InetAddress; import java.net.UnknownHostException; -import static org.junit.Assert.fail; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; /** Unit tests for respecting {@link AddressResolution}. */ -public class AddressResolutionTest extends TestLogger { +class AddressResolutionTest { private static final RpcSystem RPC_SYSTEM = RpcSystem.load(); @@ -41,8 +40,8 @@ public class AddressResolutionTest extends TestLogger { private static final String NON_EXISTING_HOSTNAME = "foo.bar.com.invalid"; private static final int PORT = 17234; - @BeforeClass - public static void check() { + @BeforeAll + static void check() { checkPreconditions(); } @@ -60,11 +59,11 @@ private static void checkPreconditions() { throwsException = true; } - assumeTrue(throwsException); + assumeThat(throwsException).isTrue(); } @Test - public void testNoAddressResolution() throws UnknownHostException { + void testNoAddressResolution() throws UnknownHostException { RPC_SYSTEM.getRpcUrl( NON_EXISTING_HOSTNAME, PORT, @@ -74,17 +73,15 @@ public void testNoAddressResolution() throws UnknownHostException { } @Test - public void testTryAddressResolution() { - try { - RPC_SYSTEM.getRpcUrl( - NON_EXISTING_HOSTNAME, - PORT, - ENDPOINT_NAME, - AddressResolution.TRY_ADDRESS_RESOLUTION, - new Configuration()); - fail("This should fail with an UnknownHostException"); - } catch (UnknownHostException ignore) { - // expected - } + void testTryAddressResolution() { + assertThatThrownBy( + () -> + RPC_SYSTEM.getRpcUrl( + NON_EXISTING_HOSTNAME, + PORT, + ENDPOINT_NAME, + AddressResolution.TRY_ADDRESS_RESOLUTION, + new Configuration())) + .isInstanceOf(UnknownHostException.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java index b4083fef6e9a1d..d6406ca9c19d2b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingShutdownTest.java @@ -21,27 +21,26 @@ import org.apache.flink.runtime.testutils.TestJvmProcess; import org.apache.flink.util.OperatingSystem; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; import java.util.UUID; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; /** * Test that verifies the behavior of blocking shutdown hooks and of the {@link * JvmShutdownSafeguard} that guards against it. */ -public class BlockingShutdownTest { +class BlockingShutdownTest { @Test - public void testProcessShutdownBlocking() throws Exception { + void testProcessShutdownBlocking() throws Exception { // this test works only on linux - assumeTrue(OperatingSystem.isLinux()); + assumeThat(OperatingSystem.isLinux()).isTrue(); final File markerFile = new File( @@ -54,7 +53,7 @@ public void testProcessShutdownBlocking() throws Exception { try { blockingProcess.startProcess(); long pid = blockingProcess.getProcessId(); - assertTrue("Cannot determine process ID", pid != -1); + assertThat(pid).withFailMessage("Cannot determine process ID").isNotEqualTo(-1); // wait for the marker file to appear, which means the process is up properly TestJvmProcess.waitForMarkerFile(markerFile, 30000); @@ -69,9 +68,9 @@ public void testProcessShutdownBlocking() throws Exception { Thread.sleep(50); // the process should not go away by itself - assertTrue( - "Test broken, process shutdown blocking does not work", - blockingProcess.isAlive()); + assertThat(blockingProcess.isAlive()) + .withFailMessage("Test broken, process shutdown blocking does not work") + .isTrue(); } finally { blockingProcess.destroy(); @@ -81,9 +80,9 @@ public void testProcessShutdownBlocking() throws Exception { } @Test - public void testProcessExitsDespiteBlockingShutdownHook() throws Exception { + void testProcessExitsDespiteBlockingShutdownHook() throws Exception { // this test works only on linux - assumeTrue(OperatingSystem.isLinux()); + assumeThat(OperatingSystem.isLinux()).isTrue(); final File markerFile = new File( @@ -96,7 +95,7 @@ public void testProcessExitsDespiteBlockingShutdownHook() throws Exception { try { blockingProcess.startProcess(); long pid = blockingProcess.getProcessId(); - assertTrue("Cannot determine process ID", pid != -1); + assertThat(pid).withFailMessage("Cannot determine process ID").isNotEqualTo(-1); // wait for the marker file to appear, which means the process is up properly TestJvmProcess.waitForMarkerFile(markerFile, 30000); @@ -109,9 +108,9 @@ public void testProcessExitsDespiteBlockingShutdownHook() throws Exception { Thread.sleep(50); } - assertFalse( - "shutdown blocking process does not properly terminate itself", - blockingProcess.isAlive()); + assertThat(blockingProcess.isAlive()) + .withFailMessage("shutdown blocking process does not properly terminate itself") + .isFalse(); } finally { blockingProcess.destroy(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BoundedFIFOQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BoundedFIFOQueueTest.java index 818b7e1310bce6..16f234acc56cc4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BoundedFIFOQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BoundedFIFOQueueTest.java @@ -18,58 +18,51 @@ package org.apache.flink.runtime.util; -import org.apache.flink.util.TestLogger; +import org.junit.jupiter.api.Test; -import org.junit.Test; - -import static org.hamcrest.collection.IsIterableContainingInOrder.contains; -import static org.hamcrest.collection.IsIterableWithSize.iterableWithSize; -import static org.hamcrest.core.Is.is; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** {@code BoundedFIFOQueueTest} tests {@link BoundedFIFOQueue}. */ -public class BoundedFIFOQueueTest extends TestLogger { +class BoundedFIFOQueueTest { - @Test(expected = IllegalArgumentException.class) - public void testConstructorFailing() { - new BoundedFIFOQueue<>(-1); + @Test + void testConstructorFailing() { + assertThatThrownBy(() -> new BoundedFIFOQueue<>(-1)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testQueueWithMaxSize0() { + void testQueueWithMaxSize0() { final BoundedFIFOQueue testInstance = new BoundedFIFOQueue<>(0); - assertThat(testInstance, iterableWithSize(0)); + assertThat(testInstance).isEmpty(); testInstance.add(1); - assertThat(testInstance, iterableWithSize(0)); + assertThat(testInstance).isEmpty(); } @Test - public void testQueueWithMaxSize2() { + void testQueueWithMaxSize2() { final BoundedFIFOQueue testInstance = new BoundedFIFOQueue<>(2); - assertThat(testInstance, iterableWithSize(0)); + assertThat(testInstance).isEmpty(); testInstance.add(1); - assertThat(testInstance, contains(1)); + assertThat(testInstance).contains(1); testInstance.add(2); - assertThat(testInstance, contains(1, 2)); + assertThat(testInstance).contains(1, 2); testInstance.add(3); - assertThat(testInstance, contains(2, 3)); + assertThat(testInstance).contains(2, 3); } @Test - public void testAddNullHandling() { + void testAddNullHandling() { final BoundedFIFOQueue testInstance = new BoundedFIFOQueue<>(1); - try { - testInstance.add(null); - fail("A NullPointerException is expected to be thrown."); - } catch (NullPointerException e) { - // NullPointerException is expected - } - - assertThat(testInstance, iterableWithSize(0)); + assertThatThrownBy(() -> testInstance.add(null)) + .withFailMessage("A NullPointerException is expected to be thrown.") + .isInstanceOf(NullPointerException.class); + + assertThat(testInstance).isEmpty(); } /** @@ -77,12 +70,12 @@ public void testAddNullHandling() { * the queue with a {@code maxSize} of 0. */ @Test - public void testSizeWithMaxSize0() { + void testSizeWithMaxSize0() { final BoundedFIFOQueue testInstance = new BoundedFIFOQueue<>(0); - assertThat(testInstance.size(), is(0)); + assertThat(testInstance).isEmpty(); testInstance.add(1); - assertThat(testInstance.size(), is(0)); + assertThat(testInstance).isEmpty(); } /** @@ -90,18 +83,18 @@ public void testSizeWithMaxSize0() { * the queue with a {@code maxSize} of 2. */ @Test - public void testSizeWithMaxSize2() { + void testSizeWithMaxSize2() { final BoundedFIFOQueue testInstance = new BoundedFIFOQueue<>(2); - assertThat(testInstance.size(), is(0)); + assertThat(testInstance).isEmpty(); testInstance.add(5); - assertThat(testInstance.size(), is(1)); + assertThat(testInstance).hasSize(1); testInstance.add(6); - assertThat(testInstance.size(), is(2)); + assertThat(testInstance).hasSize(2); // adding a 3rd element won't increase the size anymore testInstance.add(7); - assertThat(testInstance.size(), is(2)); + assertThat(testInstance).hasSize(2); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DualKeyLinkedMapTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DualKeyLinkedMapTest.java index cd506942189ae2..6b7ef81c14216b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DualKeyLinkedMapTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DualKeyLinkedMapTest.java @@ -19,25 +19,21 @@ package org.apache.flink.runtime.util; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.util.TestLogger; -import org.hamcrest.Matchers; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.HashSet; import java.util.Random; import java.util.Set; import java.util.stream.Collectors; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.hamcrest.MatcherAssert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link DualKeyLinkedMap}. */ -public class DualKeyLinkedMapTest extends TestLogger { +class DualKeyLinkedMapTest { @Test - public void testKeySets() { + void testKeySets() { final Random random = new Random(); final Set> keys = new HashSet<>(); @@ -53,42 +49,40 @@ public void testKeySets() { dualKeyMap.put(key.f0, key.f1, "foobar"); } - assertThat( - dualKeyMap.keySetA(), - Matchers.equalTo(keys.stream().map(t -> t.f0).collect(Collectors.toSet()))); - assertThat( - dualKeyMap.keySetB(), - Matchers.equalTo(keys.stream().map(t -> t.f1).collect(Collectors.toSet()))); + assertThat(dualKeyMap.keySetA()) + .isEqualTo(keys.stream().map(t -> t.f0).collect(Collectors.toSet())); + assertThat(dualKeyMap.keySetB()) + .isEqualTo(keys.stream().map(t -> t.f1).collect(Collectors.toSet())); } @Test - public void ensuresOneToOneMappingBetweenKeysSamePrimaryKey() { + void ensuresOneToOneMappingBetweenKeysSamePrimaryKey() { final DualKeyLinkedMap map = new DualKeyLinkedMap<>(); final String secondValue = "barfoo"; map.put(1, 1, "foobar"); map.put(1, 2, secondValue); - assertThat(map.getValueByKeyB(1), nullValue()); - assertThat(map.getValueByKeyA(1), is(secondValue)); - assertThat(map.getValueByKeyB(2), is(secondValue)); + assertThat(map.getValueByKeyB(1)).isNull(); + assertThat(map.getValueByKeyA(1)).isEqualTo(secondValue); + assertThat(map.getValueByKeyB(2)).isEqualTo(secondValue); } @Test - public void ensuresOneToOneMappingBetweenKeysSameSecondaryKey() { + void ensuresOneToOneMappingBetweenKeysSameSecondaryKey() { final DualKeyLinkedMap map = new DualKeyLinkedMap<>(); final String secondValue = "barfoo"; map.put(1, 1, "foobar"); map.put(2, 1, secondValue); - assertThat(map.getValueByKeyA(1), nullValue()); - assertThat(map.getValueByKeyB(1), is(secondValue)); - assertThat(map.getValueByKeyA(2), is(secondValue)); + assertThat(map.getValueByKeyA(1)).isNull(); + assertThat(map.getValueByKeyB(1)).isEqualTo(secondValue); + assertThat(map.getValueByKeyA(2)).isEqualTo(secondValue); } @Test - public void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithSameSecondaryKey() { + void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithSameSecondaryKey() { final DualKeyLinkedMap map = new DualKeyLinkedMap<>(); final String value1 = "1"; @@ -98,12 +92,12 @@ public void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithSameSecondaryKey() { final String value3 = "3"; map.put(1, 1, value3); - assertThat(map.keySetA().iterator().next(), is(1)); - assertThat(map.values().iterator().next(), is(value3)); + assertThat(map.keySetA().iterator().next()).isOne(); + assertThat(map.values().iterator().next()).isEqualTo(value3); } @Test - public void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithDifferentSecondaryKey() { + void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithDifferentSecondaryKey() { final DualKeyLinkedMap map = new DualKeyLinkedMap<>(); final String value1 = "1"; @@ -113,7 +107,7 @@ public void testPrimaryKeyOrderIsNotAffectedIfReInsertedWithDifferentSecondaryKe final String value3 = "3"; map.put(1, 3, value3); - assertThat(map.keySetA().iterator().next(), is(1)); - assertThat(map.values().iterator().next(), is(value3)); + assertThat(map.keySetA().iterator().next()).isOne(); + assertThat(map.values().iterator().next()).isEqualTo(value3); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java index 034870cca7f90a..74208d86434e92 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java @@ -18,17 +18,19 @@ package org.apache.flink.runtime.util; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Arrays; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +class EnvironmentInformationTest { -public class EnvironmentInformationTest extends TestLogger { + private final Logger log = LoggerFactory.getLogger(getClass()); @Test public void testJavaMemory() { @@ -36,15 +38,15 @@ public void testJavaMemory() { long fullHeap = EnvironmentInformation.getMaxJvmHeapMemory(); long freeWithGC = EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag(); - assertTrue(fullHeap > 0); - assertTrue(freeWithGC >= 0); + assertThat(fullHeap).isGreaterThan(0); + assertThat(freeWithGC).isGreaterThanOrEqualTo(0); try { long free = EnvironmentInformation.getSizeOfFreeHeapMemory(); - assertTrue(free >= 0); + assertThat(free).isGreaterThanOrEqualTo(0); } catch (RuntimeException e) { // this may only occur if the Xmx is not set - assertEquals(Long.MAX_VALUE, EnvironmentInformation.getMaxJvmHeapMemory()); + assertThat(EnvironmentInformation.getMaxJvmHeapMemory()).isEqualTo(Long.MAX_VALUE); } // we cannot make these assumptions, because the test JVM may grow / shrink during the @@ -60,21 +62,21 @@ public void testJavaMemory() { @Test public void testEnvironmentMethods() { try { - assertNotNull(EnvironmentInformation.getJvmStartupOptions()); - assertNotNull(EnvironmentInformation.getJvmStartupOptionsArray()); - assertNotNull(EnvironmentInformation.getJvmVersion()); - assertNotNull(EnvironmentInformation.getRevisionInformation()); - assertNotNull(EnvironmentInformation.getVersion()); - assertNotNull(EnvironmentInformation.getScalaVersion()); - assertNotNull(EnvironmentInformation.getBuildTime()); - assertNotNull(EnvironmentInformation.getBuildTimeString()); - assertNotNull(EnvironmentInformation.getGitCommitId()); - assertNotNull(EnvironmentInformation.getGitCommitIdAbbrev()); - assertNotNull(EnvironmentInformation.getGitCommitTime()); - assertNotNull(EnvironmentInformation.getGitCommitTimeString()); - EnvironmentInformation.getHadoopVersionString(); - assertNotNull(EnvironmentInformation.getHadoopUser()); - assertTrue(EnvironmentInformation.getOpenFileHandlesLimit() >= -1); + assertThat(EnvironmentInformation.getJvmStartupOptions()).isNotNull(); + assertThat(EnvironmentInformation.getJvmStartupOptionsArray()).isNotNull(); + assertThat(EnvironmentInformation.getJvmVersion()).isNotNull(); + assertThat(EnvironmentInformation.getRevisionInformation()).isNotNull(); + assertThat(EnvironmentInformation.getVersion()).isNotNull(); + assertThat(EnvironmentInformation.getScalaVersion()).isNotNull(); + assertThat(EnvironmentInformation.getBuildTime()).isNotNull(); + assertThat(EnvironmentInformation.getBuildTimeString()).isNotNull(); + assertThat(EnvironmentInformation.getGitCommitId()).isNotNull(); + assertThat(EnvironmentInformation.getGitCommitIdAbbrev()).isNotNull(); + assertThat(EnvironmentInformation.getGitCommitTime()).isNotNull(); + assertThat(EnvironmentInformation.getGitCommitTimeString()).isNotNull(); + assertThat(EnvironmentInformation.getHadoopVersionString()).isNotNull(); + assertThat(EnvironmentInformation.getHadoopUser()).isNotNull(); + assertThat(EnvironmentInformation.getOpenFileHandlesLimit()).isGreaterThanOrEqualTo(-1); if (log.isInfoEnabled()) { // Visual inspection of the available Environment variables diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/FlinkSecurityManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/FlinkSecurityManagerITCase.java index 402628f9a253a6..35c5d114ea8230 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/FlinkSecurityManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/FlinkSecurityManagerITCase.java @@ -23,48 +23,46 @@ import org.apache.flink.core.security.FlinkSecurityManager; import org.apache.flink.runtime.testutils.TestJvmProcess; import org.apache.flink.util.OperatingSystem; -import org.apache.flink.util.TestLogger; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.junit.Assume.assumeTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; /** Integration tests for the {@link FlinkSecurityManager}. */ -public class FlinkSecurityManagerITCase extends TestLogger { +class FlinkSecurityManagerITCase { - @Before - public void ensureSupportedOS() { + @BeforeEach + void ensureSupportedOS() { // based on the assumption in JvmExitOnFatalErrorTest, and manual testing on Mac, we do not // support all platforms (in particular not Windows) - assumeTrue(OperatingSystem.isLinux() || OperatingSystem.isMac()); + assumeThat(OperatingSystem.isLinux() || OperatingSystem.isMac()).isTrue(); } @Test - public void testForcedJVMExit() throws Exception { + void testForcedJVMExit() throws Exception { final ForcedJVMExitProcess testProcess = new ForcedJVMExitProcess(ForcedExitEntryPoint.class); testProcess.startProcess(); try { testProcess.waitFor(); - assertThat(testProcess.exitCode(), is(222)); + assertThat(testProcess.exitCode()).isEqualTo(222); } finally { testProcess.destroy(); } } @Test - public void testIgnoredJVMExit() throws Exception { + void testIgnoredJVMExit() throws Exception { final ForcedJVMExitProcess testProcess = new ForcedJVMExitProcess(IgnoredExitEntryPoint.class); testProcess.startProcess(); try { testProcess.waitFor(); - assertThat(testProcess.exitCode(), is(0)); + assertThat(testProcess.exitCode()).isZero(); } finally { testProcess.destroy(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/HardwareTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/HardwareTest.java index 074ced65cafeaf..206b4538739dbd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/HardwareTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/HardwareTest.java @@ -18,16 +18,17 @@ package org.apache.flink.runtime.util; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.*; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; -public class HardwareTest { +class HardwareTest { @Test - public void testCpuCores() { + void testCpuCores() { try { - assertTrue(Hardware.getNumberCPUCores() >= 0); + assertThat(Hardware.getNumberCPUCores()).isGreaterThanOrEqualTo(0); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -35,10 +36,10 @@ public void testCpuCores() { } @Test - public void testPhysicalMemory() { + void testPhysicalMemory() { try { long physMem = Hardware.getSizeOfPhysicalMemory(); - assertTrue(physMem >= -1); + assertThat(physMem).isGreaterThanOrEqualTo(-1); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index 79439059074512..b5966bed100e0c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -69,13 +69,12 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.testutils.TestJvmProcess; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.SerializedValue; -import org.apache.flink.util.TestLogger; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.net.InetAddress; @@ -84,27 +83,27 @@ import java.util.concurrent.Executors; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionAttemptId; -import static org.junit.Assume.assumeFalse; +import static org.assertj.core.api.Assumptions.assumeThat; /** * Test that verifies the behavior of blocking shutdown hooks and of the {@link * JvmShutdownSafeguard} that guards against it. */ -public class JvmExitOnFatalErrorTest extends TestLogger { +class JvmExitOnFatalErrorTest { - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir private java.nio.file.Path tempDir; @Test - public void testExitJvmOnOutOfMemory() throws Exception { + void testExitJvmOnOutOfMemory() throws Exception { // this test works only on linux and MacOS - assumeFalse(OperatingSystem.isWindows()); + assumeThat(OperatingSystem.isWindows()).isFalse(); // to check what went wrong (when the test hangs) uncomment this line // ProcessEntryPoint.main(new - // String[]{temporaryFolder.newFolder().getAbsolutePath()}); + // String[]{TempDirUtils.newFolder(tempDir).getAbsolutePath()}); final KillOnFatalErrorProcess testProcess = - new KillOnFatalErrorProcess(temporaryFolder.newFolder()); + new KillOnFatalErrorProcess(TempDirUtils.newFolder(tempDir)); try { testProcess.startProcess(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java index 3d490cafc59a6f..dc899f6bcbc429 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/NonReusingKeyGroupedIteratorTest.java @@ -24,28 +24,31 @@ import org.apache.flink.types.StringValue; import org.apache.flink.util.MutableObjectIterator; -import org.junit.Assert; -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.util.ArrayList; import java.util.Iterator; import java.util.NoSuchElementException; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; + /** * Test for the safe key grouped iterator, which advances in windows containing the same key and * provides a sub-iterator over the records with the same key. */ -public class NonReusingKeyGroupedIteratorTest { +class NonReusingKeyGroupedIteratorTest { private MutableObjectIterator sourceIter; // the iterator that provides the input private NonReusingKeyGroupedIterator psi; // the grouping iterator, progressing in key steps - @Before - public void setup() { + @BeforeEach + void setup() { final ArrayList source = new ArrayList(); // add elements to the source @@ -100,587 +103,607 @@ public Record next() throws IOException { } @Test - public void testNextKeyOnly() throws Exception { + void testNextKeyOnly() throws Exception { try { - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(1)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(2)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertNull( - "KeyGroupedIterator must not have another value.", this.psi.getValues()); - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(1)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(2)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat((Iterable) this.psi.getValues()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isNull(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testFullIterationThroughAllValues() throws IOException { + void testFullIterationThroughAllValues() throws IOException { try { // Key 1, Value A - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(1)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(1)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(1)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); // Key 2, Value B - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(2)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(2)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(2)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); // Key 3, Values C, D - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); // Key 4, Values E, F, G - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("E"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("F"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("G"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("E"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("F"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("G"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); // Key 5, Values H, I, J, K, L - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("H"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("I"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("J"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("K"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("L"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("H"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("I"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("J"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("K"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("L"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testMixedProgress() throws Exception { + void testMixedProgress() throws Exception { try { // Progression only via nextKey() and hasNext() - Key 1, Value A - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); // Progression only through nextKey() - Key 2, Value B - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); // Progression first though haNext() and next(), then through hasNext() - Key 3, Values // C, D - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); // Progression first via next() only, then hasNext() only Key 4, Values E, F, G - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("E"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("E"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); // Key 5, Values H, I, J, K, L - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("H"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("I"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("H"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("I"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); // end - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testHasNextDoesNotOverweiteCurrentRecord() throws Exception { + void testHasNextDoesNotOverwriteCurrentRecord() throws Exception { try { Iterator valsIter = null; Record rec = null; - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - rec.getField(1, StringValue.class)); - Assert.assertFalse("KeyGroupedIterator must have another value.", valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must not have another value.", - valsIter.hasNext()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - rec.getField(1, StringValue.class)); - Assert.assertFalse("KeyGroupedIterator must have another value.", valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must not have another value.", - valsIter.hasNext()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - rec.getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - rec.getField(1, StringValue.class)); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceCounterTest.java index 085405c3b47ec8..8c368644b61b75 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceCounterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceCounterTest.java @@ -19,25 +19,21 @@ package org.apache.flink.runtime.util; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; -import org.hamcrest.Matchers; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Map; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link ResourceCounter}. */ -public class ResourceCounterTest extends TestLogger { +class ResourceCounterTest { private ResourceProfile resourceProfile1 = ResourceProfile.newBuilder().setManagedMemoryMB(42).build(); @@ -45,83 +41,82 @@ public class ResourceCounterTest extends TestLogger { ResourceProfile.newBuilder().setCpuCores(1.7).build(); @Test - public void testIsEmpty() { + void testIsEmpty() { final ResourceCounter empty = ResourceCounter.empty(); - assertTrue(empty.isEmpty()); + assertThat(empty.isEmpty()).isTrue(); } - @Test(expected = IllegalArgumentException.class) + @Test public void testWithResourceRejectsNegativeCount() { - ResourceCounter.withResource(ResourceProfile.UNKNOWN, -1); + assertThatThrownBy(() -> ResourceCounter.withResource(ResourceProfile.UNKNOWN, -1)) + .isInstanceOf(IllegalArgumentException.class); } @Test - public void testWithResourceCreatesEmptyCounterIfCountIsZero() { + void testWithResourceCreatesEmptyCounterIfCountIsZero() { final ResourceCounter empty = ResourceCounter.withResource(ResourceProfile.UNKNOWN, 0); - assertTrue(empty.isEmpty()); + assertThat(empty.isEmpty()).isTrue(); } @Test - public void testIsNonEmpty() { + void testIsNonEmpty() { final ResourceCounter resourceCounter = ResourceCounter.withResource(ResourceProfile.UNKNOWN, 1); - assertFalse(resourceCounter.isEmpty()); - assertTrue(resourceCounter.containsResource(ResourceProfile.UNKNOWN)); + assertThat(resourceCounter.isEmpty()).isFalse(); + assertThat(resourceCounter.containsResource(ResourceProfile.UNKNOWN)).isTrue(); } @Test - public void testGetResourceCount() { + void testGetResourceCount() { final Map resources = createResources(); final ResourceCounter resourceCounter = ResourceCounter.withResources(resources); for (Map.Entry resource : resources.entrySet()) { - assertThat( - resourceCounter.getResourceCount(resource.getKey()), is(resource.getValue())); + assertThat(resourceCounter.getResourceCount(resource.getKey())) + .isEqualTo(resource.getValue()); } } @Test - public void testGetResourceCountReturnsZeroForUnknownResourceProfile() { + void testGetResourceCountReturnsZeroForUnknownResourceProfile() { final ResourceCounter resourceCounter = ResourceCounter.withResources(createResources()); - assertThat(resourceCounter.getResourceCount(ResourceProfile.newBuilder().build()), is(0)); + assertThat(resourceCounter.getResourceCount(ResourceProfile.newBuilder().build())).isZero(); } @Test - public void testGetTotalResourceCount() { + void testGetTotalResourceCount() { final Map resources = createResources(); final ResourceCounter resourceCounter = ResourceCounter.withResources(resources); - assertThat(resourceCounter.getTotalResourceCount(), is(5)); + assertThat(resourceCounter.getTotalResourceCount()).isEqualTo(5); } @Test - public void testGetResources() { + void testGetResources() { final Map resources = createResources(); final ResourceCounter resourceCounter = ResourceCounter.withResources(resources); - assertThat( - resourceCounter.getResources(), - Matchers.containsInAnyOrder(resources.keySet().toArray())); + assertThat(resourceCounter.getResources()) + .containsExactlyInAnyOrderElementsOf(resources.keySet()); } @Test - public void testGetResourceWithCount() { + void testGetResourceWithCount() { final Map resources = createResources(); final ResourceCounter resourceCounter = ResourceCounter.withResources(resources); - assertThat( - resourceCounter.getResourcesWithCount(), - Matchers.containsInAnyOrder(resources.entrySet().toArray())); + assertThat(resourceCounter.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf(resources.entrySet()); } @Test - public void testAddSameResourceProfile() { + void testAddSameResourceProfile() { final int value1 = 1; final int value2 = 42; @@ -132,29 +127,21 @@ public void testAddSameResourceProfile() { final ResourceCounter result = resourceCounter1.add(resourceCounter2); - assertThat( - resourceCounter1.getResourcesWithCount(), - Matchers.containsInAnyOrder( - Collections.singletonMap(ResourceProfile.UNKNOWN, value1) - .entrySet() - .toArray())); - assertThat( - resourceCounter2.getResourcesWithCount(), - Matchers.containsInAnyOrder( - Collections.singletonMap(ResourceProfile.UNKNOWN, value2) - .entrySet() - .toArray())); - - assertThat( - result.getResourcesWithCount(), - Matchers.containsInAnyOrder( + assertThat(resourceCounter1.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( + Collections.singletonMap(ResourceProfile.UNKNOWN, value1).entrySet()); + assertThat(resourceCounter2.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( + Collections.singletonMap(ResourceProfile.UNKNOWN, value2).entrySet()); + + assertThat(result.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( Collections.singletonMap(ResourceProfile.UNKNOWN, value1 + value2) - .entrySet() - .toArray())); + .entrySet()); } @Test - public void testAddDifferentResourceProfile() { + void testAddDifferentResourceProfile() { final ResourceCounter resourceCounter1 = ResourceCounter.withResource(resourceProfile1, 1); final ResourceCounter resourceCounter2 = ResourceCounter.withResource(resourceProfile2, 1); @@ -164,31 +151,30 @@ public void testAddDifferentResourceProfile() { new ArrayList<>(resourceCounter1.getResourcesWithCount()); expectedResult.addAll(resourceCounter2.getResourcesWithCount()); - assertThat( - result.getResourcesWithCount(), - Matchers.containsInAnyOrder(expectedResult.toArray())); + assertThat(result.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf(expectedResult); } @Test - public void testCountEqualToZeroRemovesResource() { + void testCountEqualToZeroRemovesResource() { final ResourceCounter resourceCounter = ResourceCounter.withResource(resourceProfile1, 2); final ResourceCounter result = resourceCounter.subtract(resourceProfile1, 2); - assertTrue(result.isEmpty()); + assertThat(result.isEmpty()).isTrue(); } @Test - public void testCountBelowZeroRemovesResources() { + void testCountBelowZeroRemovesResources() { final ResourceCounter resourceCounter = ResourceCounter.withResource(resourceProfile1, 1); final ResourceCounter result = resourceCounter.subtract(resourceProfile1, 2); - assertTrue(result.isEmpty()); + assertThat(result.isEmpty()).isTrue(); } @Test - public void testSubtractSameResourceProfile() { + void testSubtractSameResourceProfile() { final int value1 = 5; final int value2 = 3; @@ -199,37 +185,28 @@ public void testSubtractSameResourceProfile() { final ResourceCounter result = resourceCounter1.subtract(resourceCounter2); - assertThat( - resourceCounter1.getResourcesWithCount(), - Matchers.containsInAnyOrder( - Collections.singletonMap(ResourceProfile.UNKNOWN, value1) - .entrySet() - .toArray())); - assertThat( - resourceCounter2.getResourcesWithCount(), - Matchers.containsInAnyOrder( - Collections.singletonMap(ResourceProfile.UNKNOWN, value2) - .entrySet() - .toArray())); - - assertThat( - result.getResourcesWithCount(), - Matchers.containsInAnyOrder( + assertThat(resourceCounter1.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( + Collections.singletonMap(ResourceProfile.UNKNOWN, value1).entrySet()); + assertThat(resourceCounter2.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( + Collections.singletonMap(ResourceProfile.UNKNOWN, value2).entrySet()); + + assertThat(result.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf( Collections.singletonMap(ResourceProfile.UNKNOWN, value1 - value2) - .entrySet() - .toArray())); + .entrySet()); } @Test - public void testSubtractDifferentResourceProfile() { + void testSubtractDifferentResourceProfile() { final ResourceCounter resourceCounter1 = ResourceCounter.withResource(resourceProfile1, 1); final ResourceCounter resourceCounter2 = ResourceCounter.withResource(resourceProfile2, 1); final ResourceCounter result = resourceCounter1.subtract(resourceCounter2); - assertThat( - result.getResourcesWithCount(), - Matchers.containsInAnyOrder(resourceCounter1.getResourcesWithCount().toArray())); + assertThat(result.getResourcesWithCount()) + .containsExactlyInAnyOrderElementsOf(resourceCounter1.getResourcesWithCount()); } private Map createResources() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceManagerUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceManagerUtilsTest.java index b9ce6069a66aa5..0fbf79a1a3f31f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceManagerUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ResourceManagerUtilsTest.java @@ -18,41 +18,37 @@ package org.apache.flink.runtime.util; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; -public class ResourceManagerUtilsTest { +class ResourceManagerUtilsTest { @Test - public void testParseRestBindPortFromWebInterfaceUrlWithEmptyUrl() { - assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl(""), is(-1)); + void testParseRestBindPortFromWebInterfaceUrlWithEmptyUrl() { + assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("")).isEqualTo(-1); } @Test - public void testParseRestBindPortFromWebInterfaceUrlWithNullUrl() { - assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl(null), is(-1)); + void testParseRestBindPortFromWebInterfaceUrlWithNullUrl() { + assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl(null)).isEqualTo(-1); } @Test - public void testParseRestBindPortFromWebInterfaceUrlWithInvalidSchema() { - assertThat( - ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:8080//"), - is(-1)); + void testParseRestBindPortFromWebInterfaceUrlWithInvalidSchema() { + assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:8080//")) + .isEqualTo(-1); } @Test - public void testParseRestBindPortFromWebInterfaceUrlWithInvalidPort() { - assertThat( - ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:port1"), - is(-1)); + void testParseRestBindPortFromWebInterfaceUrlWithInvalidPort() { + assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:port1")) + .isEqualTo(-1); } @Test - public void testParseRestBindPortFromWebInterfaceUrlWithValidPort() { - assertThat( - ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:8080"), - is(8080)); + void testParseRestBindPortFromWebInterfaceUrlWithValidPort() { + assertThat(ResourceManagerUtils.parseRestBindPortFromWebInterfaceUrl("localhost:8080")) + .isEqualTo(8080); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java index a88eed37602488..5ccbff284adb09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ReusingKeyGroupedIteratorTest.java @@ -26,28 +26,31 @@ import org.apache.flink.util.MutableObjectIterator; import org.apache.flink.util.TraversableOnceException; -import org.junit.Assert; -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.util.ArrayList; import java.util.Iterator; import java.util.NoSuchElementException; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; + /** * Test for the key grouped iterator, which advances in windows containing the same key and provides * a sub-iterator over the records with the same key. */ -public class ReusingKeyGroupedIteratorTest { +class ReusingKeyGroupedIteratorTest { private MutableObjectIterator sourceIter; // the iterator that provides the input private ReusingKeyGroupedIterator psi; // the grouping iterator, progressing in key steps - @Before - public void setup() { + @BeforeEach + void setup() { final ArrayList source = new ArrayList(); // add elements to the source @@ -103,600 +106,616 @@ public Record next() throws IOException { } @Test - public void testNextKeyOnly() throws Exception { + void testNextKeyOnly() throws Exception { try { - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(1)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(2)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertNull( - "KeyGroupedIterator must not have another value.", this.psi.getValues()); - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(1)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(2)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat((Iterable) this.psi.getValues()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isNull(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testFullIterationThroughAllValues() throws IOException { + void testFullIterationThroughAllValues() throws IOException { try { // Key 1, Value A - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(1)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(1)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); // Key 2, Value B - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(2)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(2)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); // Key 3, Values C, D - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(3)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); // Key 4, Values E, F, G - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("E"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("F"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("G"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(4)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 4, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("E"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("F"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("G"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(4)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(4); // Key 5, Values H, I, J, K, L - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("H"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("I"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("J"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("K"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("L"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - Assert.assertFalse( - "KeyGroupedIterator must not have another value.", - this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - try { - this.psi.getValues().next(); - Assert.fail( - "A new KeyGroupedIterator must not have any value available and hence throw an exception on next()."); - } catch (NoSuchElementException nseex) { - } - - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertNull(this.psi.getValues()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("H"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("I"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("J"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("K"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("L"); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isFalse(); + assertThat( + this.psi + .getComparatorWithCurrentReference() + .equalToReference(new Record(new IntValue(5)))) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThatThrownBy(() -> this.psi.getValues().next()) + .withFailMessage( + "A new KeyGroupedIterator must not have any value available and hence throw an exception on next().") + .isInstanceOf(NoSuchElementException.class); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat((Iterable) this.psi.getValues()).isNull(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testMixedProgress() throws Exception { + void testMixedProgress() throws Exception { try { // Progression only via nextKey() and hasNext() - Key 1, Value A - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); // Progression only through nextKey() - Key 2, Value B - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); // Progression first though haNext() and next(), then through hasNext() - Key 3, Values // C, D - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(3)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); // Progression first via next() only, then hasNext() only Key 4, Values E, F, G - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("E"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("E"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); // Key 5, Values H, I, J, K, L - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("H"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); - Assert.assertTrue( - "KeyGroupedIterator returned a wrong key.", - this.psi - .getComparatorWithCurrentReference() - .equalToReference(new Record(new IntValue(5)))); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 5, - this.psi.getCurrent().getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("I"), - this.psi.getValues().next().getField(1, StringValue.class)); - Assert.assertTrue(hasIterator(this.psi.getValues())); - Assert.assertFalse(hasIterator(this.psi.getValues())); - Assert.assertTrue( - "KeyGroupedIterator must have another value.", this.psi.getValues().hasNext()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("H"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getCurrent().getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(5); + assertThat(this.psi.getValues().next().getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("I"); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); + assertThat(hasIterator(this.psi.getValues())).isTrue(); + assertThat(hasIterator(this.psi.getValues())).isFalse(); + assertThat(this.psi.getValues().hasNext()) + .withFailMessage("KeyGroupedIterator must have another value.") + .isTrue(); // end - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); - Assert.assertFalse("KeyGroupedIterator must not have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must not have another key.") + .isFalse(); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } @Test - public void testHasNextDoesNotOverweiteCurrentRecord() throws Exception { + void testHasNextDoesNotOverwriteCurrentRecord() throws Exception { try { Iterator valsIter = null; Record rec = null; - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - rec.getField(1, StringValue.class)); - Assert.assertFalse("KeyGroupedIterator must have another value.", valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 1, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("A"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must not have another value.", - valsIter.hasNext()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isOne(); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("A"); + assertThat(valsIter) + .withFailMessage( + "KeyGroupedIterator's value iterator must not have another value.") + .isExhausted(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - rec.getField(1, StringValue.class)); - Assert.assertFalse("KeyGroupedIterator must have another value.", valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 2, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("B"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must not have another value.", - valsIter.hasNext()); - - Assert.assertTrue("KeyGroupedIterator must have another key.", this.psi.nextKey()); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(2); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("B"); + assertThat(valsIter) + .withFailMessage( + "KeyGroupedIterator's value iterator must not have another value.") + .isExhausted(); + + assertThat(this.psi.nextKey()) + .withFailMessage("KeyGroupedIterator must have another key.") + .isTrue(); valsIter = this.psi.getValues(); - Assert.assertNotNull("Returned Iterator must not be null", valsIter); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); + assertThat(valsIter).withFailMessage("Returned Iterator must not be null").isNotNull(); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - rec.getField(1, StringValue.class)); - Assert.assertTrue( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("C"), - rec.getField(1, StringValue.class)); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator's value iterator must have another value.") + .hasNext(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("C"); rec = valsIter.next(); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); - Assert.assertFalse( - "KeyGroupedIterator's value iterator must have another value.", - valsIter.hasNext()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong key.", - 3, - rec.getField(0, IntValue.class).getValue()); - Assert.assertEquals( - "KeyGroupedIterator returned a wrong value.", - new StringValue("D"), - rec.getField(1, StringValue.class)); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat(valsIter) + .withFailMessage("KeyGroupedIterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); + assertThat(valsIter) + .withFailMessage( + "KeyGroupedIterator's value iterator must not have another value.") + .isExhausted(); + assertThat(rec.getField(0, IntValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong key.") + .isEqualTo(3); + assertThat(rec.getField(1, StringValue.class).getValue()) + .withFailMessage("KeyGroupedIterator returned a wrong value.") + .isEqualTo("D"); } catch (Exception e) { e.printStackTrace(); - Assert.fail("The test encountered an unexpected exception."); + fail("The test encountered an unexpected exception."); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/RunnablesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/RunnablesTest.java index 2af89d4b995e41..07631d10479b3d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/RunnablesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/RunnablesTest.java @@ -18,12 +18,9 @@ package org.apache.flink.runtime.util; -import org.apache.flink.util.TestLogger; - import org.apache.flink.shaded.guava31.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -33,7 +30,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -public class RunnablesTest extends TestLogger { +import static org.assertj.core.api.Assertions.assertThat; + +class RunnablesTest { private static final int TIMEOUT_MS = 100; @@ -42,7 +41,7 @@ public class RunnablesTest extends TestLogger { // ------------------------------------------------------------------------ @Test - public void testExecutorService_uncaughtExceptionHandler() throws InterruptedException { + void testExecutorService_uncaughtExceptionHandler() throws InterruptedException { final CountDownLatch handlerCalled = new CountDownLatch(1); final ThreadFactory threadFactory = new ThreadFactoryBuilder() @@ -64,8 +63,7 @@ public void testExecutorService_uncaughtExceptionHandler() throws InterruptedExc } @Test - public void testScheduledExecutorService_uncaughtExceptionHandler() - throws InterruptedException { + void testScheduledExecutorService_uncaughtExceptionHandler() throws InterruptedException { final CountDownLatch handlerCalled = new CountDownLatch(1); final ThreadFactory threadFactory = new ThreadFactoryBuilder() @@ -78,9 +76,9 @@ public void testScheduledExecutorService_uncaughtExceptionHandler() () -> { throw new RuntimeException("foo"); }); - Assert.assertFalse( - "Expected handler not to be called.", - handlerCalled.await(TIMEOUT_MS, TimeUnit.MILLISECONDS)); + assertThat(handlerCalled.await(TIMEOUT_MS, TimeUnit.MILLISECONDS)) + .withFailMessage("Expected handler not to be called.") + .isFalse(); } // ------------------------------------------------------------------------ @@ -88,7 +86,7 @@ public void testScheduledExecutorService_uncaughtExceptionHandler() // ------------------------------------------------------------------------ @Test - public void testWithUncaughtExceptionHandler_runtimeException() throws InterruptedException { + void testWithUncaughtExceptionHandler_runtimeException() throws InterruptedException { final RuntimeException expected = new RuntimeException("foo"); testWithUncaughtExceptionHandler( () -> { @@ -98,7 +96,7 @@ public void testWithUncaughtExceptionHandler_runtimeException() throws Interrupt } @Test - public void testWithUncaughtExceptionHandler_error() throws InterruptedException { + void testWithUncaughtExceptionHandler_error() throws InterruptedException { final Error expected = new Error("foo"); testWithUncaughtExceptionHandler( () -> { @@ -125,11 +123,11 @@ private static void testWithUncaughtExceptionHandler(Runnable runnable, Throwabl handlerCalled.countDown(); }); scheduledExecutorService.execute(guardedRunnable); - Assert.assertTrue(handlerCalled.await(100, TimeUnit.MILLISECONDS)); - Assert.assertNotNull(thread.get()); - Assert.assertNotNull(throwable.get()); - Assert.assertEquals("ueh-test-0", thread.get().getName()); - Assert.assertEquals(expected.getClass(), throwable.get().getClass()); - Assert.assertEquals("foo", throwable.get().getMessage()); + assertThat(handlerCalled.await(100, TimeUnit.MILLISECONDS)).isTrue(); + assertThat(thread).isNotNull(); + assertThat(throwable).isNotNull(); + assertThat(thread.get().getName()).isEqualTo("ueh-test-0"); + assertThat(throwable.get().getClass()).isEqualTo(expected.getClass()); + assertThat(throwable.get().getMessage()).isEqualTo("foo"); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java index a98b9a89c62516..cb05671e953b40 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java @@ -24,28 +24,25 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedThrowable; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; /** Tests for {@link SerializedThrowable}. */ -public class SerializedThrowableTest { +class SerializedThrowableTest { @Test - public void testIdenticalMessageAndStack() { + void testIdenticalMessageAndStack() { try { IllegalArgumentException original = new IllegalArgumentException("test message"); SerializedThrowable serialized = new SerializedThrowable(original); - assertEquals( - ExceptionUtils.stringifyException(original), - ExceptionUtils.stringifyException(serialized)); + assertThat(ExceptionUtils.stringifyException(original)) + .isEqualTo(ExceptionUtils.stringifyException(serialized)); - assertArrayEquals(original.getStackTrace(), serialized.getStackTrace()); + assertThat(original.getStackTrace()).isEqualTo(serialized.getStackTrace()); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -53,7 +50,7 @@ public void testIdenticalMessageAndStack() { } @Test - public void testSerialization() { + void testSerialization() { try { // We need an exception whose class is not in the core class loader final ClassLoaderUtils.ObjectAndClassLoader outsideClassLoading = @@ -63,20 +60,21 @@ public void testSerialization() { Class clazz = userException.getClass(); // check that we cannot simply copy the exception - try { - byte[] serialized = InstantiationUtil.serializeObject(userException); - InstantiationUtil.deserializeObject(serialized, getClass().getClassLoader()); - fail("should fail with a class not found exception"); - } catch (ClassNotFoundException e) { - // as we want it - } + assertThatThrownBy( + () -> { + byte[] serialized = + InstantiationUtil.serializeObject(userException); + InstantiationUtil.deserializeObject( + serialized, getClass().getClassLoader()); + }) + .withFailMessage("should fail with a class not found exception") + .isInstanceOf(ClassNotFoundException.class); // validate that the SerializedThrowable mimics the original exception SerializedThrowable serialized = new SerializedThrowable(userException); - assertEquals( - ExceptionUtils.stringifyException(userException), - ExceptionUtils.stringifyException(serialized)); - assertArrayEquals(userException.getStackTrace(), serialized.getStackTrace()); + assertThat(ExceptionUtils.stringifyException(userException)) + .isEqualTo(ExceptionUtils.stringifyException(serialized)); + assertThat(userException.getStackTrace()).isEqualTo(serialized.getStackTrace()); // validate the detailMessage of SerializedThrowable contains the class name of original // exception @@ -86,24 +84,22 @@ public void testSerialization() { String.format( "%s: %s", userException2.getClass().getName(), userException2.getMessage()); - assertEquals(serialized2.getMessage(), result); + assertThat(serialized2).hasMessage(result); // copy the serialized throwable and make sure everything still works SerializedThrowable copy = CommonTestUtils.createCopySerializable(serialized); - assertEquals( - ExceptionUtils.stringifyException(userException), - ExceptionUtils.stringifyException(copy)); - assertArrayEquals(userException.getStackTrace(), copy.getStackTrace()); + assertThat(ExceptionUtils.stringifyException(userException)) + .isEqualTo(ExceptionUtils.stringifyException(copy)); + assertThat(userException.getStackTrace()).isEqualTo(copy.getStackTrace()); // deserialize the proper exception Throwable deserialized = copy.deserializeError(loader); - assertEquals(clazz, deserialized.getClass()); + assertThat(deserialized.getClass()).isEqualTo(clazz); // deserialization with the wrong classloader does not lead to a failure Throwable wronglyDeserialized = copy.deserializeError(getClass().getClassLoader()); - assertEquals( - ExceptionUtils.stringifyException(userException), - ExceptionUtils.stringifyException(wronglyDeserialized)); + assertThat(ExceptionUtils.stringifyException(userException)) + .isEqualTo(ExceptionUtils.stringifyException(wronglyDeserialized)); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -111,24 +107,20 @@ public void testSerialization() { } @Test - public void testCauseChaining() { + void testCauseChaining() { Exception cause2 = new Exception("level2"); Exception cause1 = new Exception("level1", cause2); Exception root = new Exception("level0", cause1); SerializedThrowable st = new SerializedThrowable(root); - assertEquals("java.lang.Exception: level0", st.getMessage()); - - assertNotNull(st.getCause()); - assertEquals("java.lang.Exception: level1", st.getCause().getMessage()); - - assertNotNull(st.getCause().getCause()); - assertEquals("java.lang.Exception: level2", st.getCause().getCause().getMessage()); + assertThat(st).hasMessage("java.lang.Exception: level0"); + assertThat(st.getCause()).isNotNull().hasMessage("java.lang.Exception: level1"); + assertThat(st.getCause().getCause()).isNotNull().hasMessage("java.lang.Exception: level2"); } @Test - public void testCyclicCauseChaining() { + void testCyclicCauseChaining() { Exception cause3 = new Exception("level3"); Exception cause2 = new Exception("level2", cause3); Exception cause1 = new Exception("level1", cause2); @@ -139,43 +131,43 @@ public void testCyclicCauseChaining() { SerializedThrowable st = new SerializedThrowable(root); - assertArrayEquals(root.getStackTrace(), st.getStackTrace()); - assertEquals( - ExceptionUtils.stringifyException(root), ExceptionUtils.stringifyException(st)); + assertThat(root.getStackTrace()).isEqualTo(st.getStackTrace()); + assertThat(ExceptionUtils.stringifyException(root)) + .isEqualTo(ExceptionUtils.stringifyException(st)); } @Test - public void testCopyPreservesCause() { + void testCopyPreservesCause() { Exception original = new Exception("original message"); Exception parent = new Exception("parent message", original); SerializedThrowable serialized = new SerializedThrowable(parent); - assertNotNull(serialized.getCause()); + assertThat(serialized.getCause()).isNotNull(); SerializedThrowable copy = new SerializedThrowable(serialized); - assertEquals( - "org.apache.flink.util.SerializedThrowable: java.lang.Exception: parent message", - copy.getMessage()); - assertNotNull(copy.getCause()); - assertEquals("java.lang.Exception: original message", copy.getCause().getMessage()); + assertThat(copy) + .hasMessage( + "org.apache.flink.util.SerializedThrowable: java.lang.Exception: parent message"); + assertThat(copy.getCause()).isNotNull().hasMessage("java.lang.Exception: original message"); } @Test - public void testSuppressedTransferring() { + void testSuppressedTransferring() { Exception root = new Exception("root"); Exception suppressed = new Exception("suppressed"); root.addSuppressed(suppressed); SerializedThrowable serializedThrowable = new SerializedThrowable(root); - assertEquals(1, serializedThrowable.getSuppressed().length); + assertThat(serializedThrowable.getSuppressed()).hasSize(1); Throwable actualSuppressed = serializedThrowable.getSuppressed()[0]; - assertTrue(actualSuppressed instanceof SerializedThrowable); - assertEquals("java.lang.Exception: suppressed", actualSuppressed.getMessage()); + assertThat(actualSuppressed) + .isInstanceOf(SerializedThrowable.class) + .hasMessage("java.lang.Exception: suppressed"); } @Test - public void testCopySuppressed() { + void testCopySuppressed() { Exception root = new Exception("root"); Exception suppressed = new Exception("suppressed"); root.addSuppressed(suppressed); @@ -183,9 +175,10 @@ public void testCopySuppressed() { SerializedThrowable serializedThrowable = new SerializedThrowable(root); SerializedThrowable copy = new SerializedThrowable(serializedThrowable); - assertEquals(1, copy.getSuppressed().length); + assertThat(copy.getSuppressed()).hasSize(1); Throwable actualSuppressed = copy.getSuppressed()[0]; - assertTrue(actualSuppressed instanceof SerializedThrowable); - assertEquals("java.lang.Exception: suppressed", actualSuppressed.getMessage()); + assertThat(actualSuppressed) + .isInstanceOf(SerializedThrowable.class) + .hasMessage("java.lang.Exception: suppressed"); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtilsTest.java index ba9bdb82f032e4..563cf06b9af798 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SlotSelectionStrategyUtilsTest.java @@ -24,18 +24,16 @@ import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; import org.apache.flink.runtime.jobmaster.slotpool.PreviousAllocationSlotSelectionStrategy; import org.apache.flink.runtime.jobmaster.slotpool.SlotSelectionStrategy; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; +import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link SlotSelectionStrategyUtils}. */ -public class SlotSelectionStrategyUtilsTest extends TestLogger { +class SlotSelectionStrategyUtilsTest { @Test - public void testCreatePreviousAllocationSlotSelectionStrategyForLocalRecoveryStreamingJob() { + void testCreatePreviousAllocationSlotSelectionStrategyForLocalRecoveryStreamingJob() { final Configuration configuration = new Configuration(); configuration.set(CheckpointingOptions.LOCAL_RECOVERY, true); @@ -43,12 +41,12 @@ public void testCreatePreviousAllocationSlotSelectionStrategyForLocalRecoveryStr SlotSelectionStrategyUtils.selectSlotSelectionStrategy( JobType.STREAMING, configuration); - assertThat( - slotSelectionStrategy, instanceOf(PreviousAllocationSlotSelectionStrategy.class)); + assertThat(slotSelectionStrategy) + .isInstanceOf(PreviousAllocationSlotSelectionStrategy.class); } @Test - public void testCreateLocationPreferenceSlotSelectionStrategyForLocalRecoveryBatchJob() { + void testCreateLocationPreferenceSlotSelectionStrategyForLocalRecoveryBatchJob() { final Configuration configuration = new Configuration(); configuration.set(CheckpointingOptions.LOCAL_RECOVERY, true); @@ -56,7 +54,7 @@ public void testCreateLocationPreferenceSlotSelectionStrategyForLocalRecoveryBat SlotSelectionStrategyUtils.selectSlotSelectionStrategy( JobType.BATCH, configuration); - assertThat( - slotSelectionStrategy, instanceOf(LocationPreferenceSlotSelectionStrategy.class)); + assertThat(slotSelectionStrategy) + .isInstanceOf(LocationPreferenceSlotSelectionStrategy.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/StateHandleStoreUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/StateHandleStoreUtilsTest.java index eb79ade5b6f7fe..48f0710dd5efda 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/StateHandleStoreUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/StateHandleStoreUtilsTest.java @@ -20,55 +20,50 @@ import org.apache.flink.runtime.persistence.TestingLongStateHandleHelper; import org.apache.flink.runtime.state.StateObject; -import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.RunnableWithException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.ObjectOutputStream; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * {@code StateHandleStoreUtilsTest} tests the utility classes collected in {@link * StateHandleStoreUtils}. */ -public class StateHandleStoreUtilsTest extends TestLogger { +class StateHandleStoreUtilsTest { @Test - public void testSerializationAndDeserialization() throws Exception { + void testSerializationAndDeserialization() throws Exception { final TestingLongStateHandleHelper.LongStateHandle original = new TestingLongStateHandleHelper.LongStateHandle(42L); byte[] serializedData = StateHandleStoreUtils.serializeOrDiscard(original); final TestingLongStateHandleHelper.LongStateHandle deserializedInstance = StateHandleStoreUtils.deserialize(serializedData); - assertThat(deserializedInstance.getStateSize(), is(original.getStateSize())); - assertThat(deserializedInstance.getValue(), is(original.getValue())); + assertThat(deserializedInstance.getStateSize()).isEqualTo(original.getStateSize()); + assertThat(deserializedInstance.getValue()).isEqualTo(original.getValue()); } @Test - public void testSerializeOrDiscardFailureHandling() throws Exception { + void testSerializeOrDiscardFailureHandling() throws Exception { final AtomicBoolean discardCalled = new AtomicBoolean(false); final StateObject original = new FailingSerializationStateObject(() -> discardCalled.set(true)); - try { - StateHandleStoreUtils.serializeOrDiscard(original); - fail("An IOException is expected to be thrown."); - } catch (IOException e) { - // IOException is expected - } + assertThatThrownBy(() -> StateHandleStoreUtils.serializeOrDiscard(original)) + .withFailMessage("An IOException is expected to be thrown.") + .isInstanceOf(IOException.class); - assertThat(discardCalled.get(), is(true)); + assertThat(discardCalled).isTrue(); } @Test - public void testSerializationOrDiscardWithDiscardFailure() throws Exception { + void testSerializationOrDiscardWithDiscardFailure() throws Exception { final Exception discardException = new IllegalStateException( "Expected IllegalStateException that should be suppressed."); @@ -78,14 +73,14 @@ public void testSerializationOrDiscardWithDiscardFailure() throws Exception { throw discardException; }); - try { - StateHandleStoreUtils.serializeOrDiscard(original); - fail("An IOException is expected to be thrown."); - } catch (IOException e) { - // IOException is expected - assertThat(e.getSuppressed().length, is(1)); - assertThat(e.getSuppressed()[0], is(discardException)); - } + assertThatThrownBy(() -> StateHandleStoreUtils.serializeOrDiscard(original)) + .withFailMessage("An IOException is expected to be thrown.") + .isInstanceOf(IOException.class) + .satisfies( + e -> { + assertThat(e.getSuppressed()).hasSize(1); + assertThat(e.getSuppressed()[0]).isEqualTo(discardException); + }); } private static class FailingSerializationStateObject implements StateObject { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTest.java index 18ce166fd6aed0..a9282429794d01 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTest.java @@ -20,23 +20,19 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.flink.shaded.curator5.org.apache.curator.retry.ExponentialBackoffRetry; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link ZooKeeperUtils}. */ -public class ZooKeeperUtilsTest extends TestLogger { +class ZooKeeperUtilsTest { @Test - public void testZookeeperPathGeneration() { + void testZookeeperPathGeneration() { runZookeeperPathGenerationTest("/root/namespace", "root", "namespace"); runZookeeperPathGenerationTest("/root/namespace", "/root/", "/namespace/"); runZookeeperPathGenerationTest("/root/namespace", "//root//", "//namespace//"); @@ -49,7 +45,7 @@ public void testZookeeperPathGeneration() { } @Test - public void testZooKeeperEnsembleConnectStringConfiguration() throws Exception { + void testZooKeeperEnsembleConnectStringConfiguration() throws Exception { // ZooKeeper does not like whitespace in the quorum connect String. String actual, expected; Configuration conf = new Configuration(); @@ -59,15 +55,15 @@ public void testZooKeeperEnsembleConnectStringConfiguration() throws Exception { setQuorum(conf, expected); actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); setQuorum(conf, " localhost:2891 "); // with leading and trailing whitespace actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); setQuorum(conf, "localhost :2891"); // whitespace after port actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } { @@ -75,24 +71,24 @@ public void testZooKeeperEnsembleConnectStringConfiguration() throws Exception { setQuorum(conf, "localhost:2891,localhost:2891"); actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); setQuorum(conf, "localhost:2891, localhost:2891"); actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); setQuorum(conf, "localhost :2891, localhost:2891"); actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); setQuorum(conf, " localhost:2891, localhost:2891 "); actual = ZooKeeperUtils.getZooKeeperEnsemble(conf); - assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } @Test - public void testStartCuratorFrameworkFailed() throws Exception { + void testStartCuratorFrameworkFailed() throws Exception { TestingFatalErrorHandler handler = new TestingFatalErrorHandler(); String errorMsg = "unexpected exception"; final CuratorFrameworkFactory.Builder curatorFrameworkBuilder = @@ -105,13 +101,13 @@ public void testStartCuratorFrameworkFailed() throws Exception { }) .namespace("flink"); ZooKeeperUtils.startCuratorFramework(curatorFrameworkBuilder, handler); - Assert.assertEquals(errorMsg, handler.getErrorFuture().get().getMessage()); + assertThat(handler.getErrorFuture().get()).hasMessage(errorMsg); } private void runZookeeperPathGenerationTest(String expectedValue, String... paths) { final String result = ZooKeeperUtils.generateZookeeperPath(paths); - assertThat(result, is(expectedValue)); + assertThat(result).isEqualTo(expectedValue); } private Configuration setQuorum(Configuration conf, String quorum) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTreeCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTreeCacheTest.java index 0eb63aa582c12e..22e6d8b5c144d1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTreeCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsTreeCacheTest.java @@ -20,11 +20,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache; @@ -33,21 +31,21 @@ import org.apache.flink.shaded.guava31.com.google.common.io.Closer; import org.apache.curator.test.TestingServer; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import static org.hamcrest.MatcherAssert.assertThat; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; /** * Tests for {@link ZooKeeperUtils#createTreeCache(CuratorFramework, String, * org.apache.flink.util.function.RunnableWithException)}. */ -public class ZooKeeperUtilsTreeCacheTest extends TestLogger { +class ZooKeeperUtilsTreeCacheTest { private static final String PARENT_PATH = "/foo"; private static final String CHILD_PATH = PARENT_PATH + "/bar"; @@ -59,8 +57,8 @@ public class ZooKeeperUtilsTreeCacheTest extends TestLogger { private final AtomicReference> callbackFutureReference = new AtomicReference<>(); - @Before - public void setUp() throws Exception { + @BeforeEach + void setUp() throws Exception { closer = Closer.create(); final TestingServer testingServer = closer.register(ZooKeeperTestUtils.createAndStartZookeeperTestingServer()); @@ -85,14 +83,14 @@ public void setUp() throws Exception { cache.start(); } - @After - public void tearDown() throws Exception { + @AfterEach + void tearDown() throws Exception { closer.close(); callbackFutureReference.set(null); } @Test - public void testCallbackCalledOnNodeCreation() throws Exception { + void testCallbackCalledOnNodeCreation() throws Exception { client.create().forPath(PARENT_PATH); callbackFutureReference.set(new CompletableFuture<>()); client.create().forPath(CHILD_PATH); @@ -100,7 +98,7 @@ public void testCallbackCalledOnNodeCreation() throws Exception { } @Test - public void testCallbackCalledOnNodeModification() throws Exception { + void testCallbackCalledOnNodeModification() throws Exception { testCallbackCalledOnNodeCreation(); callbackFutureReference.set(new CompletableFuture<>()); @@ -109,7 +107,7 @@ public void testCallbackCalledOnNodeModification() throws Exception { } @Test - public void testCallbackCalledOnNodeDeletion() throws Exception { + void testCallbackCalledOnNodeDeletion() throws Exception { testCallbackCalledOnNodeCreation(); callbackFutureReference.set(new CompletableFuture<>()); @@ -118,38 +116,35 @@ public void testCallbackCalledOnNodeDeletion() throws Exception { } @Test - public void testCallbackNotCalledOnCreationOfParents() throws Exception { + void testCallbackNotCalledOnCreationOfParents() throws Exception { callbackFutureReference.set(new CompletableFuture<>()); client.create().forPath(PARENT_PATH); - assertThat( - callbackFutureReference.get(), - FlinkMatchers.willNotComplete(Duration.ofMillis(20))); + assertThatFuture(callbackFutureReference.get()) + .willNotCompleteWithin(Duration.ofMillis(20)); } @Test - public void testCallbackNotCalledOnCreationOfChildren() throws Exception { + void testCallbackNotCalledOnCreationOfChildren() throws Exception { testCallbackCalledOnNodeCreation(); callbackFutureReference.set(new CompletableFuture<>()); client.create().forPath(CHILD_PATH + "/baz"); - assertThat( - callbackFutureReference.get(), - FlinkMatchers.willNotComplete(Duration.ofMillis(20))); + assertThatFuture(callbackFutureReference.get()) + .willNotCompleteWithin(Duration.ofMillis(20)); } @Test - public void testCallbackNotCalledOnCreationOfSimilarPaths() throws Exception { + void testCallbackNotCalledOnCreationOfSimilarPaths() throws Exception { callbackFutureReference.set(new CompletableFuture<>()); client.create() .creatingParentContainersIfNeeded() .forPath(CHILD_PATH.substring(0, CHILD_PATH.length() - 1)); - assertThat( - callbackFutureReference.get(), - FlinkMatchers.willNotComplete(Duration.ofMillis(20))); + assertThatFuture(callbackFutureReference.get()) + .willNotCompleteWithin(Duration.ofMillis(20)); } @Test - public void testCallbackNotCalledOnConnectionOrInitializationEvents() throws Exception { + void testCallbackNotCalledOnConnectionOrInitializationEvents() throws Exception { final TreeCacheListener treeCacheListener = ZooKeeperUtils.createTreeCacheListener( () -> { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsWriteLeaderInformationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsWriteLeaderInformationTest.java index 5c85c2df336a1f..639fa823aefcd8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsWriteLeaderInformationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilsWriteLeaderInformationTest.java @@ -25,10 +25,8 @@ import org.apache.flink.runtime.leaderelection.LeaderInformation; import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; -import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; import javax.annotation.Nonnull; @@ -38,7 +36,6 @@ import static org.assertj.core.api.Assertions.assertThat; /** Integration tests for the {@link ZooKeeperUtils}. */ -@ExtendWith(TestLoggerExtension.class) class ZooKeeperUtilsWriteLeaderInformationTest { private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/BashJavaUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/BashJavaUtilsTest.java index cae57870b6f6db..1ecca27bb5d198 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/BashJavaUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/BashJavaUtilsTest.java @@ -22,26 +22,24 @@ import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Map; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link BashJavaUtils}. */ -public class BashJavaUtilsTest extends TestLogger { +class BashJavaUtilsTest { @Test - public void testJmLegacyHeapOptionSetsNewJvmHeap() { + void testJmLegacyHeapOptionSetsNewJvmHeap() { Configuration configuration = new Configuration(); MemorySize heapSize = MemorySize.ofMebiBytes(10); configuration.set(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, heapSize); String jvmArgsLine = BashJavaUtils.getJmResourceParams(configuration).get(0); Map jvmArgs = ConfigurationUtils.parseJvmArgString(jvmArgsLine); String heapSizeStr = Long.toString(heapSize.getBytes()); - assertThat(jvmArgs.get("-Xmx"), is(heapSizeStr)); - assertThat(jvmArgs.get("-Xms"), is(heapSizeStr)); + assertThat(jvmArgs.get("-Xmx")).isEqualTo(heapSizeStr); + assertThat(jvmArgs.get("-Xms")).isEqualTo(heapSizeStr); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/FlinkConfigLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/FlinkConfigLoaderTest.java index 5c2507637ce332..a7f6fe96252d83 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/FlinkConfigLoaderTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/bash/FlinkConfigLoaderTest.java @@ -20,73 +20,70 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.TestLogger; +import org.apache.flink.testutils.junit.utils.TempDirUtils; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.FileWriter; +import java.io.IOException; import static org.apache.flink.configuration.ConfigOptions.key; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link FlinkConfigLoader}. */ -public class FlinkConfigLoaderTest extends TestLogger { +class FlinkConfigLoaderTest { private static final String TEST_CONFIG_KEY = "test.key"; private static final String TEST_CONFIG_VALUE = "test_value"; - @Rule - public TemporaryFolder confDir = - new TemporaryFolder() { - @Override - protected void before() throws Throwable { - super.create(); - File flinkConfFile = newFile("flink-conf.yaml"); - FileWriter fw = new FileWriter(flinkConfFile); - fw.write(TEST_CONFIG_KEY + ": " + TEST_CONFIG_VALUE + "\n"); - fw.close(); - } - }; + @TempDir private java.nio.file.Path confDir; + + @BeforeEach + void setUp() throws IOException { + File flinkConfFile = TempDirUtils.newFile(confDir.toAbsolutePath(), "flink-conf.yaml"); + FileWriter fw = new FileWriter(flinkConfFile); + fw.write(TEST_CONFIG_KEY + ": " + TEST_CONFIG_VALUE + "\n"); + fw.close(); + } @Test - public void testLoadConfigurationConfigDirLongOpt() throws Exception { - String[] args = {"--configDir", confDir.getRoot().getAbsolutePath()}; + void testLoadConfigurationConfigDirLongOpt() throws Exception { + String[] args = {"--configDir", confDir.toFile().getAbsolutePath()}; Configuration configuration = FlinkConfigLoader.loadConfiguration(args); verifyConfiguration(configuration, TEST_CONFIG_KEY, TEST_CONFIG_VALUE); } @Test - public void testLoadConfigurationConfigDirShortOpt() throws Exception { - String[] args = {"-c", confDir.getRoot().getAbsolutePath()}; + void testLoadConfigurationConfigDirShortOpt() throws Exception { + String[] args = {"-c", confDir.toFile().getAbsolutePath()}; Configuration configuration = FlinkConfigLoader.loadConfiguration(args); verifyConfiguration(configuration, TEST_CONFIG_KEY, TEST_CONFIG_VALUE); } @Test - public void testLoadConfigurationDynamicPropertyWithSpace() throws Exception { - String[] args = {"--configDir", confDir.getRoot().getAbsolutePath(), "-D", "key=value"}; + void testLoadConfigurationDynamicPropertyWithSpace() throws Exception { + String[] args = {"--configDir", confDir.toFile().getAbsolutePath(), "-D", "key=value"}; Configuration configuration = FlinkConfigLoader.loadConfiguration(args); verifyConfiguration(configuration, "key", "value"); } @Test - public void testLoadConfigurationDynamicPropertyWithoutSpace() throws Exception { - String[] args = {"--configDir", confDir.getRoot().getAbsolutePath(), "-Dkey=value"}; + void testLoadConfigurationDynamicPropertyWithoutSpace() throws Exception { + String[] args = {"--configDir", confDir.toFile().getAbsolutePath(), "-Dkey=value"}; Configuration configuration = FlinkConfigLoader.loadConfiguration(args); verifyConfiguration(configuration, "key", "value"); } @Test - public void testLoadConfigurationIgnoreUnknownToken() throws Exception { + void testLoadConfigurationIgnoreUnknownToken() throws Exception { String[] args = { "unknown", "-u", "--configDir", - confDir.getRoot().getAbsolutePath(), + confDir.toFile().getAbsolutePath(), "--unknown", "-Dkey=value" }; @@ -97,6 +94,6 @@ public void testLoadConfigurationIgnoreUnknownToken() throws Exception { private void verifyConfiguration(Configuration config, String key, String expectedValue) { ConfigOption option = key(key).stringType().noDefaultValue(); - assertThat(config.get(option), is(expectedValue)); + assertThat(config.get(option)).isEqualTo(expectedValue); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/JobManagerProcessSpecTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/JobManagerProcessSpecTest.java index daff3de2970165..195ade5828f58e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/JobManagerProcessSpecTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/JobManagerProcessSpecTest.java @@ -20,18 +20,15 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.jobmanager.JobManagerProcessSpec; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link JobManagerProcessSpec}. */ -public class JobManagerProcessSpecTest extends TestLogger { +class JobManagerProcessSpecTest { @Test - public void testEquals() { + void testEquals() { JobManagerProcessSpec spec1 = new JobManagerProcessSpec( MemorySize.parse("1m"), @@ -46,11 +43,11 @@ public void testEquals() { MemorySize.parse("3m"), MemorySize.parse("4m")); - assertThat(spec1, is(spec2)); + assertThat(spec1).isEqualTo(spec2); } @Test - public void testNotEquals() { + void testNotEquals() { JobManagerProcessSpec spec1 = new JobManagerProcessSpec( MemorySize.parse("1m"), @@ -62,6 +59,6 @@ public void testNotEquals() { new JobManagerProcessSpec( MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO); - assertThat(spec1, not(spec2)); + assertThat(spec1).isNotEqualTo(spec2); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ManagedMemoryUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ManagedMemoryUtilsTest.java index 8a05c3866c8ae6..ea4fcaba30c42e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ManagedMemoryUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ManagedMemoryUtilsTest.java @@ -23,9 +23,9 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.core.memory.ManagedMemoryUseCase; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.assertj.core.data.Offset; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; @@ -34,12 +34,11 @@ import java.util.Optional; import java.util.Set; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link ManagedMemoryUtils}. */ -public class ManagedMemoryUtilsTest extends TestLogger { +class ManagedMemoryUtilsTest { private static final double DELTA = 0.000001; @@ -100,7 +99,7 @@ public class ManagedMemoryUtilsTest extends TestLogger { }); @Test - public void testGetWeightsFromConfig() { + void testGetWeightsFromConfig() { final Map expectedWeights = new HashMap() { { @@ -114,11 +113,11 @@ public void testGetWeightsFromConfig() { ManagedMemoryUtils.getManagedMemoryUseCaseWeightsFromConfig( CONFIG_WITH_ALL_USE_CASES); - assertThat(configuredWeights, is(expectedWeights)); + assertThat(configuredWeights).isEqualTo(expectedWeights); } @Test - public void testGetWeightsFromConfigLegacy() { + void testGetWeightsFromConfigLegacy() { final Map expectedWeights = new HashMap() { { @@ -132,11 +131,11 @@ public void testGetWeightsFromConfigLegacy() { ManagedMemoryUtils.getManagedMemoryUseCaseWeightsFromConfig( CONFIG_WITH_LEGACY_USE_CASES); - assertThat(configuredWeights, is(expectedWeights)); + assertThat(configuredWeights).isEqualTo(expectedWeights); } - @Test(expected = IllegalConfigurationException.class) - public void testGetWeightsFromConfigFailNegativeWeight() { + @Test + void testGetWeightsFromConfigFailNegativeWeight() { final Configuration config = new Configuration() { { @@ -147,12 +146,13 @@ public void testGetWeightsFromConfigFailNegativeWeight() { "-123")); } }; - - ManagedMemoryUtils.getManagedMemoryUseCaseWeightsFromConfig(config); + assertThatThrownBy( + () -> ManagedMemoryUtils.getManagedMemoryUseCaseWeightsFromConfig(config)) + .isInstanceOf(IllegalConfigurationException.class); } @Test - public void testConvertToFractionOfSlot() { + void testConvertToFractionOfSlot() { final ManagedMemoryUseCase useCase = ManagedMemoryUseCase.OPERATOR; final double fractionOfUseCase = 0.3; @@ -171,11 +171,13 @@ public void testConvertToFractionOfSlot() { Optional.of(true), ClassLoader.getSystemClassLoader()); - assertEquals(fractionOfUseCase * OPERATOR_WEIGHT / TOTAL_WEIGHT, fractionOfSlot, DELTA); + assertThat(fractionOfSlot) + .isCloseTo( + fractionOfUseCase * OPERATOR_WEIGHT / TOTAL_WEIGHT, Offset.offset(DELTA)); } @Test - public void testConvertToFractionOfSlotWeightNotConfigured() { + void testConvertToFractionOfSlotWeightNotConfigured() { final ManagedMemoryUseCase useCase = ManagedMemoryUseCase.OPERATOR; final double fractionOfUseCase = 0.3; @@ -203,11 +205,11 @@ public void testConvertToFractionOfSlotWeightNotConfigured() { Optional.of(true), ClassLoader.getSystemClassLoader()); - assertEquals(0.0, fractionOfSlot, DELTA); + assertThat(fractionOfSlot).isCloseTo(0.0, Offset.offset(DELTA)); } @Test - public void testConvertToFractionOfSlotStateBackendUseManagedMemory() { + void testConvertToFractionOfSlotStateBackendUseManagedMemory() { testConvertToFractionOfSlotGivenWhetherStateBackendUsesManagedMemory( true, 1.0 * OPERATOR_WEIGHT / TOTAL_WEIGHT, @@ -216,7 +218,7 @@ public void testConvertToFractionOfSlotStateBackendUseManagedMemory() { } @Test - public void testConvertToFractionOfSlotStateBackendNotUserManagedMemory() { + void testConvertToFractionOfSlotStateBackendNotUserManagedMemory() { final int totalWeight = OPERATOR_WEIGHT + PYTHON_WEIGHT; testConvertToFractionOfSlotGivenWhetherStateBackendUsesManagedMemory( false, 1.0 * OPERATOR_WEIGHT / totalWeight, 0.0, 1.0 * PYTHON_WEIGHT / totalWeight); @@ -262,13 +264,16 @@ private void testConvertToFractionOfSlotGivenWhetherStateBackendUsesManagedMemor Optional.of(stateBackendUsesManagedMemory), ClassLoader.getSystemClassLoader()); - assertEquals(expectedOperatorFractionOfSlot, opFractionOfSlot, DELTA); - assertEquals(expectedStateFractionOfSlot, stateFractionOfSlot, DELTA); - assertEquals(expectedPythonFractionOfSlot, pythonFractionOfSlot, DELTA); + assertThat(opFractionOfSlot) + .isCloseTo(expectedOperatorFractionOfSlot, Offset.offset(DELTA)); + assertThat(stateFractionOfSlot) + .isCloseTo(expectedStateFractionOfSlot, Offset.offset(DELTA)); + assertThat(pythonFractionOfSlot) + .isCloseTo(expectedPythonFractionOfSlot, Offset.offset(DELTA)); } @Test - public void testUseCaseWeightsConfiguredWithConsistentValue() { + void testUseCaseWeightsConfiguredWithConsistentValue() { final Map existingWeights = new HashMap() { { @@ -287,8 +292,8 @@ public void testUseCaseWeightsConfiguredWithConsistentValue() { ManagedMemoryUtils.validateUseCaseWeightsNotConflict(existingWeights, newWeights); } - @Test(expected = IllegalStateException.class) - public void testUseCaseWeightsConfiguredWithConflictValue() { + @Test + void testUseCaseWeightsConfiguredWithConflictValue() { final Map existingWeights = new HashMap() { { @@ -303,6 +308,10 @@ public void testUseCaseWeightsConfiguredWithConflictValue() { } }; - ManagedMemoryUtils.validateUseCaseWeightsNotConflict(existingWeights, newWeights); + assertThatThrownBy( + () -> + ManagedMemoryUtils.validateUseCaseWeightsNotConflict( + existingWeights, newWeights)) + .isInstanceOf(IllegalStateException.class); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java index a65223bc1d67bc..7d71fb8f0d762b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/ProcessMemoryUtilsTestBase.java @@ -24,29 +24,28 @@ import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.MemorySize; import org.apache.flink.core.testutils.CommonTestUtils; -import org.apache.flink.util.TestLogger; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** * Base test suite for JM/TM memory calculations to test common methods in {@link * ProcessMemoryUtils}. */ -@SuppressWarnings("AbstractClassExtendsConcreteClass") -public abstract class ProcessMemoryUtilsTestBase extends TestLogger { +public abstract class ProcessMemoryUtilsTestBase { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + private static Map oldEnvVariables; private final ProcessMemoryOptions options; @@ -63,86 +62,84 @@ protected ProcessMemoryUtilsTestBase( this.newOptionForLegacyHeapOption = checkNotNull(newOptionForLegacyHeapOption); } - @Before - public void setup() { + @BeforeEach + void setup() { oldEnvVariables = System.getenv(); } - @After - public void teardown() { + @AfterEach + void teardown() { if (oldEnvVariables != null) { CommonTestUtils.setEnv(oldEnvVariables, true); } } @Test - public void testGenerateJvmParameters() { + void testGenerateJvmParameters() { ProcessMemorySpec spec = JvmArgTestingProcessMemorySpec.generate(); String jvmParamsStr = ProcessMemoryUtils.generateJvmParametersStr(spec, true); Map configs = ConfigurationUtils.parseJvmArgString(jvmParamsStr); - assertThat(configs.size(), is(4)); - assertThat(MemorySize.parse(configs.get("-Xmx")), is(spec.getJvmHeapMemorySize())); - assertThat(MemorySize.parse(configs.get("-Xms")), is(spec.getJvmHeapMemorySize())); - assertThat( - MemorySize.parse(configs.get("-XX:MaxMetaspaceSize=")), - is(spec.getJvmMetaspaceSize())); - assertThat( - MemorySize.parse(configs.get("-XX:MaxDirectMemorySize=")), - is(spec.getJvmDirectMemorySize())); + assertThat(configs).hasSize(4); + assertThat(MemorySize.parse(configs.get("-Xmx"))).isEqualTo(spec.getJvmHeapMemorySize()); + assertThat(MemorySize.parse(configs.get("-Xms"))).isEqualTo(spec.getJvmHeapMemorySize()); + assertThat(MemorySize.parse(configs.get("-XX:MaxMetaspaceSize="))) + .isEqualTo(spec.getJvmMetaspaceSize()); + assertThat(MemorySize.parse(configs.get("-XX:MaxDirectMemorySize="))) + .isEqualTo(spec.getJvmDirectMemorySize()); } @Test - public void testGenerateJvmParametersWithoutDirectMemoryLimit() { + void testGenerateJvmParametersWithoutDirectMemoryLimit() { ProcessMemorySpec spec = JvmArgTestingProcessMemorySpec.generate(); String jvmParamsStr = ProcessMemoryUtils.generateJvmParametersStr(spec, false); Map configs = ConfigurationUtils.parseJvmArgString(jvmParamsStr); - assertThat(configs.size(), is(3)); - assertThat(MemorySize.parse(configs.get("-Xmx")), is(spec.getJvmHeapMemorySize())); - assertThat(MemorySize.parse(configs.get("-Xms")), is(spec.getJvmHeapMemorySize())); - assertThat( - MemorySize.parse(configs.get("-XX:MaxMetaspaceSize=")), - is(spec.getJvmMetaspaceSize())); - assertThat(configs.containsKey("-XX:MaxDirectMemorySize="), is(false)); + assertThat(configs).hasSize(3); + assertThat(MemorySize.parse(configs.get("-Xmx"))).isEqualTo(spec.getJvmHeapMemorySize()); + assertThat(MemorySize.parse(configs.get("-Xms"))).isEqualTo(spec.getJvmHeapMemorySize()); + assertThat(MemorySize.parse(configs.get("-XX:MaxMetaspaceSize="))) + .isEqualTo(spec.getJvmMetaspaceSize()); + assertThat(configs.containsKey("-XX:MaxDirectMemorySize=")).isFalse(); } @Test - public void testConfigTotalFlinkMemory() { + void testConfigTotalFlinkMemory() { MemorySize totalFlinkMemorySize = MemorySize.parse("1g"); Configuration conf = new Configuration(); conf.set(options.getTotalFlinkMemoryOption(), totalFlinkMemorySize); T processSpec = processSpecFromConfig(conf); - assertThat(processSpec.getTotalFlinkMemorySize(), is(totalFlinkMemorySize)); + assertThat(processSpec.getTotalFlinkMemorySize()).isEqualTo(totalFlinkMemorySize); } @Test - public void testConfigTotalProcessMemorySize() { + void testConfigTotalProcessMemorySize() { MemorySize totalProcessMemorySize = MemorySize.parse("2g"); Configuration conf = new Configuration(); conf.set(options.getTotalProcessMemoryOption(), totalProcessMemorySize); T processSpec = processSpecFromConfig(conf); - assertThat(processSpec.getTotalProcessMemorySize(), is(totalProcessMemorySize)); + assertThat(processSpec.getTotalProcessMemorySize()).isEqualTo(totalProcessMemorySize); } @Test - public void testExceptionShouldContainRequiredConfigOptions() { + void testExceptionShouldContainRequiredConfigOptions() { try { processSpecFromConfig(new Configuration()); } catch (IllegalConfigurationException e) { options.getRequiredFineGrainedOptions() - .forEach(option -> assertThat(e.getMessage(), containsString(option.key()))); - assertThat(e.getMessage(), containsString(options.getTotalFlinkMemoryOption().key())); - assertThat(e.getMessage(), containsString(options.getTotalProcessMemoryOption().key())); + .forEach(option -> assertThat(e).hasMessageContaining(option.key())); + assertThat(e) + .hasMessageContaining(options.getTotalFlinkMemoryOption().key()) + .hasMessageContaining(options.getTotalProcessMemoryOption().key()); } } @Test - public void testDerivedTotalProcessMemoryGreaterThanConfiguredFailureWithFineGrainedOptions() { + void testDerivedTotalProcessMemoryGreaterThanConfiguredFailureWithFineGrainedOptions() { Configuration conf = getConfigurationWithJvmMetaspaceAndTotalFlinkMemory(100, 200); // Total Flink memory + JVM Metaspace > Total Process Memory (no space for JVM overhead) MemorySize totalFlinkMemorySize = MemorySize.ofMebiBytes(150); @@ -151,7 +148,7 @@ public void testDerivedTotalProcessMemoryGreaterThanConfiguredFailureWithFineGra } @Test - public void testDerivedTotalProcessMemoryGreaterThanConfiguredFailureWithTotalFlinkMemory() { + void testDerivedTotalProcessMemoryGreaterThanConfiguredFailureWithTotalFlinkMemory() { Configuration conf = getConfigurationWithJvmMetaspaceAndTotalFlinkMemory(100, 200); // Total Flink memory + JVM Metaspace > Total Process Memory (no space for JVM overhead) MemorySize totalFlinkMemorySize = MemorySize.ofMebiBytes(150); @@ -170,7 +167,7 @@ private Configuration getConfigurationWithJvmMetaspaceAndTotalFlinkMemory( } @Test - public void testConfigJvmMetaspaceSize() { + void testConfigJvmMetaspaceSize() { MemorySize jvmMetaspaceSize = MemorySize.parse("50m"); Configuration conf = new Configuration(); @@ -178,11 +175,12 @@ public void testConfigJvmMetaspaceSize() { validateInAllConfigurations( conf, - processSpec -> assertThat(processSpec.getJvmMetaspaceSize(), is(jvmMetaspaceSize))); + processSpec -> + assertThat(processSpec.getJvmMetaspaceSize()).isEqualTo(jvmMetaspaceSize)); } @Test - public void testConfigJvmOverheadRange() { + void testConfigJvmOverheadRange() { MemorySize minSize = MemorySize.parse("50m"); MemorySize maxSize = MemorySize.parse("200m"); @@ -193,17 +191,15 @@ public void testConfigJvmOverheadRange() { validateInAllConfigurations( conf, JobManagerProcessSpec -> { - assertThat( - JobManagerProcessSpec.getJvmOverheadSize().getBytes(), - greaterThanOrEqualTo(minSize.getBytes())); - assertThat( - JobManagerProcessSpec.getJvmOverheadSize().getBytes(), - lessThanOrEqualTo(maxSize.getBytes())); + assertThat(JobManagerProcessSpec.getJvmOverheadSize().getBytes()) + .isGreaterThanOrEqualTo(minSize.getBytes()); + assertThat(JobManagerProcessSpec.getJvmOverheadSize().getBytes()) + .isLessThanOrEqualTo(maxSize.getBytes()); }); } @Test - public void testConfigJvmOverheadRangeFailure() { + void testConfigJvmOverheadRangeFailure() { MemorySize minSize = MemorySize.parse("200m"); MemorySize maxSize = MemorySize.parse("50m"); @@ -215,7 +211,7 @@ public void testConfigJvmOverheadRangeFailure() { } @Test - public void testConfigJvmOverheadFraction() { + void testConfigJvmOverheadFraction() { MemorySize minSize = MemorySize.ZERO; MemorySize maxSize = MemorySize.parse("1t"); @SuppressWarnings("MagicNumber") @@ -229,16 +225,15 @@ public void testConfigJvmOverheadFraction() { validateInAllConfigurations( conf, jobManagerProcessSpec -> - assertThat( - jobManagerProcessSpec.getJvmOverheadSize(), - is( + assertThat(jobManagerProcessSpec.getJvmOverheadSize()) + .isEqualTo( jobManagerProcessSpec .getTotalProcessMemorySize() - .multiply(fraction)))); + .multiply(fraction))); } @Test - public void testConfigJvmOverheadFractionFailureNegative() { + void testConfigJvmOverheadFractionFailureNegative() { Configuration conf = new Configuration(); //noinspection MagicNumber conf.setFloat(options.getJvmOptions().getJvmOverheadFraction(), -0.1f); @@ -246,14 +241,14 @@ public void testConfigJvmOverheadFractionFailureNegative() { } @Test - public void testConfigJvmOverheadFractionFailureNoLessThanOne() { + void testConfigJvmOverheadFractionFailureNoLessThanOne() { Configuration conf = new Configuration(); conf.setFloat(options.getJvmOptions().getJvmOverheadFraction(), 1.0f); validateFailInAllConfigurations(conf); } @Test - public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySize() { + void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySize() { Configuration conf = new Configuration(); conf.set(options.getTotalProcessMemoryOption(), MemorySize.parse("1000m")); conf.set(options.getTotalFlinkMemoryOption(), MemorySize.parse("800m")); @@ -264,11 +259,11 @@ public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySize() { conf.set(options.getJvmOptions().getJvmOverheadFraction(), 0.5f); T jobManagerProcessSpec = processSpecFromConfig(conf); - assertThat(jobManagerProcessSpec.getJvmOverheadSize(), is(MemorySize.parse("100m"))); + assertThat(jobManagerProcessSpec.getJvmOverheadSize()).isEqualTo(MemorySize.parse("100m")); } @Test - public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySizeFailure() { + void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySizeFailure() { Configuration conf = new Configuration(); conf.set(options.getTotalProcessMemoryOption(), MemorySize.parse("1000m")); conf.set(options.getTotalFlinkMemoryOption(), MemorySize.parse("800m")); @@ -282,7 +277,7 @@ public void testConfigJvmOverheadDeriveFromProcessAndFlinkMemorySizeFailure() { } @Test - public void testConfigLegacyHeapSize() { + void testConfigLegacyHeapSize() { MemorySize legacyHeapSize = MemorySize.parse("1g"); Configuration conf = new Configuration(); @@ -292,7 +287,7 @@ public void testConfigLegacyHeapSize() { } @Test - public void testConfigLegacyHeapMB() { + void testConfigLegacyHeapMB() { MemorySize jvmHeapSize = MemorySize.parse("1g"); Configuration conf = new Configuration(); @@ -302,7 +297,7 @@ public void testConfigLegacyHeapMB() { } @Test - public void testConfigLegacyHeapEnv() { + void testConfigLegacyHeapEnv() { MemorySize jvmHeapSize = MemorySize.parse("1g"); Map env = new HashMap<>(); @@ -313,7 +308,7 @@ public void testConfigLegacyHeapEnv() { } @Test - public void testConfigBothNewOptionAndLegacyHeapSize() { + void testConfigBothNewOptionAndLegacyHeapSize() { MemorySize newOptionValue = MemorySize.parse("1g"); MemorySize legacyHeapSize = MemorySize.parse("2g"); @@ -328,11 +323,11 @@ private void testConfigLegacyHeapMemory(Configuration configuration, MemorySize MemorySize newOptionValue = getConfigurationWithLegacyHeapSizeMappedToNewConfigOption(configuration) .get(getNewOptionForLegacyHeapOption()); - assertThat(newOptionValue, is(expected)); + assertThat(newOptionValue).isEqualTo(expected); } @Test - public void testConfigTotalProcessMemoryAddUpFailure() { + void testConfigTotalProcessMemoryAddUpFailure() { MemorySize totalProcessMemory = MemorySize.parse("699m"); MemorySize totalFlinkMemory = MemorySize.parse("500m"); MemorySize jvmMetaspace = MemorySize.parse("100m"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/TaskExecutorProcessSpecTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/TaskExecutorProcessSpecTest.java index 62e3e2065d4462..5bedadd005666f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/TaskExecutorProcessSpecTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/config/memory/TaskExecutorProcessSpecTest.java @@ -22,22 +22,19 @@ import org.apache.flink.api.common.resources.ExternalResource; import org.apache.flink.configuration.MemorySize; import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; -import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Collections; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TaskExecutorProcessSpec}. */ -public class TaskExecutorProcessSpecTest extends TestLogger { +class TaskExecutorProcessSpecTest { private static final String EXTERNAL_RESOURCE_NAME = "gpu"; @Test - public void testEquals() { + void testEquals() { TaskExecutorProcessSpec spec1 = new TaskExecutorProcessSpec( new CPUResource(1.0), @@ -64,11 +61,11 @@ public void testEquals() { MemorySize.parse("8m"), Collections.singleton(new ExternalResource(EXTERNAL_RESOURCE_NAME, 1))); - assertThat(spec1, is(spec2)); + assertThat(spec1).isEqualTo(spec2); } @Test - public void testNotEquals() { + void testNotEquals() { TaskExecutorProcessSpec spec1 = new TaskExecutorProcessSpec( new CPUResource(1.0), @@ -95,6 +92,6 @@ public void testNotEquals() { MemorySize.ZERO, Collections.emptyList()); - assertThat(spec1, not(spec2)); + assertThat(spec1).isNotEqualTo(spec2); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java index 7372e2dfb9f9cd..26edbb731383e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/event/TaskEventHandlerTest.java @@ -23,13 +23,12 @@ import org.apache.flink.runtime.event.task.StringTaskEvent; import org.apache.flink.runtime.io.network.api.TaskEventHandler; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; +import static org.assertj.core.api.Assertions.assertThat; /** This class contains unit tests for the {@link TaskEventHandler}. */ -public class TaskEventHandlerTest { +class TaskEventHandlerTest { /** A test implementation of an {@link EventListener}. */ private static class TestEventListener implements EventListener { @@ -61,7 +60,7 @@ public TaskEvent getLastReceivedEvent() { /** Tests the publish/subscribe mechanisms implemented in the {@link TaskEventHandler}. */ @Test - public void testEventNotificationManager() { + void testEventNotificationManager() { final TaskEventHandler evm = new TaskEventHandler(); final TestEventListener listener = new TestEventListener(); @@ -73,8 +72,8 @@ public void testEventNotificationManager() { evm.publish(stringTaskEvent1); evm.publish(new IntegerTaskEvent(5)); - assertNotNull(listener.getLastReceivedEvent()); + assertThat(listener.getLastReceivedEvent()).isNotNull(); StringTaskEvent receivedStringEvent = (StringTaskEvent) listener.getLastReceivedEvent(); - assertEquals(stringTaskEvent1, receivedStringEvent); + assertThat(stringTaskEvent1).isEqualTo(receivedStringEvent); } }