浏览代码

HADOOP-19434. [JDK17] Upgrade JUnit from 4 to 5 in hadoop-federation-balance. (#7580)

Co-authored-by: Chris Nauroth <cnauroth@apache.org>
Reviewed-by: Chris Nauroth <cnauroth@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
slfan1989 3 周之前
父节点
当前提交
62db9f7d6d

+ 12 - 18
hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestDistCpProcedure.java

@@ -31,11 +31,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceJob;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceJob;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedure.RetryException;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedure.RetryException;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler;
 import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.Timeout;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.OutputStream;
@@ -46,23 +45,23 @@ import java.io.DataInputStream;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayInputStream;
 import java.net.URI;
 import java.net.URI;
 import java.util.Random;
 import java.util.Random;
-import java.util.concurrent.TimeUnit;
 
 
-import static junit.framework.TestCase.assertTrue;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.test.GenericTestUtils.getMethodName;
 import static org.apache.hadoop.test.GenericTestUtils.getMethodName;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.CURRENT_SNAPSHOT_NAME;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.CURRENT_SNAPSHOT_NAME;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.LAST_SNAPSHOT_NAME;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.LAST_SNAPSHOT_NAME;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.TrashOption;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.TrashOption;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
 
 
 /**
 /**
  * Test DistCpProcedure.
  * Test DistCpProcedure.
  */
  */
+@Timeout(180)
 public class TestDistCpProcedure {
 public class TestDistCpProcedure {
   private static MiniDFSCluster cluster;
   private static MiniDFSCluster cluster;
   private static Configuration conf;
   private static Configuration conf;
@@ -77,12 +76,7 @@ public class TestDistCpProcedure {
           new FileEntry(SRCDAT + "/b/c", false)};
           new FileEntry(SRCDAT + "/b/c", false)};
   private static String nnUri;
   private static String nnUri;
 
 
-  @Rule
-  // There are multiple unit tests with different timeouts that fail multiple times because of
-  // DataStreamer#waitAndQueuePacket, so we set a larger global timeout.
-  public Timeout globalTimeout = new Timeout(180000, TimeUnit.MILLISECONDS);
-
-  @BeforeClass
+  @BeforeAll
   public static void beforeClass() throws IOException {
   public static void beforeClass() throws IOException {
     DistCpProcedure.enableForTest();
     DistCpProcedure.enableForTest();
     conf = new Configuration();
     conf = new Configuration();
@@ -98,7 +92,7 @@ public class TestDistCpProcedure {
     nnUri = FileSystem.getDefaultUri(conf).toString();
     nnUri = FileSystem.getDefaultUri(conf).toString();
   }
   }
 
 
-  @AfterClass
+  @AfterAll
   public static void afterClass() {
   public static void afterClass() {
     DistCpProcedure.disableForTest();
     DistCpProcedure.disableForTest();
     if (cluster != null) {
     if (cluster != null) {

+ 2 - 2
hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestFedBalance.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.tools.fedbalance;
 package org.apache.hadoop.tools.fedbalance;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
 
 
-import static junit.framework.TestCase.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
 
 

+ 8 - 8
hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestTrashProcedure.java

@@ -21,9 +21,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
 
 
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutput;
 import java.io.DataOutput;
@@ -34,9 +34,9 @@ import java.io.IOException;
 
 
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.TrashOption;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.TrashOption;
 import static org.apache.hadoop.test.GenericTestUtils.getMethodName;
 import static org.apache.hadoop.test.GenericTestUtils.getMethodName;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 
 /**
 /**
  * Test TrashProcedure.
  * Test TrashProcedure.
@@ -47,7 +47,7 @@ public class TestTrashProcedure {
   private static MiniDFSCluster cluster;
   private static MiniDFSCluster cluster;
   private static String nnUri;
   private static String nnUri;
 
 
-  @BeforeClass
+  @BeforeAll
   public static void beforeClass() throws IOException {
   public static void beforeClass() throws IOException {
     conf = new Configuration();
     conf = new Configuration();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
@@ -55,7 +55,7 @@ public class TestTrashProcedure {
     nnUri = FileSystem.getDefaultUri(conf).toString();
     nnUri = FileSystem.getDefaultUri(conf).toString();
   }
   }
 
 
-  @AfterClass
+  @AfterAll
   public static void afterClass() {
   public static void afterClass() {
     if (cluster != null) {
     if (cluster != null) {
       cluster.shutdown();
       cluster.shutdown();

+ 42 - 27
hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/procedure/TestBalanceProcedureScheduler.java

@@ -26,10 +26,10 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-import org.mockito.Mockito;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.ByteArrayOutputStream;
 import java.io.ByteArrayOutputStream;
@@ -46,12 +46,16 @@ import java.util.concurrent.atomic.AtomicInteger;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
 import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.WORK_THREAD_NUM;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
 
 
 /**
 /**
  * Test BalanceProcedureScheduler.
  * Test BalanceProcedureScheduler.
@@ -63,7 +67,7 @@ public class TestBalanceProcedureScheduler {
   private static DistributedFileSystem fs;
   private static DistributedFileSystem fs;
   private static final int DEFAULT_BLOCK_SIZE = 512;
   private static final int DEFAULT_BLOCK_SIZE = 512;
 
 
-  @BeforeClass
+  @BeforeAll
   public static void setup() throws IOException {
   public static void setup() throws IOException {
     CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
     CONF.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY,
         true);
         true);
@@ -84,7 +88,7 @@ public class TestBalanceProcedureScheduler {
     fs.mkdirs(new Path(workPath));
     fs.mkdirs(new Path(workPath));
   }
   }
 
 
-  @AfterClass
+  @AfterAll
   public static void close() {
   public static void close() {
     if (cluster != null) {
     if (cluster != null) {
       cluster.shutdown();
       cluster.shutdown();
@@ -94,7 +98,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test the scheduler could be shutdown correctly.
    * Test the scheduler could be shutdown correctly.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testShutdownScheduler() throws Exception {
   public void testShutdownScheduler() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -115,7 +120,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test a successful job.
    * Test a successful job.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testSuccessfulJob() throws Exception {
   public void testSuccessfulJob() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -146,16 +152,17 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test a job fails and the error can be got.
    * Test a job fails and the error can be got.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testFailedJob() throws Exception {
   public void testFailedJob() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
     try {
     try {
       // Mock bad procedure.
       // Mock bad procedure.
-      BalanceProcedure badProcedure = Mockito.mock(BalanceProcedure.class);
-      Mockito.doThrow(new IOException("Job failed exception."))
+      BalanceProcedure badProcedure = mock(BalanceProcedure.class);
+      doThrow(new IOException("Job failed exception."))
           .when(badProcedure).execute();
           .when(badProcedure).execute();
-      Mockito.doReturn("bad-procedure").when(badProcedure).name();
+      doReturn("bad-procedure").when(badProcedure).name();
 
 
       BalanceJob.Builder builder = new BalanceJob.Builder<>();
       BalanceJob.Builder builder = new BalanceJob.Builder<>();
       builder.nextProcedure(badProcedure);
       builder.nextProcedure(badProcedure);
@@ -174,7 +181,8 @@ public class TestBalanceProcedureScheduler {
    * the last unfinished procedure, which is the first procedure without
    * the last unfinished procedure, which is the first procedure without
    * journal.
    * journal.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testGetJobAfterRecover() throws Exception {
   public void testGetJobAfterRecover() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -238,7 +246,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test RetryException is handled correctly.
    * Test RetryException is handled correctly.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRetry() throws Exception {
   public void testRetry() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -265,7 +274,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test schedule an empty job.
    * Test schedule an empty job.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testEmptyJob() throws Exception {
   public void testEmptyJob() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -281,7 +291,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test serialization and deserialization of Job.
    * Test serialization and deserialization of Job.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testJobSerializeAndDeserialize() throws Exception {
   public void testJobSerializeAndDeserialize() throws Exception {
     BalanceJob.Builder builder = new BalanceJob.Builder<RecordProcedure>();
     BalanceJob.Builder builder = new BalanceJob.Builder<RecordProcedure>();
     for (int i = 0; i < 5; i++) {
     for (int i = 0; i < 5; i++) {
@@ -305,7 +316,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test scheduler crashes and recovers.
    * Test scheduler crashes and recovers.
    */
    */
-  @Test(timeout = 180000)
+  @Test
+  @Timeout(value = 180)
   public void testSchedulerDownAndRecoverJob() throws Exception {
   public void testSchedulerDownAndRecoverJob() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
@@ -347,7 +359,8 @@ public class TestBalanceProcedureScheduler {
     }
     }
   }
   }
 
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testRecoverJobFromJournal() throws Exception {
   public void testRecoverJobFromJournal() throws Exception {
     BalanceJournal journal =
     BalanceJournal journal =
         ReflectionUtils.newInstance(BalanceJournalInfoHDFS.class, CONF);
         ReflectionUtils.newInstance(BalanceJournalInfoHDFS.class, CONF);
@@ -374,14 +387,15 @@ public class TestBalanceProcedureScheduler {
     }
     }
   }
   }
 
 
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testClearJournalFail() throws Exception {
   public void testClearJournalFail() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);
 
 
-    BalanceJournal journal = Mockito.mock(BalanceJournal.class);
+    BalanceJournal journal = mock(BalanceJournal.class);
     AtomicInteger count = new AtomicInteger(0);
     AtomicInteger count = new AtomicInteger(0);
-    Mockito.doAnswer(invocation -> {
+    doAnswer(invocation -> {
       if (count.incrementAndGet() == 1) {
       if (count.incrementAndGet() == 1) {
         throw new IOException("Mock clear failure");
         throw new IOException("Mock clear failure");
       }
       }
@@ -404,7 +418,8 @@ public class TestBalanceProcedureScheduler {
   /**
   /**
    * Test the job will be recovered if writing journal fails.
    * Test the job will be recovered if writing journal fails.
    */
    */
-  @Test(timeout = 60000)
+  @Test
+  @Timeout(value = 60)
   public void testJobRecoveryWhenWriteJournalFail() throws Exception {
   public void testJobRecoveryWhenWriteJournalFail() throws Exception {
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     BalanceProcedureScheduler scheduler = new BalanceProcedureScheduler(CONF);
     scheduler.init(true);
     scheduler.init(true);