diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml index b54e96d213a01..e167ae4f87c48 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml @@ -173,6 +173,10 @@ junit-vintage-engine test + + org.mockito + mockito-junit-jupiter + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java index 94cd5182a580f..548d1cab30ef8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java @@ -53,15 +53,17 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.event.EventHandler; -import org.junit.AfterClass; -import org.junit.Assert; -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 org.junit.jupiter.api.Timeout; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.assertNotEquals; + public class TestLocalContainerLauncher { private static final Logger LOG = LoggerFactory.getLogger(TestLocalContainerLauncher.class); @@ -75,7 +77,7 @@ private static void delete(File dir) throws IOException { fs.delete(p, true); } - @BeforeClass + @BeforeAll public static void setupTestDirs() throws IOException { testWorkDir = new File("target", TestLocalContainerLauncher.class.getCanonicalName()); @@ -89,7 +91,7 @@ public static void setupTestDirs() throws IOException { } } - @AfterClass + @AfterAll public static void cleanupTestDirs() throws IOException { if (testWorkDir != null) { delete(testWorkDir); @@ -97,7 +99,8 @@ public static void cleanupTestDirs() throws IOException { } @SuppressWarnings("rawtypes") - @Test(timeout=10000) + @Test + @Timeout(value = 10) public void testKillJob() throws Exception { JobConf conf = new JobConf(); AppContext context = mock(AppContext.class); @@ -198,8 +201,8 @@ public void testRenameMapOutputForReduce() throws Exception { final Path mapOut = mrOutputFiles.getOutputFileForWrite(1); conf.set(MRConfig.LOCAL_DIR, localDirs[1].toString()); final Path mapOutIdx = mrOutputFiles.getOutputIndexFileForWrite(1); - Assert.assertNotEquals("Paths must be different!", - mapOut.getParent(), mapOutIdx.getParent()); + assertNotEquals(mapOut.getParent(), mapOutIdx.getParent(), + "Paths must be different!"); // make both dirs part of LOCAL_DIR conf.setStrings(MRConfig.LOCAL_DIR, localDirs); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java index 49b986e225973..7389aebbd30ec 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java @@ -37,8 +37,8 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Test; -import static org.junit.Assert.assertTrue; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -87,7 +87,7 @@ public void testFinshingAttemptTimeout() } taskAttemptFinishingMonitor.stop(); - assertTrue("Finishing attempt didn't time out.", eventHandler.timedOut); + assertTrue(eventHandler.timedOut, "Finishing attempt didn't time out."); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java index b5a7694e4cc6b..819a92ae41ac6 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java @@ -19,19 +19,19 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.junit.jupiter.MockitoExtension; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -69,12 +69,12 @@ import org.apache.hadoop.yarn.util.SystemClock; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; @@ -87,7 +87,7 @@ /** * Tests the behavior of TaskAttemptListenerImpl. */ -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class TestTaskAttemptListenerImpl { private static final String ATTEMPT1_ID = "attempt_123456789012_0001_m_000001_0"; @@ -172,7 +172,7 @@ protected void stopRpcServer() { } } - @After + @AfterEach public void after() throws IOException { if (listener != null) { listener.close(); @@ -180,7 +180,8 @@ public void after() throws IOException { } } - @Test (timeout=5000) + @Test + @Timeout(value = 5) public void testGetTask() throws IOException { configureMocks(); startListener(false); @@ -238,7 +239,8 @@ public void testGetTask() throws IOException { } - @Test (timeout=5000) + @Test + @Timeout(value = 5) public void testJVMId() { JVMId jvmid = new JVMId("test", 1, true, 2); @@ -247,7 +249,8 @@ public void testJVMId() { assertEquals(0, jvmid.compareTo(jvmid1)); } - @Test (timeout=10000) + @Test + @Timeout(value = 10) public void testGetMapCompletionEvents() throws IOException { TaskAttemptCompletionEvent[] empty = {}; TaskAttemptCompletionEvent[] taskEvents = { @@ -257,12 +260,6 @@ public void testGetMapCompletionEvents() throws IOException { createTce(3, false, TaskAttemptCompletionEventStatus.FAILED) }; TaskAttemptCompletionEvent[] mapEvents = { taskEvents[0], taskEvents[2] }; Job mockJob = mock(Job.class); - when(mockJob.getTaskAttemptCompletionEvents(0, 100)) - .thenReturn(taskEvents); - when(mockJob.getTaskAttemptCompletionEvents(0, 2)) - .thenReturn(Arrays.copyOfRange(taskEvents, 0, 2)); - when(mockJob.getTaskAttemptCompletionEvents(2, 100)) - .thenReturn(Arrays.copyOfRange(taskEvents, 2, 4)); when(mockJob.getMapAttemptCompletionEvents(0, 100)).thenReturn( TypeConverter.fromYarn(mapEvents)); when(mockJob.getMapAttemptCompletionEvents(0, 2)).thenReturn( @@ -312,7 +309,8 @@ private static TaskAttemptCompletionEvent createTce(int eventId, return tce; } - @Test (timeout=10000) + @Test + @Timeout(value = 10) public void testCommitWindow() throws IOException { SystemClock clock = SystemClock.getInstance(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java index 8ad62065fa1c0..7730dc6c0615b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java @@ -21,10 +21,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ClusterStorageCapacityExceededException; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests the behavior of YarnChild. @@ -36,7 +41,7 @@ public class TestYarnChild { final static private String KILL_LIMIT_EXCEED_CONF_NAME = "mapreduce.job.dfs.storage.capacity.kill-limit-exceed"; - @Before + @BeforeEach public void setUp() throws Exception { task = mock(Task.class); umbilical = mock(TaskUmbilicalProtocol.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java index 08896b7b2cc6a..1819a64199960 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java @@ -19,8 +19,8 @@ package org.apache.hadoop.mapreduce.jobhistory; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -40,7 +40,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; public class TestEvents { @@ -50,7 +51,8 @@ public class TestEvents { * * @throws Exception */ - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTaskAttemptFinishedEvent() throws Exception { JobID jid = new JobID("001", 1); @@ -79,7 +81,8 @@ public void testTaskAttemptFinishedEvent() throws Exception { * @throws Exception */ - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testJobPriorityChange() throws Exception { org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1); JobPriorityChangeEvent test = new JobPriorityChangeEvent(jid, @@ -89,7 +92,8 @@ public void testJobPriorityChange() throws Exception { } - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testJobQueueChange() throws Exception { org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1); JobQueueChangeEvent test = new JobQueueChangeEvent(jid, @@ -103,7 +107,8 @@ public void testJobQueueChange() throws Exception { * * @throws Exception */ - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testTaskUpdated() throws Exception { JobID jid = new JobID("001", 1); TaskID tid = new TaskID(jid, TaskType.REDUCE, 2); @@ -118,7 +123,8 @@ public void testTaskUpdated() throws Exception { * instance of HistoryEvent Different HistoryEvent should have a different * datum. */ - @Test(timeout = 10000) + @Test + @Timeout(value = 10) public void testEvents() throws Exception { EventReader reader = new EventReader(new DataInputStream( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java index 8159bc2456ca0..9da454c9f92ba 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java @@ -19,9 +19,9 @@ package org.apache.hadoop.mapreduce.jobhistory; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; @@ -81,11 +81,11 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.timeline.TimelineStore; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +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 org.mockito.Mockito; import com.fasterxml.jackson.databind.JsonNode; @@ -101,7 +101,7 @@ public class TestJobHistoryEventHandler { private static MiniDFSCluster dfsCluster = null; private static String coreSitePath; - @BeforeClass + @BeforeAll public static void setUpClass() throws Exception { coreSitePath = "." + File.separator + "target" + File.separator + "test-classes" + File.separator + "core-site.xml"; @@ -109,17 +109,18 @@ public static void setUpClass() throws Exception { dfsCluster = new MiniDFSCluster.Builder(conf).build(); } - @AfterClass + @AfterAll public static void cleanUpClass() throws Exception { dfsCluster.shutdown(); } - @After + @AfterEach public void cleanTest() throws Exception { new File(coreSitePath).delete(); } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testFirstFlushOnCompletionEvent() throws Exception { TestParams t = new TestParams(); Configuration conf = new Configuration(); @@ -162,7 +163,8 @@ public void testFirstFlushOnCompletionEvent() throws Exception { } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testMaxUnflushedCompletionEvents() throws Exception { TestParams t = new TestParams(); Configuration conf = new Configuration(); @@ -207,7 +209,8 @@ public void testMaxUnflushedCompletionEvents() throws Exception { } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testUnflushedTimer() throws Exception { TestParams t = new TestParams(); Configuration conf = new Configuration(); @@ -238,19 +241,20 @@ public void testUnflushedTimer() throws Exception { } handleNextNEvents(jheh, 9); - Assert.assertTrue(jheh.getFlushTimerStatus()); + assertTrue(jheh.getFlushTimerStatus()); verify(mockWriter, times(0)).flush(); Thread.sleep(2 * 4 * 1000l); // 4 seconds should be enough. Just be safe. verify(mockWriter).flush(); - Assert.assertFalse(jheh.getFlushTimerStatus()); + assertFalse(jheh.getFlushTimerStatus()); } finally { jheh.stop(); verify(mockWriter).close(); } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testBatchedFlushJobEndMultiplier() throws Exception { TestParams t = new TestParams(); Configuration conf = new Configuration(); @@ -295,7 +299,8 @@ public void testBatchedFlushJobEndMultiplier() throws Exception { } // In case of all types of events, process Done files if it's last AM retry - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testProcessDoneFilesOnLastAMRetry() throws Exception { TestParams t = new TestParams(true); Configuration conf = new Configuration(); @@ -341,7 +346,8 @@ public void testProcessDoneFilesOnLastAMRetry() throws Exception { } // Skip processing Done files in case of ERROR, if it's not last AM retry - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testProcessDoneFilesNotLastAMRetry() throws Exception { TestParams t = new TestParams(false); Configuration conf = new Configuration(); @@ -421,16 +427,15 @@ public void testPropertyRedactionForJHS() throws Exception { // load the job_conf.xml in JHS directory and verify property redaction. Path jhsJobConfFile = getJobConfInIntermediateDoneDir(conf, params.jobId); - Assert.assertTrue("The job_conf.xml file is not in the JHS directory", - FileContext.getFileContext(conf).util().exists(jhsJobConfFile)); + assertTrue(FileContext.getFileContext(conf).util().exists(jhsJobConfFile), + "The job_conf.xml file is not in the JHS directory"); Configuration jhsJobConf = new Configuration(); try (InputStream input = FileSystem.get(conf).open(jhsJobConfFile)) { jhsJobConf.addResource(input); - Assert.assertEquals( - sensitivePropertyName + " is not redacted in HDFS.", - MRJobConfUtil.REDACTION_REPLACEMENT_VAL, - jhsJobConf.get(sensitivePropertyName)); + assertEquals(MRJobConfUtil.REDACTION_REPLACEMENT_VAL, + jhsJobConf.get(sensitivePropertyName), + sensitivePropertyName + " is not redacted in HDFS."); } } finally { jheh.stop(); @@ -456,7 +461,8 @@ private void purgeHdfsHistoryIntermediateDoneDirectory(Configuration conf) fs.delete(new Path(intermDoneDirPrefix), true); } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testDefaultFsIsUsedForHistory() throws Exception { // Create default configuration pointing to the minicluster Configuration conf = new Configuration(); @@ -490,11 +496,11 @@ public void testDefaultFsIsUsedForHistory() throws Exception { // If we got here then event handler worked but we don't know with which // file system. Now we check that history stuff was written to minicluster FileSystem dfsFileSystem = dfsCluster.getFileSystem(); - assertTrue("Minicluster contains some history files", - dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0); + assertTrue(dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0, + "Minicluster contains some history files"); FileSystem localFileSystem = LocalFileSystem.get(conf); - assertFalse("No history directory on non-default file system", - localFileSystem.exists(new Path(t.dfsWorkDir))); + assertFalse(localFileSystem.exists(new Path(t.dfsWorkDir)), + "No history directory on non-default file system"); } finally { jheh.stop(); purgeHdfsHistoryIntermediateDoneDirectory(conf); @@ -509,7 +515,7 @@ public void testGetHistoryIntermediateDoneDirForUser() throws IOException { "/mapred/history/done_intermediate"); conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name")); String pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf); - Assert.assertEquals("/mapred/history/done_intermediate/" + + assertEquals("/mapred/history/done_intermediate/" + System.getProperty("user.name"), pathStr); // Test fully qualified path @@ -523,13 +529,14 @@ public void testGetHistoryIntermediateDoneDirForUser() throws IOException { conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "file:///"); pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf); - Assert.assertEquals(dfsCluster.getURI().toString() + + assertEquals(dfsCluster.getURI().toString() + "/mapred/history/done_intermediate/" + System.getProperty("user.name"), pathStr); } // test AMStartedEvent for submitTime and startTime - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testAMStartedEvent() throws Exception { TestParams t = new TestParams(); Configuration conf = new Configuration(); @@ -571,7 +578,8 @@ public void testAMStartedEvent() throws Exception { // Have JobHistoryEventHandler handle some events and make sure they get // stored to the Timeline store - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testTimelineEventHandling() throws Exception { TestParams t = new TestParams(RunningAppContext.class, false); Configuration conf = new YarnConfiguration(); @@ -598,14 +606,12 @@ public void testTimelineEventHandling() throws Exception { jheh.getDispatcher().await(); TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); TimelineEntity tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId()); - Assert.assertEquals(1, tEntity.getEvents().size()); - Assert.assertEquals(EventType.AM_STARTED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(currentTime - 10, - tEntity.getEvents().get(0).getTimestamp()); + assertEquals(t.jobId.toString(), tEntity.getEntityId()); + assertEquals(1, tEntity.getEvents().size()); + assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(currentTime - 10, tEntity.getEvents().get(0).getTimestamp()); handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobSubmittedEvent(TypeConverter.fromYarn(t.jobId), "name", @@ -615,18 +621,14 @@ public void testTimelineEventHandling() throws Exception { jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId()); - Assert.assertEquals(2, tEntity.getEvents().size()); - Assert.assertEquals(EventType.JOB_SUBMITTED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(EventType.AM_STARTED.toString(), - tEntity.getEvents().get(1).getEventType()); - Assert.assertEquals(currentTime + 10, - tEntity.getEvents().get(0).getTimestamp()); - Assert.assertEquals(currentTime - 10, - tEntity.getEvents().get(1).getTimestamp()); + assertEquals(t.jobId.toString(), tEntity.getEntityId()); + assertEquals(2, tEntity.getEvents().size()); + assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(1).getEventType()); + assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp()); + assertEquals(currentTime - 10, tEntity.getEvents().get(1).getTimestamp()); handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobQueueChangeEvent(TypeConverter.fromYarn(t.jobId), "q2"), @@ -634,22 +636,17 @@ public void testTimelineEventHandling() throws Exception { jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId()); - Assert.assertEquals(3, tEntity.getEvents().size()); - Assert.assertEquals(EventType.JOB_SUBMITTED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(EventType.AM_STARTED.toString(), - tEntity.getEvents().get(1).getEventType()); - Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), - tEntity.getEvents().get(2).getEventType()); - Assert.assertEquals(currentTime + 10, - tEntity.getEvents().get(0).getTimestamp()); - Assert.assertEquals(currentTime - 10, - tEntity.getEvents().get(1).getTimestamp()); - Assert.assertEquals(currentTime - 20, - tEntity.getEvents().get(2).getTimestamp()); + assertEquals(t.jobId.toString(), tEntity.getEntityId()); + assertEquals(3, tEntity.getEvents().size()); + assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(1).getEventType()); + assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), + tEntity.getEvents().get(2).getEventType()); + assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp()); + assertEquals(currentTime - 10, tEntity.getEvents().get(1).getTimestamp()); + assertEquals(currentTime - 20, tEntity.getEvents().get(2).getTimestamp()); handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0, @@ -657,26 +654,19 @@ public void testTimelineEventHandling() throws Exception { jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId()); - Assert.assertEquals(4, tEntity.getEvents().size()); - Assert.assertEquals(EventType.JOB_SUBMITTED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(EventType.JOB_FINISHED.toString(), - tEntity.getEvents().get(1).getEventType()); - Assert.assertEquals(EventType.AM_STARTED.toString(), - tEntity.getEvents().get(2).getEventType()); - Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), - tEntity.getEvents().get(3).getEventType()); - Assert.assertEquals(currentTime + 10, - tEntity.getEvents().get(0).getTimestamp()); - Assert.assertEquals(currentTime, - tEntity.getEvents().get(1).getTimestamp()); - Assert.assertEquals(currentTime - 10, - tEntity.getEvents().get(2).getTimestamp()); - Assert.assertEquals(currentTime - 20, - tEntity.getEvents().get(3).getTimestamp()); + assertEquals(t.jobId.toString(), tEntity.getEntityId()); + assertEquals(4, tEntity.getEvents().size()); + assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(EventType.JOB_FINISHED.toString(), tEntity.getEvents().get(1).getEventType()); + assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(2).getEventType()); + assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), + tEntity.getEvents().get(3).getEventType()); + assertEquals(currentTime + 10, tEntity.getEvents().get(0).getTimestamp()); + assertEquals(currentTime, tEntity.getEvents().get(1).getTimestamp()); + assertEquals(currentTime - 10, tEntity.getEvents().get(2).getTimestamp()); + assertEquals(currentTime - 20, tEntity.getEvents().get(3).getTimestamp()); handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), @@ -685,64 +675,54 @@ public void testTimelineEventHandling() throws Exception { jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_JOB", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId()); - Assert.assertEquals(5, tEntity.getEvents().size()); - Assert.assertEquals(EventType.JOB_KILLED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(EventType.JOB_SUBMITTED.toString(), - tEntity.getEvents().get(1).getEventType()); - Assert.assertEquals(EventType.JOB_FINISHED.toString(), - tEntity.getEvents().get(2).getEventType()); - Assert.assertEquals(EventType.AM_STARTED.toString(), - tEntity.getEvents().get(3).getEventType()); - Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), - tEntity.getEvents().get(4).getEventType()); - Assert.assertEquals(currentTime + 20, - tEntity.getEvents().get(0).getTimestamp()); - Assert.assertEquals(currentTime + 10, - tEntity.getEvents().get(1).getTimestamp()); - Assert.assertEquals(currentTime, - tEntity.getEvents().get(2).getTimestamp()); - Assert.assertEquals(currentTime - 10, - tEntity.getEvents().get(3).getTimestamp()); - Assert.assertEquals(currentTime - 20, - tEntity.getEvents().get(4).getTimestamp()); + assertEquals(t.jobId.toString(), tEntity.getEntityId()); + assertEquals(5, tEntity.getEvents().size()); + assertEquals(EventType.JOB_KILLED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(EventType.JOB_SUBMITTED.toString(), tEntity.getEvents().get(1).getEventType()); + assertEquals(EventType.JOB_FINISHED.toString(), tEntity.getEvents().get(2).getEventType()); + assertEquals(EventType.AM_STARTED.toString(), tEntity.getEvents().get(3).getEventType()); + assertEquals(EventType.JOB_QUEUE_CHANGED.toString(), + tEntity.getEvents().get(4).getEventType()); + assertEquals(currentTime + 20, tEntity.getEvents().get(0).getTimestamp()); + assertEquals(currentTime + 10, tEntity.getEvents().get(1).getTimestamp()); + assertEquals(currentTime, tEntity.getEvents().get(2).getTimestamp()); + assertEquals(currentTime - 10, tEntity.getEvents().get(3).getTimestamp()); + assertEquals(currentTime - 20, tEntity.getEvents().get(4).getTimestamp()); handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskStartedEvent(t.taskID, 0, TaskType.MAP, ""))); jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_TASK", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId()); - Assert.assertEquals(1, tEntity.getEvents().size()); - Assert.assertEquals(EventType.TASK_STARTED.toString(), - tEntity.getEvents().get(0).getEventType()); - Assert.assertEquals(TaskType.MAP.toString(), - tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE")); + assertEquals(t.taskID.toString(), tEntity.getEntityId()); + assertEquals(1, tEntity.getEvents().size()); + assertEquals(EventType.TASK_STARTED.toString(), tEntity.getEvents().get(0).getEventType()); + assertEquals(TaskType.MAP.toString(), + tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE")); handleEvent(jheh, new JobHistoryEvent(t.jobId, new TaskStartedEvent(t.taskID, 0, TaskType.REDUCE, ""))); jheh.getDispatcher().await(); entities = ts.getEntities("MAPREDUCE_TASK", null, null, null, null, null, null, null, null, null); - Assert.assertEquals(1, entities.getEntities().size()); + assertEquals(1, entities.getEntities().size()); tEntity = entities.getEntities().get(0); - Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId()); - Assert.assertEquals(2, tEntity.getEvents().size()); - Assert.assertEquals(EventType.TASK_STARTED.toString(), - tEntity.getEvents().get(1).getEventType()); - Assert.assertEquals(TaskType.REDUCE.toString(), - tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE")); - Assert.assertEquals(TaskType.MAP.toString(), - tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE")); + assertEquals(t.taskID.toString(), tEntity.getEntityId()); + assertEquals(2, tEntity.getEvents().size()); + assertEquals(EventType.TASK_STARTED.toString(), tEntity.getEvents().get(1).getEventType()); + assertEquals(TaskType.REDUCE.toString(), + tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE")); + assertEquals(TaskType.MAP.toString(), + tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE")); } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testCountersToJSON() throws Exception { JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0); Counters counters = new Counters(); @@ -775,30 +755,31 @@ public void testCountersToJSON() throws Exception { + "{\"NAME\":\"MATT_SMITH\",\"DISPLAY_NAME\":\"Matt Smith\",\"VALUE\":" + "11},{\"NAME\":\"PETER_CAPALDI\",\"DISPLAY_NAME\":\"Peter Capaldi\"," + "\"VALUE\":12}]}]"; - Assert.assertEquals(expected, jsonStr); + assertEquals(expected, jsonStr); } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testCountersToJSONEmpty() throws Exception { JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0); Counters counters = null; JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters); String jsonStr = new ObjectMapper().writeValueAsString(jsonNode); String expected = "[]"; - Assert.assertEquals(expected, jsonStr); + assertEquals(expected, jsonStr); counters = new Counters(); jsonNode = JobHistoryEventUtils.countersToJSON(counters); jsonStr = new ObjectMapper().writeValueAsString(jsonNode); expected = "[]"; - Assert.assertEquals(expected, jsonStr); + assertEquals(expected, jsonStr); counters.addGroup("DOCTORS", "Incarnations of the Doctor"); jsonNode = JobHistoryEventUtils.countersToJSON(counters); jsonStr = new ObjectMapper().writeValueAsString(jsonNode); expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the " + "Doctor\",\"COUNTERS\":[]}]"; - Assert.assertEquals(expected, jsonStr); + assertEquals(expected, jsonStr); } private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) { @@ -912,8 +893,8 @@ public void testSigTermedFunctionality() throws IOException { } jheh.stop(); //Make sure events were handled - assertTrue("handleEvent should've been called only 4 times but was " - + jheh.eventsHandled, jheh.eventsHandled == 4); + assertTrue(jheh.eventsHandled == 4, "handleEvent should've been called only 4 times but was " + + jheh.eventsHandled); //Create a new jheh because the last stop closed the eventWriter etc. jheh = new JHEventHandlerForSigtermTest(mockedContext, 0); @@ -934,14 +915,14 @@ public void testSigTermedFunctionality() throws IOException { } jheh.stop(); //Make sure events were handled, 4 + 1 finish event - assertTrue("handleEvent should've been called only 5 times but was " - + jheh.eventsHandled, jheh.eventsHandled == 5); - assertTrue("Last event handled wasn't JobUnsuccessfulCompletionEvent", - jheh.lastEventHandled.getHistoryEvent() - instanceof JobUnsuccessfulCompletionEvent); + assertTrue(jheh.eventsHandled == 5, "handleEvent should've been called only 5 times but was " + + jheh.eventsHandled); + assertTrue(jheh.lastEventHandled.getHistoryEvent() instanceof JobUnsuccessfulCompletionEvent, + "Last event handled wasn't JobUnsuccessfulCompletionEvent"); } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testSetTrackingURLAfterHistoryIsWritten() throws Exception { TestParams t = new TestParams(true); Configuration conf = new Configuration(); @@ -972,7 +953,8 @@ public void testSetTrackingURLAfterHistoryIsWritten() throws Exception { } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception { TestParams t = new TestParams(true); Configuration conf = new Configuration(); @@ -1003,7 +985,8 @@ public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception { jheh.stop(); } } - @Test (timeout=50000) + @Test + @Timeout(value = 50) public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception { TestParams t = new TestParams(true); Configuration conf = new Configuration(); @@ -1039,7 +1022,8 @@ public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception { } } - @Test(timeout = 50000) + @Test + @Timeout(value = 50) public void testJobHistoryFilePermissions() throws Exception { TestParams t = new TestParams(true); Configuration conf = new Configuration(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java index b81f716ebc79a..9ee402b5db7a2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java @@ -19,12 +19,13 @@ package org.apache.hadoop.mapreduce.jobhistory; import org.apache.hadoop.mapreduce.v2.api.records.JobId; -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 org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -34,7 +35,7 @@ public class TestJobSummary { LoggerFactory.getLogger(TestJobSummary.class); private JobSummary summary = new JobSummary(); - @Before + @BeforeEach public void before() { JobId mockJobId = mock(JobId.class); when(mockJobId.toString()).thenReturn("testJobId"); @@ -64,8 +65,8 @@ public void testEscapeJobSummary() { summary.setJobName("aa\rbb\ncc\r\ndd"); String out = summary.getJobSummaryString(); LOG.info("summary: " + out); - Assert.assertFalse(out.contains("\r")); - Assert.assertFalse(out.contains("\n")); - Assert.assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd")); + assertFalse(out.contains("\r")); + assertFalse(out.contains("\n")); + assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd")); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java index c8d81aea99bd4..4d4be84a74bd7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java @@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.proto.MRProtos; import org.apache.hadoop.yarn.util.Records; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestTaskAttemptReport { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java index a9b34eea7cf56..bc25ac4e9cd4e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java @@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.proto.MRProtos; import org.apache.hadoop.yarn.util.Records; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; public class TestTaskReport { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java index 4be80c44a3e05..214c5fe4f6d64 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java @@ -98,10 +98,11 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Mock MRAppMaster. Doesn't start RPC servers. @@ -326,8 +327,8 @@ public void waitForInternalState(JobImpl job, iState = job.getInternalState(); } LOG.info("Job {} Internal State is : {}", job.getID(), iState); - Assert.assertEquals("Task Internal state is not correct (timedout)", - finalState, iState); + assertEquals( + finalState, iState, "Task Internal state is not correct (timeout)"); } public void waitForInternalState(TaskImpl task, @@ -339,8 +340,8 @@ public void waitForInternalState(TaskImpl task, iState = task.getInternalState(); } LOG.info("Task {} Internal State is : {}", task.getID(), iState); - Assert.assertEquals("Task Internal state is not correct (timedout)", - finalState, iState); + assertEquals( + finalState, iState, "Task Internal state is not correct (timeout)"); } public void waitForInternalState(TaskAttemptImpl attempt, @@ -352,8 +353,8 @@ public void waitForInternalState(TaskAttemptImpl attempt, iState = attempt.getInternalState(); } LOG.info("TaskAttempt {} Internal State is : {}", attempt.getID(), iState); - Assert.assertEquals("TaskAttempt Internal state is not correct (timedout)", - finalState, iState); + assertEquals(finalState, iState, + "TaskAttempt Internal state is not correct (timeout)"); } public void waitForState(TaskAttempt attempt, @@ -367,9 +368,8 @@ public void waitForState(TaskAttempt attempt, } LOG.info("TaskAttempt {} State is : {}", attempt.getID(), report.getTaskAttemptState()); - Assert.assertEquals("TaskAttempt state is not correct (timedout)", - finalState, - report.getTaskAttemptState()); + assertEquals(finalState, + report.getTaskAttemptState(), "TaskAttempt state is not correct (timeout)"); } public void waitForState(Task task, TaskState finalState) throws Exception { @@ -381,8 +381,8 @@ public void waitForState(Task task, TaskState finalState) throws Exception { report = task.getReport(); } LOG.info("Task {} State is : {}", task.getID(), report.getTaskState()); - Assert.assertEquals("Task state is not correct (timedout)", finalState, - report.getTaskState()); + assertEquals(finalState, + report.getTaskState(), "Task state is not correct (timeout)"); } public void waitForState(Job job, JobState finalState) throws Exception { @@ -394,14 +394,14 @@ public void waitForState(Job job, JobState finalState) throws Exception { Thread.sleep(WAIT_FOR_STATE_INTERVAL); } LOG.info("Job {} State is : {}", job.getID(), report.getJobState()); - Assert.assertEquals("Job state is not correct (timedout)", finalState, - job.getState()); + assertEquals(finalState, + job.getState(), "Job state is not correct (timeout)"); } public void waitForState(Service.STATE finalState) throws Exception { if (finalState == Service.STATE.STOPPED) { - Assert.assertTrue("Timeout while waiting for MRApp to stop", - waitForServiceToStop(20 * 1000)); + assertTrue(waitForServiceToStop(20 * 1000), + "Timeout while waiting for MRApp to stop"); } else { int timeoutSecs = 0; while (!finalState.equals(getServiceState()) @@ -409,8 +409,8 @@ public void waitForState(Service.STATE finalState) throws Exception { Thread.sleep(WAIT_FOR_STATE_INTERVAL); } LOG.info("MRApp State is : {}", getServiceState()); - Assert.assertEquals("MRApp state is not correct (timedout)", finalState, - getServiceState()); + assertEquals(finalState, getServiceState(), + "MRApp state is not correct (timeout)"); } } @@ -418,23 +418,23 @@ public void verifyCompleted() { for (Job job : getContext().getAllJobs().values()) { JobReport jobReport = job.getReport(); LOG.info("Job start time :{}", jobReport.getStartTime()); - LOG.info("Job finish time :", jobReport.getFinishTime()); - Assert.assertTrue("Job start time is not less than finish time", - jobReport.getStartTime() <= jobReport.getFinishTime()); - Assert.assertTrue("Job finish time is in future", - jobReport.getFinishTime() <= System.currentTimeMillis()); + LOG.info("Job finish time :{}", jobReport.getFinishTime()); + assertTrue(jobReport.getStartTime() <= jobReport.getFinishTime(), + "Job start time is not less than finish time"); + assertTrue(jobReport.getFinishTime() <= System.currentTimeMillis(), + "Job finish time is in future"); for (Task task : job.getTasks().values()) { TaskReport taskReport = task.getReport(); LOG.info("Task {} start time : {}", task.getID(), taskReport.getStartTime()); LOG.info("Task {} finish time : {}", task.getID(), taskReport.getFinishTime()); - Assert.assertTrue("Task start time is not less than finish time", - taskReport.getStartTime() <= taskReport.getFinishTime()); + assertTrue(taskReport.getStartTime() <= taskReport.getFinishTime(), + "Task start time is not less than finish time"); for (TaskAttempt attempt : task.getAttempts().values()) { TaskAttemptReport attemptReport = attempt.getReport(); - Assert.assertTrue("Attempt start time is not less than finish time", - attemptReport.getStartTime() <= attemptReport.getFinishTime()); + assertTrue(attemptReport.getStartTime() <= attemptReport.getFinishTime(), + "Attempt start time is not less than finish time"); } } } @@ -443,7 +443,7 @@ public void verifyCompleted() { @Override protected Job createJob(Configuration conf, JobStateInternal forcedState, String diagnostic) { - UserGroupInformation currentUser = null; + UserGroupInformation currentUser; try { currentUser = UserGroupInformation.getCurrentUser(); } catch (IOException e) { @@ -455,15 +455,10 @@ protected Job createJob(Configuration conf, JobStateInternal forcedState, getCommitter(), isNewApiCommitter(), currentUser.getUserName(), getContext(), forcedState, diagnostic); - ((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob); + getContext().getAllJobs().put(newJob.getID(), newJob); getDispatcher().register(JobFinishEvent.Type.class, - new EventHandler() { - @Override - public void handle(JobFinishEvent event) { - stop(); - } - }); + (EventHandler) event -> stop()); return newJob; } @@ -507,11 +502,7 @@ public void registerPendingTask(org.apache.hadoop.mapred.Task task, @Override protected EventHandler createJobHistoryHandler( AppContext context) {//disable history - return new EventHandler() { - @Override - public void handle(JobHistoryEvent event) { - } - }; + return event -> {}; } @Override @@ -790,7 +781,7 @@ public static ContainerId newContainerId(int appId, int appAttemptId, public static ContainerTokenIdentifier newContainerTokenIdentifier( Token containerToken) throws IOException { org.apache.hadoop.security.token.Token token = - new org.apache.hadoop.security.token.Token( + new org.apache.hadoop.security.token.Token<>( containerToken.getIdentifier() .array(), containerToken.getPassword().array(), new Text( containerToken.getKind()), diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java index efe150fad19fb..34f4c8c7164cf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java @@ -56,7 +56,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.util.Records; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.event.Level; public class MRAppBenchmark { @@ -150,8 +151,8 @@ public void run() { .getApplicationAttemptId(), containerCount++); //System.out.println("Allocating " + containerCount); - - Container container = + + Container container = recordFactory.newRecordInstance(Container.class); container.setId(cId); NodeId nodeId = NodeId.newInstance("dummy", 1234); @@ -196,7 +197,8 @@ public void runOnNextHeartbeat(Runnable callback) { } } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void benchmark1() throws Exception { int maps = 100; // Adjust for benchmarking. Start with thousands. int reduces = 0; @@ -275,7 +277,8 @@ public AllocateResponse allocate(AllocateRequest request) }); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60) public void benchmark2() throws Exception { int maps = 100; // Adjust for benchmarking, start with a couple of thousands int reduces = 50; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java index 4b9015f10c5da..c86ea72690459 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java @@ -21,8 +21,6 @@ import java.util.Iterator; import java.util.List; -import org.junit.Assert; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.AMInfo; @@ -33,7 +31,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Task; import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestAMInfos { @@ -50,7 +50,7 @@ public void testAMInfosWithoutRecoveryEnabled() throws Exception { long am1StartTime = app.getAllAMInfos().get(0).getStartTime(); - Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); app.waitForState(mapTask, TaskState.RUNNING); @@ -71,14 +71,14 @@ public void testAMInfosWithoutRecoveryEnabled() throws Exception { conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask = it.next(); // There should be two AMInfos List amInfos = app.getAllAMInfos(); - Assert.assertEquals(2, amInfos.size()); + assertEquals(2, amInfos.size()); AMInfo amInfoOne = amInfos.get(0); - Assert.assertEquals(am1StartTime, amInfoOne.getStartTime()); + assertEquals(am1StartTime, amInfoOne.getStartTime()); app.stop(); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java index 59778161f20d7..6598bd0c05902 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java @@ -22,8 +22,9 @@ import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.util.resource.Resources; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.Collections; @@ -58,8 +59,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestCheckpointPreemptionPolicy { @@ -77,7 +78,7 @@ public class TestCheckpointPreemptionPolicy { private int minAlloc = 1024; - @Before + @BeforeEach @SuppressWarnings("rawtypes") // mocked generics public void setup() { ApplicationId appId = ApplicationId.newInstance(200, 1); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java index 3b5cfe221eded..bc891726cc7f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java @@ -24,7 +24,6 @@ import java.util.Map; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.TaskAttemptListenerImpl; @@ -48,7 +47,9 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Tests the state machine with respect to Job/Task/TaskAttempt failure @@ -68,20 +69,19 @@ public void testFailTask() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.SUCCEEDED); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + assertEquals(1, tasks.size(), "Num tasks is not correct"); Task task = tasks.values().iterator().next(); - Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, - task.getReport().getTaskState()); + assertEquals(TaskState.SUCCEEDED, task.getReport().getTaskState(), "Task state not correct"); Map attempts = tasks.values().iterator().next().getAttempts(); - Assert.assertEquals("Num attempts is not correct", 2, attempts.size()); + assertEquals(2, attempts.size(), "Num attempts is not correct"); //one attempt must be failed //and another must have succeeded Iterator it = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED, - it.next().getReport().getTaskAttemptState()); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, - it.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.FAILED, it.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); + assertEquals(TaskAttemptState.SUCCEEDED, it.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); } @Test @@ -159,17 +159,15 @@ public void testTimedOutTask() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.FAILED); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + assertEquals(1, tasks.size(), "Num tasks is not correct"); Task task = tasks.values().iterator().next(); - Assert.assertEquals("Task state not correct", TaskState.FAILED, - task.getReport().getTaskState()); + assertEquals(TaskState.FAILED, task.getReport().getTaskState(), "Task state not correct"); Map attempts = tasks.values().iterator().next().getAttempts(); - Assert.assertEquals("Num attempts is not correct", maxAttempts, - attempts.size()); + assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct"); for (TaskAttempt attempt : attempts.values()) { - Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED, - attempt.getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.FAILED, attempt.getReport().getTaskAttemptState(), + "Attempt state not correct"); } } @@ -185,13 +183,12 @@ public void testTaskFailWithUnusedContainer() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + assertEquals(1, tasks.size(), "Num tasks is not correct"); Task task = tasks.values().iterator().next(); app.waitForState(task, TaskState.SCHEDULED); Map attempts = tasks.values().iterator() .next().getAttempts(); - Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts - .size()); + assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct"); TaskAttempt attempt = attempts.values().iterator().next(); app.waitForInternalState((TaskAttemptImpl) attempt, TaskAttemptStateInternal.ASSIGNED); @@ -204,7 +201,7 @@ public void testTaskFailWithUnusedContainer() throws Exception { static class MRAppWithFailingTaskAndUnusedContainer extends MRApp { public MRAppWithFailingTaskAndUnusedContainer() { - super(1, 0, false, "TaskFailWithUnsedContainer", true); + super(1, 0, false, "TaskFailWithUnusedContainer", true); } @Override @@ -241,7 +238,7 @@ public ContainerManagementProtocolProxyData getCMProxy( return null; } }; - }; + } } static class TimeOutTaskMRApp extends MRApp { @@ -258,17 +255,17 @@ protected TaskAttemptListener createTaskAttemptListener( //leading to Attempt failure return new TaskAttemptListenerImpl(getContext(), null, null, policy) { @Override - public void startRpcServer(){}; + public void startRpcServer(){} @Override - public void stopRpcServer(){}; + public void stopRpcServer(){} @Override public InetSocketAddress getAddress() { return NetUtils.createSocketAddr("localhost", 1234); } protected void serviceInit(Configuration conf) throws Exception { - conf.setInt(MRJobConfig.TASK_TIMEOUT, 1*1000);//reduce timeout - conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1*1000); + conf.setInt(MRJobConfig.TASK_TIMEOUT, 1000); //reduce timeout + conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1000); conf.setDouble(MRJobConfig.TASK_LOG_PROGRESS_DELTA_THRESHOLD, 0.01); super.serviceInit(conf); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java index d2bd0104fff6f..b8ab1e14699d0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java @@ -19,7 +19,8 @@ package org.apache.hadoop.mapreduce.v2.app; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import java.util.ArrayList; import java.util.Arrays; @@ -50,8 +51,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.event.EventHandler; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFetchFailure { @@ -65,8 +65,7 @@ public void testFetchFailure() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("Num tasks not correct", - 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -97,10 +96,9 @@ public Boolean get() { TaskAttemptCompletionEvent[] events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", - 1, events.length); - Assert.assertEquals("Event status not correct", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus()); + assertEquals(1, events.length, "Num completion events not correct"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(), + "Event status not correct"); // wait for reduce to start running app.waitForState(reduceTask, TaskState.RUNNING); @@ -117,11 +115,11 @@ public Boolean get() { app.waitForState(mapTask, TaskState.RUNNING); //map attempt must have become FAILED - Assert.assertEquals("Map TaskAttempt state not correct", - TaskAttemptState.FAILED, mapAttempt1.getState()); + assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(), + "Map TaskAttempt state not correct"); - Assert.assertEquals("Num attempts in Map Task not correct", - 2, mapTask.getAttempts().size()); + assertEquals(2, mapTask.getAttempts().size(), + "Num attempts in Map Task not correct"); Iterator atIt = mapTask.getAttempts().values().iterator(); atIt.next(); @@ -144,39 +142,37 @@ public Boolean get() { app.waitForState(job, JobState.SUCCEEDED); //previous completion event now becomes obsolete - Assert.assertEquals("Event status not correct", - TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus()); + assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(), + "Event status not correct"); events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", - 4, events.length); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt1.getID(), events[0].getAttemptId()); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt1.getID(), events[1].getAttemptId()); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt2.getID(), events[2].getAttemptId()); - Assert.assertEquals("Event redude attempt id not correct", - reduceAttempt.getID(), events[3].getAttemptId()); - Assert.assertEquals("Event status not correct for map attempt1", - TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus()); - Assert.assertEquals("Event status not correct for map attempt1", - TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus()); - Assert.assertEquals("Event status not correct for map attempt2", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus()); - Assert.assertEquals("Event status not correct for reduce attempt1", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus()); + assertEquals(4, events.length, "Num completion events not correct"); + assertEquals(mapAttempt1.getID(), events[0].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(mapAttempt1.getID(), events[1].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(mapAttempt2.getID(), events[2].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(reduceAttempt.getID(), events[3].getAttemptId(), + "Event reduce attempt id not correct"); + assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(), + "Event status not correct for map attempt1"); + assertEquals(TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus(), + "Event status not correct for map attempt1"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus(), + "Event status not correct for map attempt2"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus(), + "Event status not correct for reduce attempt1"); TaskCompletionEvent mapEvents[] = job.getMapAttemptCompletionEvents(0, 2); TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events); - Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length); - Assert.assertArrayEquals("Unexpected map events", - Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents); + assertEquals(2, mapEvents.length, "Incorrect number of map events"); + assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents, + "Unexpected map events"); mapEvents = job.getMapAttemptCompletionEvents(2, 200); - Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length); - Assert.assertEquals("Unexpected map event", convertedEvents[2], - mapEvents[0]); + assertEquals(1, mapEvents.length, "Incorrect number of map events"); + assertEquals(convertedEvents[2], mapEvents[0], "Unexpected map event"); } /** @@ -197,8 +193,7 @@ public void testFetchFailureWithRecovery() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("Num tasks not correct", - 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -218,10 +213,9 @@ public void testFetchFailureWithRecovery() throws Exception { TaskAttemptCompletionEvent[] events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", - 1, events.length); - Assert.assertEquals("Event status not correct", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus()); + assertEquals(1, events.length, "Num completion events not correct"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(), + "Event status not correct"); // wait for reduce to start running app.waitForState(reduceTask, TaskState.RUNNING); @@ -250,8 +244,7 @@ public void testFetchFailureWithRecovery() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("Num tasks not correct", - 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); it = job.getTasks().values().iterator(); mapTask = it.next(); reduceTask = it.next(); @@ -277,7 +270,7 @@ public void testFetchFailureWithRecovery() throws Exception { app.waitForState(job, JobState.SUCCEEDED); events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", 2, events.length); + assertEquals(2, events.length, "Num completion events not correct"); } @Test @@ -290,8 +283,7 @@ public void testFetchFailureMultipleReduces() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("Num tasks not correct", - 4, job.getTasks().size()); + assertEquals(4, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -313,10 +305,10 @@ public void testFetchFailureMultipleReduces() throws Exception { TaskAttemptCompletionEvent[] events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", - 1, events.length); - Assert.assertEquals("Event status not correct", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus()); + assertEquals(1, events.length, + "Num completion events not correct"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(), + "Event status not correct"); // wait for reduce to start running app.waitForState(reduceTask, TaskState.RUNNING); @@ -354,16 +346,16 @@ public void testFetchFailureMultipleReduces() throws Exception { app.waitForState(mapTask, TaskState.RUNNING); //map attempt must have become FAILED - Assert.assertEquals("Map TaskAttempt state not correct", - TaskAttemptState.FAILED, mapAttempt1.getState()); + assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(), + "Map TaskAttempt state not correct"); assertThat(mapAttempt1.getDiagnostics().get(0)) .isEqualTo("Too many fetch failures. Failing the attempt. " + "Last failure reported by " + reduceAttempt3.getID().toString() + " from host host3"); - Assert.assertEquals("Num attempts in Map Task not correct", - 2, mapTask.getAttempts().size()); + assertEquals(2, mapTask.getAttempts().size(), + "Num attempts in Map Task not correct"); Iterator atIt = mapTask.getAttempts().values().iterator(); atIt.next(); @@ -396,39 +388,37 @@ public void testFetchFailureMultipleReduces() throws Exception { app.waitForState(job, JobState.SUCCEEDED); //previous completion event now becomes obsolete - Assert.assertEquals("Event status not correct", - TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus()); + assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(), + "Event status not correct"); events = job.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Num completion events not correct", - 6, events.length); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt1.getID(), events[0].getAttemptId()); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt1.getID(), events[1].getAttemptId()); - Assert.assertEquals("Event map attempt id not correct", - mapAttempt2.getID(), events[2].getAttemptId()); - Assert.assertEquals("Event reduce attempt id not correct", - reduceAttempt.getID(), events[3].getAttemptId()); - Assert.assertEquals("Event status not correct for map attempt1", - TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus()); - Assert.assertEquals("Event status not correct for map attempt1", - TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus()); - Assert.assertEquals("Event status not correct for map attempt2", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus()); - Assert.assertEquals("Event status not correct for reduce attempt1", - TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus()); + assertEquals(6, events.length, "Num completion events not correct"); + assertEquals(mapAttempt1.getID(), events[0].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(mapAttempt1.getID(), events[1].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(mapAttempt2.getID(), events[2].getAttemptId(), + "Event map attempt id not correct"); + assertEquals(reduceAttempt.getID(), events[3].getAttemptId(), + "Event reduce attempt id not correct"); + assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(), + "Event status not correct for map attempt1"); + assertEquals(TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus(), + "Event status not correct for map attempt1"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus(), + "Event status not correct for map attempt2"); + assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus(), + "Event status not correct for reduce attempt1"); TaskCompletionEvent mapEvents[] = job.getMapAttemptCompletionEvents(0, 2); TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events); - Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length); - Assert.assertArrayEquals("Unexpected map events", - Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents); + assertEquals(2, mapEvents.length, "Incorrect number of map events"); + assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents, + "Unexpected map events"); mapEvents = job.getMapAttemptCompletionEvents(2, 200); - Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length); - Assert.assertEquals("Unexpected map event", convertedEvents[2], - mapEvents[0]); + assertEquals(1, mapEvents.length, "Incorrect number of map events"); + assertEquals(convertedEvents[2], mapEvents[0], "Unexpected map event"); } private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) { @@ -452,7 +442,7 @@ private void sendFetchFailure(MRApp app, TaskAttempt reduceAttempt, TaskAttempt mapAttempt, String hostname) { app.getContext().getEventHandler().handle( new JobTaskAttemptFetchFailureEvent( - reduceAttempt.getID(), + reduceAttempt.getID(), Arrays.asList(new TaskAttemptId[] {mapAttempt.getID()}), hostname)); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java index 1cd625551a620..b2af290d9bb4e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java @@ -18,6 +18,10 @@ package org.apache.hadoop.mapreduce.v2.app; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -59,8 +63,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; /** * Tests job end notification @@ -74,18 +77,16 @@ private void testNumRetries(Configuration conf) { conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "0"); conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "10"); setConf(conf); - Assert.assertTrue("Expected numTries to be 0, but was " + numTries, - numTries == 0 ); + assertEquals(0, numTries, "Expected numTries to be 0, but was " + numTries); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "1"); setConf(conf); - Assert.assertTrue("Expected numTries to be 1, but was " + numTries, - numTries == 1 ); + assertEquals(1, numTries, "Expected numTries to be 1, but was " + numTries); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "20"); setConf(conf); - Assert.assertTrue("Expected numTries to be 11, but was " + numTries, - numTries == 11 ); //11 because number of _retries_ is 10 + assertEquals(11, numTries, + "Expected numTries to be 11, but was " + numTries); //11 because number of _retries_ is 10 } //Test maximum retry interval is capped by @@ -94,54 +95,49 @@ private void testWaitInterval(Configuration conf) { conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_RETRY_INTERVAL, "5000"); conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "1000"); setConf(conf); - Assert.assertTrue("Expected waitInterval to be 1000, but was " - + waitInterval, waitInterval == 1000); + assertEquals(1000, waitInterval, "Expected waitInterval to be 1000, but was " + waitInterval); conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "10000"); setConf(conf); - Assert.assertTrue("Expected waitInterval to be 5000, but was " - + waitInterval, waitInterval == 5000); + assertEquals(5000, waitInterval, "Expected waitInterval to be 5000, but was " + waitInterval); //Test negative numbers are set to default conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "-10"); setConf(conf); - Assert.assertTrue("Expected waitInterval to be 5000, but was " - + waitInterval, waitInterval == 5000); + assertEquals(5000, waitInterval, "Expected waitInterval to be 5000, but was " + waitInterval); } private void testTimeout(Configuration conf) { conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_TIMEOUT, "1000"); setConf(conf); - Assert.assertTrue("Expected timeout to be 1000, but was " - + timeout, timeout == 1000); + assertEquals(1000, timeout, "Expected timeout to be 1000, but was " + timeout); } private void testProxyConfiguration(Configuration conf) { conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost"); setConf(conf); - Assert.assertTrue("Proxy shouldn't be set because port wasn't specified", - proxyToUse.type() == Proxy.Type.DIRECT); + assertTrue(proxyToUse.type() == Proxy.Type.DIRECT, + "Proxy shouldn't be set because port wasn't specified"); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:someport"); setConf(conf); - Assert.assertTrue("Proxy shouldn't be set because port wasn't numeric", - proxyToUse.type() == Proxy.Type.DIRECT); + assertTrue(proxyToUse.type() == Proxy.Type.DIRECT, + "Proxy shouldn't be set because port wasn't numeric"); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:1000"); setConf(conf); - Assert.assertEquals("Proxy should have been set but wasn't ", - "HTTP @ somehost:1000", proxyToUse.toString()); + assertEquals("HTTP @ somehost:1000", proxyToUse.toString(), + "Proxy should have been set but wasn't "); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "socks@somehost:1000"); setConf(conf); - Assert.assertEquals("Proxy should have been socks but wasn't ", - "SOCKS @ somehost:1000", proxyToUse.toString()); + assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(), + "Proxy should have been socks but wasn't "); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "SOCKS@somehost:1000"); setConf(conf); - Assert.assertEquals("Proxy should have been socks but wasn't ", - "SOCKS @ somehost:1000", proxyToUse.toString()); + assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(), + "Proxy should have been socks but wasn't "); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "sfafn@somehost:1000"); setConf(conf); - Assert.assertEquals("Proxy should have been http but wasn't ", - "HTTP @ somehost:1000", proxyToUse.toString()); - + assertEquals("HTTP @ somehost:1000", proxyToUse.toString(), + "Proxy should have been http but wasn't "); } /** @@ -181,10 +177,10 @@ public void testNotifyRetries() throws InterruptedException { this.setConf(conf); this.notify(jobReport); long endTime = System.currentTimeMillis(); - Assert.assertEquals("Only 1 try was expected but was : " - + this.notificationCount, 1, this.notificationCount); - Assert.assertTrue("Should have taken more than 5 seconds it took " - + (endTime - startTime), endTime - startTime > 5000); + assertEquals(1, this.notificationCount, "Only 1 try was expected but was : " + + this.notificationCount); + assertTrue(endTime - startTime > 5000, "Should have taken more than 5 seconds it took " + + (endTime - startTime)); conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "3"); conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "3"); @@ -196,10 +192,10 @@ public void testNotifyRetries() throws InterruptedException { this.setConf(conf); this.notify(jobReport); endTime = System.currentTimeMillis(); - Assert.assertEquals("Only 3 retries were expected but was : " - + this.notificationCount, 3, this.notificationCount); - Assert.assertTrue("Should have taken more than 9 seconds it took " - + (endTime - startTime), endTime - startTime > 9000); + assertEquals(3, this.notificationCount, "Only 3 retries were expected but was : " + + this.notificationCount); + assertTrue(endTime - startTime > 9000, "Should have taken more than 9 seconds it took " + + (endTime - startTime)); } @@ -222,12 +218,11 @@ private void testNotificationOnLastRetry(boolean withRuntimeException) doThrow(runtimeException).when(app).stop(); } app.shutDownJob(); - Assert.assertTrue(app.isLastAMRetry()); - Assert.assertEquals(1, JobEndServlet.calledTimes); - Assert.assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED", + assertTrue(app.isLastAMRetry()); + assertEquals(1, JobEndServlet.calledTimes); + assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED", JobEndServlet.requestUri.getQuery()); - Assert.assertEquals(JobState.SUCCEEDED.toString(), - JobEndServlet.foundJobState); + assertEquals(JobState.SUCCEEDED.toString(), JobEndServlet.foundJobState); server.stop(); } @@ -262,10 +257,10 @@ public void testAbsentNotificationOnNotLastRetryUnregistrationFailure() app.shutDownJob(); // Not the last AM attempt. So user should that the job is still running. app.waitForState(job, JobState.RUNNING); - Assert.assertFalse(app.isLastAMRetry()); - Assert.assertEquals(0, JobEndServlet.calledTimes); - Assert.assertNull(JobEndServlet.requestUri); - Assert.assertNull(JobEndServlet.foundJobState); + assertFalse(app.isLastAMRetry()); + assertEquals(0, JobEndServlet.calledTimes); + assertNull(JobEndServlet.requestUri); + assertNull(JobEndServlet.foundJobState); server.stop(); } @@ -294,11 +289,11 @@ public void testNotificationOnLastRetryUnregistrationFailure() // Unregistration fails: isLastAMRetry is recalculated, this is ///reboot will stop service internally, we don't need to shutdown twice app.waitForServiceToStop(10000); - Assert.assertFalse(app.isLastAMRetry()); + assertFalse(app.isLastAMRetry()); // Since it's not last retry, JobEndServlet didn't called - Assert.assertEquals(0, JobEndServlet.calledTimes); - Assert.assertNull(JobEndServlet.requestUri); - Assert.assertNull(JobEndServlet.foundJobState); + assertEquals(0, JobEndServlet.calledTimes); + assertNull(JobEndServlet.requestUri); + assertNull(JobEndServlet.foundJobState); server.stop(); } @@ -321,8 +316,8 @@ public void testCustomNotifierClass() throws InterruptedException { this.notify(jobReport); final URL urlToNotify = CustomNotifier.urlToNotify; - Assert.assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED", - urlToNotify.toString()); + assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED", + urlToNotify.toString()); } public static final class CustomNotifier implements CustomJobEndNotifier { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java index f681cf816502d..9a817622b1674 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java @@ -23,7 +23,6 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.service.Service; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.api.records.JobId; @@ -48,7 +47,9 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Event; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Tests the state machine with respect to Job/Task/TaskAttempt kill scenarios. @@ -66,7 +67,7 @@ public void testKillJob() throws Exception { //blocked Job job = app.submit(new Configuration()); - //wait and vailidate for Job to become RUNNING + //wait and validate for Job to become RUNNING app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING); //send the kill signal to Job @@ -83,18 +84,15 @@ public void testKillJob() throws Exception { app.waitForState(Service.STATE.STOPPED); Map tasks = job.getTasks(); - Assert.assertEquals("No of tasks is not correct", 1, - tasks.size()); + assertEquals(1, tasks.size(), "No of tasks is not correct"); Task task = tasks.values().iterator().next(); - Assert.assertEquals("Task state not correct", TaskState.KILLED, - task.getReport().getTaskState()); + assertEquals(TaskState.KILLED, task.getReport().getTaskState(), "Task state not correct"); Map attempts = tasks.values().iterator().next().getAttempts(); - Assert.assertEquals("No of attempts is not correct", 1, - attempts.size()); + assertEquals(1, attempts.size(), "No of attempts is not correct"); Iterator it = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, - it.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.KILLED, it.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); } @Test @@ -104,11 +102,10 @@ public void testKillTask() throws Exception { //this will start the job but job won't complete as Task is blocked Job job = app.submit(new Configuration()); - //wait and vailidate for Job to become RUNNING + //wait and validate for Job to become RUNNING app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING); Map tasks = job.getTasks(); - Assert.assertEquals("No of tasks is not correct", 2, - tasks.size()); + assertEquals(2, tasks.size(), "No of tasks is not correct"); Iterator it = tasks.values().iterator(); Task task1 = it.next(); Task task2 = it.next(); @@ -126,23 +123,22 @@ public void testKillTask() throws Exception { //first Task is killed and second is Succeeded //Job is succeeded - Assert.assertEquals("Task state not correct", TaskState.KILLED, - task1.getReport().getTaskState()); - Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, - task2.getReport().getTaskState()); + assertEquals(TaskState.KILLED, task1.getReport().getTaskState(), + "Task state not correct"); + assertEquals(TaskState.SUCCEEDED, task2.getReport().getTaskState(), + "Task state not correct"); Map attempts = task1.getAttempts(); - Assert.assertEquals("No of attempts is not correct", 1, - attempts.size()); + assertEquals(1, attempts.size(), + "No of attempts is not correct"); Iterator iter = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, - iter.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.KILLED, iter.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); attempts = task2.getAttempts(); - Assert.assertEquals("No of attempts is not correct", 1, - attempts.size()); + assertEquals(1, attempts.size(), "No of attempts is not correct"); iter = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, - iter.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); } @Test @@ -194,7 +190,7 @@ public Dispatcher createDispatcher() { Job job = app.submit(new Configuration()); JobId jobId = app.getJobId(); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -232,7 +228,7 @@ public Dispatcher createDispatcher() { Job job = app.submit(new Configuration()); JobId jobId = app.getJobId(); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -280,7 +276,7 @@ public Dispatcher createDispatcher() { Job job = app.submit(new Configuration()); JobId jobId = app.getJobId(); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size()); + assertEquals(2, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask = it.next(); Task reduceTask = it.next(); @@ -367,11 +363,10 @@ public void testKillTaskAttempt() throws Exception { //this will start the job but job won't complete as Task is blocked Job job = app.submit(new Configuration()); - //wait and vailidate for Job to become RUNNING + //wait and validate for Job to become RUNNING app.waitForState(job, JobState.RUNNING); Map tasks = job.getTasks(); - Assert.assertEquals("No of tasks is not correct", 2, - tasks.size()); + assertEquals(2, tasks.size(), "No of tasks is not correct"); Iterator it = tasks.values().iterator(); Task task1 = it.next(); Task task2 = it.next(); @@ -394,26 +389,24 @@ public void testKillTaskAttempt() throws Exception { //first Task will have two attempts 1st is killed, 2nd Succeeds //both Tasks and Job succeeds - Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, - task1.getReport().getTaskState()); - Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED, - task2.getReport().getTaskState()); + assertEquals(TaskState.SUCCEEDED, task1.getReport().getTaskState(), + "Task state not correct"); + assertEquals(TaskState.SUCCEEDED, task2.getReport().getTaskState(), + "Task state not correct"); Map attempts = task1.getAttempts(); - Assert.assertEquals("No of attempts is not correct", 2, - attempts.size()); + assertEquals(2, attempts.size(), "No of attempts is not correct"); Iterator iter = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED, - iter.next().getReport().getTaskAttemptState()); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, - iter.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.KILLED, iter.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); + assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); attempts = task2.getAttempts(); - Assert.assertEquals("No of attempts is not correct", 1, - attempts.size()); + assertEquals(1, attempts.size(), "No of attempts is not correct"); iter = attempts.values().iterator(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED, - iter.next().getReport().getTaskAttemptState()); + assertEquals(TaskAttemptState.SUCCEEDED, iter.next().getReport().getTaskAttemptState(), + "Attempt state not correct"); } static class BlockingMRApp extends MRApp { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java index 3c3c4c90625a8..62e016a734b5d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java @@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestKillAMPreemptionPolicy { private final RecordFactory recordFactory = RecordFactoryProvider diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java index 534bcd0940895..da91cbf5acf7d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.v2.app; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -30,7 +32,6 @@ import java.util.function.Supplier; import org.apache.hadoop.test.GenericTestUtils; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -68,8 +69,7 @@ import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.Test; /** * Tests the state machine of MR App. @@ -83,7 +83,7 @@ public void testMapReduce() throws Exception { Job job = app.submit(new Configuration()); app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertEquals(System.getProperty("user.name"),job.getUserName()); + assertEquals(System.getProperty("user.name"), job.getUserName()); } @Test @@ -106,7 +106,7 @@ public void testCommitPending() throws Exception { MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true); Job job = app.submit(new Configuration()); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -151,7 +151,7 @@ public void testCompletedMapsForReduceSlowstart() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("Num tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -170,8 +170,8 @@ public void testCompletedMapsForReduceSlowstart() throws Exception { app.waitForState(task2Attempt, TaskAttemptState.RUNNING); // reduces must be in NEW state - Assert.assertEquals("Reduce Task state not correct", - TaskState.NEW, reduceTask.getReport().getTaskState()); + assertEquals(TaskState.NEW, reduceTask.getReport().getTaskState(), + "Reduce Task state not correct"); //send the done signal to the 1st map task app.getContext().getEventHandler().handle( @@ -210,7 +210,7 @@ public void testUpdatedNodes() throws Exception { int runCount = 0; AsyncDispatcher dispatcher = new AsyncDispatcher(); dispatcher.init(new Configuration()); - Dispatcher disp = Mockito.spy(dispatcher); + Dispatcher disp = spy(dispatcher); MRApp app = new MRAppWithHistory(2, 2, false, this.getClass().getName(), true, ++runCount, disp); Configuration conf = new Configuration(); @@ -224,7 +224,7 @@ public void testUpdatedNodes() throws Exception { final Job job1 = app.submit(conf); app.waitForState(job1, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 4, job1.getTasks().size()); + assertEquals(4, job1.getTasks().size(), "Num tasks not correct"); Iterator it = job1.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -239,7 +239,7 @@ public void testUpdatedNodes() throws Exception { .next(); NodeId node1 = task1Attempt.getNodeId(); NodeId node2 = task2Attempt.getNodeId(); - Assert.assertEquals(node1, node2); + assertEquals(node1, node2); // send the done signal to the task app.getContext() @@ -260,22 +260,19 @@ public void testUpdatedNodes() throws Exception { final int checkIntervalMillis = 100; final int waitForMillis = 800; - waitFor(new Supplier() { - @Override - public Boolean get() { - TaskAttemptCompletionEvent[] events = job1 - .getTaskAttemptCompletionEvents(0, 100); - return events.length == 2; - } + waitFor(() -> { + TaskAttemptCompletionEvent[] events = job1 + .getTaskAttemptCompletionEvents(0, 100); + return events.length == 2; }, checkIntervalMillis, waitForMillis); TaskAttemptCompletionEvent[] events = job1.getTaskAttemptCompletionEvents (0, 100); - Assert.assertEquals("Expecting 2 completion events for success", 2, - events.length); + assertEquals(2, + events.length, "Expecting 2 completion events for success"); // send updated nodes info - ArrayList updatedNodes = new ArrayList(); + ArrayList updatedNodes = new ArrayList<>(); NodeReport nr = RecordFactoryProvider.getRecordFactory(null) .newRecordInstance(NodeReport.class); nr.setNodeId(node1); @@ -287,18 +284,15 @@ public Boolean get() { app.waitForState(task1Attempt, TaskAttemptState.KILLED); app.waitForState(task2Attempt, TaskAttemptState.KILLED); - waitFor(new Supplier() { - @Override - public Boolean get() { - TaskAttemptCompletionEvent[] events = job1 - .getTaskAttemptCompletionEvents(0, 100); - return events.length == 4; - } + waitFor(() -> { + TaskAttemptCompletionEvent[] events1 = job1 + .getTaskAttemptCompletionEvents(0, 100); + return events1.length == 4; }, checkIntervalMillis, waitForMillis); events = job1.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Expecting 2 more completion events for killed", 4, - events.length); + assertEquals(4, + events.length, "Expecting 2 more completion events for killed"); // 2 map task attempts which were killed above should be requested from // container allocator with the previous map task marked as failed. If // this happens allocator will request the container for this mapper from @@ -335,8 +329,8 @@ public Boolean get() { }, checkIntervalMillis, waitForMillis); events = job1.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Expecting 1 more completion events for success", 5, - events.length); + assertEquals(5, + events.length, "Expecting 1 more completion events for success"); // Crash the app again. app.stop(); @@ -351,7 +345,7 @@ public Boolean get() { final Job job2 = app.submit(conf); app.waitForState(job2, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 4, job2.getTasks().size()); + assertEquals(4, job2.getTasks().size(), "No of tasks not correct"); it = job2.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -362,19 +356,15 @@ public Boolean get() { app.waitForState(mapTask1, TaskState.SUCCEEDED); app.waitForState(mapTask2, TaskState.RUNNING); - waitFor(new Supplier() { - @Override - public Boolean get() { - TaskAttemptCompletionEvent[] events = job2 - .getTaskAttemptCompletionEvents(0, 100); - return events.length == 2; - } + waitFor(() -> { + TaskAttemptCompletionEvent[] events12 = job2 + .getTaskAttemptCompletionEvents(0, 100); + return events12.length == 2; }, checkIntervalMillis, waitForMillis); events = job2.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals( - "Expecting 2 completion events for killed & success of map1", 2, - events.length); + assertEquals(2, events.length, + "Expecting 2 completion events for killed & success of map1"); task2Attempt = mapTask2.getAttempts().values().iterator().next(); app.getContext() @@ -384,18 +374,15 @@ public Boolean get() { TaskAttemptEventType.TA_DONE)); app.waitForState(mapTask2, TaskState.SUCCEEDED); - waitFor(new Supplier() { - @Override - public Boolean get() { - TaskAttemptCompletionEvent[] events = job2 - .getTaskAttemptCompletionEvents(0, 100); - return events.length == 3; - } + waitFor(() -> { + TaskAttemptCompletionEvent[] events13 = job2 + .getTaskAttemptCompletionEvents(0, 100); + return events13.length == 3; }, checkIntervalMillis, waitForMillis); events = job2.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Expecting 1 more completion events for success", 3, - events.length); + assertEquals(3, + events.length, "Expecting 1 more completion events for success"); app.waitForState(reduceTask1, TaskState.RUNNING); app.waitForState(reduceTask2, TaskState.RUNNING); @@ -424,17 +411,14 @@ public Boolean get() { TaskAttemptEventType.TA_DONE)); app.waitForState(reduceTask2, TaskState.SUCCEEDED); - waitFor(new Supplier() { - @Override - public Boolean get() { - TaskAttemptCompletionEvent[] events = job2 - .getTaskAttemptCompletionEvents(0, 100); - return events.length == 5; - } + waitFor(() -> { + TaskAttemptCompletionEvent[] events14 = job2 + .getTaskAttemptCompletionEvents(0, 100); + return events14.length == 5; }, checkIntervalMillis, waitForMillis); events = job2.getTaskAttemptCompletionEvents(0, 100); - Assert.assertEquals("Expecting 2 more completion events for reduce success", - 5, events.length); + assertEquals(5, events.length, + "Expecting 2 more completion events for reduce success"); // job succeeds app.waitForState(job2, JobState.SUCCEEDED); @@ -472,7 +456,7 @@ public void testJobError() throws Exception { MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true); Job job = app.submit(new Configuration()); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -493,7 +477,7 @@ public void testJobSuccess() throws Exception { JobImpl job = (JobImpl) app.submit(new Configuration()); app.waitForInternalState(job, JobStateInternal.SUCCEEDED); // AM is not unregistered - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); // imitate that AM is unregistered app.successfullyUnregistered.set(true); app.waitForState(job, JobState.SUCCEEDED); @@ -505,7 +489,7 @@ public void testJobRebootNotLastRetryOnUnregistrationFailure() MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true); Job job = app.submit(new Configuration()); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -514,7 +498,7 @@ public void testJobRebootNotLastRetryOnUnregistrationFailure() app.getContext().getEventHandler().handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT)); - // return exteranl state as RUNNING since otherwise the JobClient will + // return external state as RUNNING since otherwise the JobClient will // prematurely exit. app.waitForState(job, JobState.RUNNING); } @@ -530,7 +514,7 @@ public void testJobRebootOnLastRetryOnUnregistrationFailure() Configuration conf = new Configuration(); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -540,7 +524,7 @@ public void testJobRebootOnLastRetryOnUnregistrationFailure() JobEventType.JOB_AM_REBOOT)); app.waitForInternalState((JobImpl) job, JobStateInternal.REBOOT); - // return exteranl state as RUNNING if this is the last retry while + // return external state as RUNNING if this is the last retry while // unregistration fails app.waitForState(job, JobState.RUNNING); } @@ -611,7 +595,7 @@ public void handle(ContainerLauncherEvent event) { super.handle(event); } }; - }; + } }; Job job = app.submit(new Configuration()); app.waitForState(job, JobState.SUCCEEDED); @@ -624,8 +608,7 @@ public void handle(ContainerLauncherEvent event) { (TaskAttemptImpl) taskAttempts.iterator().next(); // Container from RM should pass through to the launcher. Container object // should be the same. - Assert.assertTrue(taskAttempt.container - == containerObtainedByContainerLauncher); + assertSame(taskAttempt.container, containerObtainedByContainerLauncher); } private final class MRAppWithHistory extends MRApp { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java index 9710ec94a6969..f39503ecef486 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java @@ -20,8 +20,6 @@ import java.io.IOException; -import org.junit.Assert; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent; import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler; @@ -35,11 +33,15 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestMRAppComponentDependencies { - @Test(timeout = 20000) + @Test + @Timeout(value = 20) public void testComponentStopOrder() throws Exception { @SuppressWarnings("resource") TestMRApp app = new TestMRApp(1, 1, true, this.getClass().getName(), true); @@ -54,8 +56,8 @@ public void testComponentStopOrder() throws Exception { } // assert JobHistoryEventHandlerStopped and then clientServiceStopped - Assert.assertEquals(1, app.JobHistoryEventHandlerStopped); - Assert.assertEquals(2, app.clientServiceStopped); + assertEquals(1, app.JobHistoryEventHandlerStopped); + assertEquals(2, app.clientServiceStopped); } private final class TestMRApp extends MRApp { @@ -74,7 +76,7 @@ public TestMRApp(int maps, int reduces, boolean autoComplete, @Override protected Job createJob(Configuration conf, JobStateInternal forcedState, String diagnostic) { - UserGroupInformation currentUser = null; + UserGroupInformation currentUser; try { currentUser = UserGroupInformation.getCurrentUser(); } catch (IOException e) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java index 06550378ba939..ec9cf975cc6e5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java @@ -18,9 +18,10 @@ package org.apache.hadoop.mapreduce.v2.app; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; @@ -40,7 +41,6 @@ import java.util.HashMap; import java.util.Map; -import org.junit.Assert; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileContext; @@ -84,10 +84,11 @@ import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import org.slf4j.event.Level; @@ -104,7 +105,7 @@ public class TestMRAppMaster { static String stagingDir = new Path(testDir, "staging").toString(); private static FileContext localFS = null; - @BeforeClass + @BeforeAll public static void setup() throws AccessControlException, FileNotFoundException, IllegalArgumentException, IOException { //Do not error out if metrics are inited multiple times @@ -116,7 +117,7 @@ public static void setup() throws AccessControlException, new File(testDir.toString()).mkdir(); } - @Before + @BeforeEach public void prepare() throws IOException { File dir = new File(stagingDir); if(dir.exists()) { @@ -125,7 +126,7 @@ public void prepare() throws IOException { dir.mkdirs(); } - @AfterClass + @AfterAll public static void cleanup() throws IOException { localFS.delete(testDir, true); } @@ -226,8 +227,8 @@ public void testMRAppMasterJobLaunchTime() throws IOException, "host", -1, -1, System.currentTimeMillis()); MRAppMaster.initAndStartAppMaster(appMaster, conf, userName); appMaster.stop(); - assertTrue("Job launch time should not be negative.", - appMaster.jobLaunchTime.get() >= 0); + assertTrue( + appMaster.jobLaunchTime.get() >= 0, "Job launch time should not be negative."); } @Test @@ -343,7 +344,8 @@ public void testMRAppMasterMissingStaging() throws IOException, appMaster.stop(); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testMRAppMasterMaxAppAttempts() throws IOException, InterruptedException { // No matter what's the maxAppAttempt or attempt id, the isLastRetry always @@ -368,8 +370,8 @@ public void testMRAppMasterMaxAppAttempts() throws IOException, new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1, System.currentTimeMillis(), false, true); MRAppMaster.initAndStartAppMaster(appMaster, conf, userName); - assertEquals("isLastAMRetry is correctly computed.", expectedBools[i], - appMaster.isLastAMRetry()); + assertEquals(expectedBools[i], + appMaster.isLastAMRetry(), "isLastAMRetry is correctly computed."); } } @@ -418,14 +420,14 @@ public void testMRAppMasterCredentials() throws Exception { Text kind = new Text("MyTokenKind"); Text service = new Text("host:port"); Token myToken = - new Token(identifier, password, kind, service); + new Token<>(identifier, password, kind, service); Text tokenAlias = new Text("myToken"); credentials.addToken(tokenAlias, myToken); Text appTokenService = new Text("localhost:0"); Token appToken = - new Token(identifier, password, - AMRMTokenIdentifier.KIND_NAME, appTokenService); + new Token<>(identifier, password, + AMRMTokenIdentifier.KIND_NAME, appTokenService); credentials.addToken(appTokenService, appToken); Text keyAlias = new Text("mySecretKeyAlias"); @@ -436,7 +438,7 @@ public void testMRAppMasterCredentials() throws Exception { JobConf conf = new JobConf(); Path tokenFilePath = new Path(testDir, "tokens-file"); - Map newEnv = new HashMap(); + Map newEnv = new HashMap<>(); newEnv.put(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION, tokenFilePath .toUri().getPath()); setNewEnvironmentHack(newEnv); @@ -465,38 +467,36 @@ public void testMRAppMasterCredentials() throws Exception { // Now validate the task credentials Credentials appMasterCreds = appMaster.getCredentials(); - Assert.assertNotNull(appMasterCreds); - Assert.assertEquals(1, appMasterCreds.numberOfSecretKeys()); - Assert.assertEquals(1, appMasterCreds.numberOfTokens()); + assertNotNull(appMasterCreds); + assertEquals(1, appMasterCreds.numberOfSecretKeys()); + assertEquals(1, appMasterCreds.numberOfTokens()); // Validate the tokens - app token should not be present Token usedToken = appMasterCreds.getToken(tokenAlias); - Assert.assertNotNull(usedToken); - Assert.assertEquals(storedToken, usedToken); + assertNotNull(usedToken); + assertEquals(storedToken, usedToken); // Validate the keys byte[] usedKey = appMasterCreds.getSecretKey(keyAlias); - Assert.assertNotNull(usedKey); - Assert.assertEquals("mySecretKey", new String(usedKey)); + assertNotNull(usedKey); + assertEquals("mySecretKey", new String(usedKey)); // The credentials should also be added to conf so that OuputCommitter can // access it - app token should not be present Credentials confCredentials = conf.getCredentials(); - Assert.assertEquals(1, confCredentials.numberOfSecretKeys()); - Assert.assertEquals(1, confCredentials.numberOfTokens()); - Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias)); - Assert.assertEquals("mySecretKey", - new String(confCredentials.getSecretKey(keyAlias))); + assertEquals(1, confCredentials.numberOfSecretKeys()); + assertEquals(1, confCredentials.numberOfTokens()); + assertEquals(storedToken, confCredentials.getToken(tokenAlias)); + assertEquals("mySecretKey", new String(confCredentials.getSecretKey(keyAlias))); // Verify the AM's ugi - app token should be present Credentials ugiCredentials = appMaster.getUgi().getCredentials(); - Assert.assertEquals(1, ugiCredentials.numberOfSecretKeys()); - Assert.assertEquals(2, ugiCredentials.numberOfTokens()); - Assert.assertEquals(storedToken, ugiCredentials.getToken(tokenAlias)); - Assert.assertEquals(appToken, ugiCredentials.getToken(appTokenService)); - Assert.assertEquals("mySecretKey", - new String(ugiCredentials.getSecretKey(keyAlias))); + assertEquals(1, ugiCredentials.numberOfSecretKeys()); + assertEquals(2, ugiCredentials.numberOfTokens()); + assertEquals(storedToken, ugiCredentials.getToken(tokenAlias)); + assertEquals(appToken, ugiCredentials.getToken(appTokenService)); + assertEquals("mySecretKey", new String(ugiCredentials.getSecretKey(keyAlias))); } @@ -525,10 +525,10 @@ public void testMRAppMasterShutDownJob() throws Exception, doNothing().when(appMaster).serviceStop(); // Test normal shutdown. appMaster.shutDownJob(); - Assert.assertTrue("Expected shutDownJob to terminate.", - ExitUtil.terminateCalled()); - Assert.assertEquals("Expected shutDownJob to exit with status code of 0.", - 0, ExitUtil.getFirstExitException().status); + assertTrue(ExitUtil.terminateCalled(), + "Expected shutDownJob to terminate."); + assertEquals(0, ExitUtil.getFirstExitException().status, + "Expected shutDownJob to exit with status code of 0."); // Test shutdown with exception. ExitUtil.resetFirstExitException(); @@ -536,10 +536,10 @@ public void testMRAppMasterShutDownJob() throws Exception, doThrow(new RuntimeException(msg)) .when(appMaster).notifyIsLastAMRetry(anyBoolean()); appMaster.shutDownJob(); - assertTrue("Expected message from ExitUtil.ExitException to be " + msg, - ExitUtil.getFirstExitException().getMessage().contains(msg)); - Assert.assertEquals("Expected shutDownJob to exit with status code of 1.", - 1, ExitUtil.getFirstExitException().status); + assertTrue(ExitUtil.getFirstExitException().getMessage().contains(msg), + "Expected message from ExitUtil.ExitException to be " + msg); + assertEquals(1, ExitUtil.getFirstExitException().status, + "Expected shutDownJob to exit with status code of 1."); } private void verifyFailedStatus(MRAppMasterTest appMaster, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java index 9906def3ac950..b4f271f607f09 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java @@ -18,7 +18,11 @@ package org.apache.hadoop.mapreduce.v2.app; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import java.io.IOException; import java.security.PrivilegedExceptionAction; @@ -26,8 +30,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicReference; -import org.junit.Assert; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobACL; import org.apache.hadoop.mapreduce.JobID; @@ -70,7 +72,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.ipc.YarnRPC; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestMRClientService { @@ -82,7 +84,7 @@ public void test() throws Exception { Configuration conf = new Configuration(); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -116,8 +118,7 @@ public void test() throws Exception { GetCountersRequest gcRequest = recordFactory.newRecordInstance(GetCountersRequest.class); gcRequest.setJobId(job.getID()); - Assert.assertNotNull("Counters is null", - proxy.getCounters(gcRequest).getCounters()); + assertNotNull(proxy.getCounters(gcRequest).getCounters(), "Counters is null"); GetJobReportRequest gjrRequest = recordFactory.newRecordInstance(GetJobReportRequest.class); @@ -131,14 +132,14 @@ public void test() throws Exception { gtaceRequest.setJobId(job.getID()); gtaceRequest.setFromEventId(0); gtaceRequest.setMaxEvents(10); - Assert.assertNotNull("TaskCompletionEvents is null", - proxy.getTaskAttemptCompletionEvents(gtaceRequest).getCompletionEventList()); + assertNotNull(proxy.getTaskAttemptCompletionEvents(gtaceRequest).getCompletionEventList(), + "TaskCompletionEvents is null"); GetDiagnosticsRequest gdRequest = recordFactory.newRecordInstance(GetDiagnosticsRequest.class); gdRequest.setTaskAttemptId(attempt.getID()); - Assert.assertNotNull("Diagnostics is null", - proxy.getDiagnostics(gdRequest).getDiagnosticsList()); + assertNotNull(proxy.getDiagnostics(gdRequest).getDiagnosticsList(), + "Diagnostics is null"); GetTaskAttemptReportRequest gtarRequest = recordFactory.newRecordInstance(GetTaskAttemptReportRequest.class); @@ -151,31 +152,29 @@ public void test() throws Exception { GetTaskReportRequest gtrRequest = recordFactory.newRecordInstance(GetTaskReportRequest.class); gtrRequest.setTaskId(task.getID()); - Assert.assertNotNull("TaskReport is null", - proxy.getTaskReport(gtrRequest).getTaskReport()); + assertNotNull(proxy.getTaskReport(gtrRequest).getTaskReport(), + "TaskReport is null"); GetTaskReportsRequest gtreportsRequest = recordFactory.newRecordInstance(GetTaskReportsRequest.class); gtreportsRequest.setJobId(job.getID()); gtreportsRequest.setTaskType(TaskType.MAP); - Assert.assertNotNull("TaskReports for map is null", - proxy.getTaskReports(gtreportsRequest).getTaskReportList()); + assertNotNull(proxy.getTaskReports(gtreportsRequest).getTaskReportList(), + "TaskReports for map is null"); gtreportsRequest = recordFactory.newRecordInstance(GetTaskReportsRequest.class); gtreportsRequest.setJobId(job.getID()); gtreportsRequest.setTaskType(TaskType.REDUCE); - Assert.assertNotNull("TaskReports for reduce is null", - proxy.getTaskReports(gtreportsRequest).getTaskReportList()); + assertNotNull(proxy.getTaskReports(gtreportsRequest).getTaskReportList(), + "TaskReports for reduce is null"); List diag = proxy.getDiagnostics(gdRequest).getDiagnosticsList(); - Assert.assertEquals("Num diagnostics not correct", 1 , diag.size()); - Assert.assertEquals("Diag 1 not correct", - diagnostic1, diag.get(0).toString()); + assertEquals(1, diag.size(), "Num diagnostics not correct"); + assertEquals(diagnostic1, diag.get(0), "Diag 1 not correct"); TaskReport taskReport = proxy.getTaskReport(gtrRequest).getTaskReport(); - Assert.assertEquals("Num diagnostics not correct", 1, - taskReport.getDiagnosticsCount()); + assertEquals(1, taskReport.getDiagnosticsCount(), "Num diagnostics not correct"); //send the done signal to the task app.getContext().getEventHandler().handle( @@ -207,7 +206,7 @@ public void testViewAclOnlyCannotModify() throws Exception { conf.set(MRJobConfig.JOB_ACL_VIEW_JOB, "viewonlyuser"); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size()); + assertEquals(1, job.getTasks().size(), "Num tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task task = it.next(); app.waitForState(task, TaskState.RUNNING); @@ -217,18 +216,13 @@ public void testViewAclOnlyCannotModify() throws Exception { UserGroupInformation viewOnlyUser = UserGroupInformation.createUserForTesting( "viewonlyuser", new String[] {}); - Assert.assertTrue("viewonlyuser cannot view job", - job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB)); - Assert.assertFalse("viewonlyuser can modify job", - job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB)); + assertTrue(job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB), "viewonlyuser cannot view job"); + assertFalse(job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB), "viewonlyuser can modify job"); MRClientProtocol client = viewOnlyUser.doAs( - new PrivilegedExceptionAction() { - @Override - public MRClientProtocol run() throws Exception { - YarnRPC rpc = YarnRPC.create(conf); - return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class, - app.clientService.getBindAddress(), conf); - } + (PrivilegedExceptionAction) () -> { + YarnRPC rpc = YarnRPC.create(conf); + return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class, + app.clientService.getBindAddress(), conf); }); KillJobRequest killJobRequest = recordFactory.newRecordInstance( @@ -273,29 +267,27 @@ public MRClientProtocol run() throws Exception { } private void verifyJobReport(JobReport jr) { - Assert.assertNotNull("JobReport is null", jr); + assertNotNull(jr, "JobReport is null"); List amInfos = jr.getAMInfos(); - Assert.assertEquals(1, amInfos.size()); - Assert.assertEquals(JobState.RUNNING, jr.getJobState()); + assertEquals(1, amInfos.size()); + assertEquals(JobState.RUNNING, jr.getJobState()); AMInfo amInfo = amInfos.get(0); - Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); - Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); - Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); - Assert.assertEquals(1, amInfo.getAppAttemptId().getAttemptId()); - Assert.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId() - .getAttemptId()); - Assert.assertTrue(amInfo.getStartTime() > 0); - Assert.assertFalse(jr.isUber()); + assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); + assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); + assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); + assertEquals(1, amInfo.getAppAttemptId().getAttemptId()); + assertEquals(1, amInfo.getContainerId().getApplicationAttemptId().getAttemptId()); + assertTrue(amInfo.getStartTime() > 0); + assertFalse(jr.isUber()); } private void verifyTaskAttemptReport(TaskAttemptReport tar) { - Assert.assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState()); - Assert.assertNotNull("TaskAttemptReport is null", tar); - Assert.assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost()); - Assert.assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort()); - Assert.assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort()); - Assert.assertEquals(1, tar.getContainerId().getApplicationAttemptId() - .getAttemptId()); + assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState()); + assertNotNull(tar, "TaskAttemptReport is null"); + assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost()); + assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort()); + assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort()); + assertEquals(1, tar.getContainerId().getApplicationAttemptId().getAttemptId()); } class MRAppWithClientService extends MRApp { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java index 6c69dde2e92b0..601d2c39373fa 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java @@ -19,9 +19,9 @@ package org.apache.hadoop.mapreduce.v2.app; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.mock; @@ -43,7 +43,6 @@ import org.apache.hadoop.mapreduce.util.MRJobConfUtil; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -108,8 +107,9 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentCaptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -127,7 +127,7 @@ public class TestRecovery { private Text val1 = new Text("val1"); private Text val2 = new Text("val2"); - @BeforeClass + @BeforeAll public static void setupClass() throws Exception { // setup the test root directory testRootDir = @@ -159,8 +159,7 @@ public void testCrashed() throws Exception { app.waitForState(job, JobState.RUNNING); long jobStartTime = job.getReport().getStartTime(); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -193,7 +192,7 @@ public void testCrashed() throws Exception { Thread.sleep(2000); LOG.info("Waiting for next attempt to start"); } - Assert.assertEquals(2, mapTask1.getAttempts().size()); + assertEquals(2, mapTask1.getAttempts().size()); Iterator itr = mapTask1.getAttempts().values().iterator(); itr.next(); TaskAttempt task1Attempt2 = itr.next(); @@ -214,7 +213,7 @@ public void testCrashed() throws Exception { Thread.sleep(2000); LOG.info("Waiting for next attempt to start"); } - Assert.assertEquals(3, mapTask1.getAttempts().size()); + assertEquals(3, mapTask1.getAttempts().size()); itr = mapTask1.getAttempts().values().iterator(); itr.next(); itr.next(); @@ -235,7 +234,7 @@ public void testCrashed() throws Exception { Thread.sleep(2000); LOG.info("Waiting for next attempt to start"); } - Assert.assertEquals(4, mapTask1.getAttempts().size()); + assertEquals(4, mapTask1.getAttempts().size()); itr = mapTask1.getAttempts().values().iterator(); itr.next(); itr.next(); @@ -273,8 +272,7 @@ public void testCrashed() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -309,30 +307,26 @@ public void testCrashed() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertEquals("Job Start time not correct", - jobStartTime, job.getReport().getStartTime()); - Assert.assertEquals("Task Start time not correct", - task1StartTime, mapTask1.getReport().getStartTime()); - Assert.assertEquals("Task Finish time not correct", - task1FinishTime, mapTask1.getReport().getFinishTime()); - Assert.assertEquals(2, job.getAMInfos().size()); + assertEquals(jobStartTime, job.getReport().getStartTime(), "Job Start time not correct"); + assertEquals(task1StartTime, mapTask1.getReport().getStartTime(), + "Task Start time not correct"); + assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(), + "Task Finish time not correct"); + assertEquals(2, job.getAMInfos().size()); int attemptNum = 1; // Verify AMInfo for (AMInfo amInfo : job.getAMInfos()) { - Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId() - .getAttemptId()); - Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId() - .getApplicationAttemptId()); - Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); - Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); - Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); + assertEquals(attemptNum++, amInfo.getAppAttemptId().getAttemptId()); + assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId().getApplicationAttemptId()); + assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); + assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); + assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); } long am1StartTimeReal = job.getAMInfos().get(0).getStartTime(); long am2StartTimeReal = job.getAMInfos().get(1).getStartTime(); - Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst - && am1StartTimeReal <= am2StartTimeEst); - Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst - && am2StartTimeReal <= System.currentTimeMillis()); + assertTrue(am1StartTimeReal >= am1StartTimeEst && am1StartTimeReal <= am2StartTimeEst); + assertTrue(am2StartTimeReal >= am2StartTimeEst && + am2StartTimeReal <= System.currentTimeMillis()); // TODO Add verification of additional data from jobHistory - whatever was // available in the failed attempt should be available here } @@ -372,7 +366,7 @@ public void testCrashOfMapsOnlyJob() throws Exception { app.waitForState(job, JobState.RUNNING); // all maps would be running - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -430,7 +424,7 @@ public void testCrashOfMapsOnlyJob() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -517,7 +511,7 @@ public void testRecoverySuccessUsingCustomOutputCommitter() throws Exception { app.waitForState(job, JobState.RUNNING); // all maps would be running - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -576,7 +570,7 @@ public void testRecoverySuccessUsingCustomOutputCommitter() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -642,8 +636,8 @@ public void testRecoveryWithSpillEncryption() throws Exception { app = new MRAppWithHistory(1, 1, false, this.getClass().getName(), false, ++runCount); Job jobAttempt2 = app.submit(conf); - Assert.assertTrue("Recovery from previous job attempt is processed even " + - "though intermediate data encryption is enabled.", !app.recovered()); + assertFalse(app.recovered(), "Recovery from previous job attempt is processed even " + + "though intermediate data encryption is enabled."); // The map task succeeded from previous job attempt will not be recovered // because the data spill encryption is enabled. @@ -695,7 +689,7 @@ public void testRecoveryFailsUsingCustomOutputCommitter() throws Exception { app.waitForState(job, JobState.RUNNING); // all maps would be running - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -754,7 +748,7 @@ public void testRecoveryFailsUsingCustomOutputCommitter() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -814,8 +808,7 @@ public void testMultipleCrashes() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -834,8 +827,8 @@ public void testMultipleCrashes() throws Exception { app.waitForState(task2Attempt, TaskAttemptState.RUNNING); // reduces must be in NEW state - Assert.assertEquals("Reduce Task state not correct", - TaskState.RUNNING, reduceTask.getReport().getTaskState()); + assertEquals(TaskState.RUNNING, reduceTask.getReport().getTaskState(), + "Reduce Task state not correct"); //send the done signal to the 1st map app.getContext().getEventHandler().handle( @@ -863,8 +856,7 @@ public void testMultipleCrashes() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -906,8 +898,7 @@ public void testMultipleCrashes() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -941,8 +932,7 @@ public void testOutputRecovery() throws Exception { conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task reduceTask1 = it.next(); @@ -967,7 +957,7 @@ public void testOutputRecovery() throws Exception { app.waitForState(mapTask1, TaskState.SUCCEEDED); // Verify the shuffle-port - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); app.waitForState(reduceTask1, TaskState.RUNNING); TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next(); @@ -999,8 +989,7 @@ public void testOutputRecovery() throws Exception { conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); reduceTask1 = it.next(); @@ -1011,7 +1000,7 @@ public void testOutputRecovery() throws Exception { // Verify the shuffle-port after recovery task1Attempt1 = mapTask1.getAttempts().values().iterator().next(); - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); // first reduce will be recovered, no need to send done app.waitForState(reduceTask1, TaskState.SUCCEEDED); @@ -1052,7 +1041,7 @@ public void testPreviousJobOutputCleanedWhenNoRecovery() throws Exception { conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); //stop the app before the job completes. app.stop(); app.close(); @@ -1062,11 +1051,10 @@ public void testPreviousJobOutputCleanedWhenNoRecovery() throws Exception { ++runCount); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); TestFileOutputCommitter committer = ( TestFileOutputCommitter) app.getCommitter(); - assertTrue("commiter.abortJob() has not been called", - committer.isAbortJobCalled()); + assertTrue(committer.isAbortJobCalled(), "commiter.abortJob() has not been called"); app.close(); } @@ -1087,7 +1075,7 @@ public void testPreviousJobIsNotCleanedWhenRecovery() conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); //stop the app before the job completes. app.stop(); app.close(); @@ -1097,11 +1085,11 @@ public void testPreviousJobIsNotCleanedWhenRecovery() ++runCount); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); TestFileOutputCommitter committer = ( TestFileOutputCommitter) app.getCommitter(); - assertFalse("commiter.abortJob() has been called", - committer.isAbortJobCalled()); + assertFalse(committer.isAbortJobCalled(), + "commiter.abortJob() has been called"); app.close(); } @@ -1117,8 +1105,7 @@ public void testOutputRecoveryMapsOnly() throws Exception { conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -1148,7 +1135,7 @@ public void testOutputRecoveryMapsOnly() throws Exception { app.waitForState(mapTask1, TaskState.SUCCEEDED); // Verify the shuffle-port - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); //stop the app before the job completes. app.stop(); @@ -1165,8 +1152,7 @@ public void testOutputRecoveryMapsOnly() throws Exception { conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -1177,7 +1163,7 @@ public void testOutputRecoveryMapsOnly() throws Exception { // Verify the shuffle-port after recovery task1Attempt1 = mapTask1.getAttempts().values().iterator().next(); - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); app.waitForState(mapTask2, TaskState.RUNNING); @@ -1198,7 +1184,7 @@ public void testOutputRecoveryMapsOnly() throws Exception { app.waitForState(mapTask2, TaskState.SUCCEEDED); // Verify the shuffle-port - Assert.assertEquals(5467, task2Attempt1.getShufflePort()); + assertEquals(5467, task2Attempt1.getShufflePort()); app.waitForState(reduceTask1, TaskState.RUNNING); TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next(); @@ -1232,8 +1218,7 @@ public void testRecoveryWithOldCommiter() throws Exception { conf.set(FileOutputFormat.OUTDIR, outputDir.toString()); Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task reduceTask1 = it.next(); @@ -1258,7 +1243,7 @@ public void testRecoveryWithOldCommiter() throws Exception { app.waitForState(mapTask1, TaskState.SUCCEEDED); // Verify the shuffle-port - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); app.waitForState(reduceTask1, TaskState.RUNNING); TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next(); @@ -1290,8 +1275,7 @@ public void testRecoveryWithOldCommiter() throws Exception { conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false); job = app.submit(conf); app.waitForState(job, JobState.RUNNING); - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); reduceTask1 = it.next(); @@ -1302,7 +1286,7 @@ public void testRecoveryWithOldCommiter() throws Exception { // Verify the shuffle-port after recovery task1Attempt1 = mapTask1.getAttempts().values().iterator().next(); - Assert.assertEquals(5467, task1Attempt1.getShufflePort()); + assertEquals(5467, task1Attempt1.getShufflePort()); // first reduce will be recovered, no need to send done app.waitForState(reduceTask1, TaskState.SUCCEEDED); @@ -1352,8 +1336,7 @@ public void testSpeculative() throws Exception { app.waitForState(job, JobState.RUNNING); long jobStartTime = job.getReport().getStartTime(); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); @@ -1426,8 +1409,7 @@ public void testSpeculative() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -1463,34 +1445,31 @@ public void testSpeculative() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertEquals("Job Start time not correct", - jobStartTime, job.getReport().getStartTime()); - Assert.assertEquals("Task Start time not correct", - task1StartTime, mapTask1.getReport().getStartTime()); - Assert.assertEquals("Task Finish time not correct", - task1FinishTime, mapTask1.getReport().getFinishTime()); - Assert.assertEquals(2, job.getAMInfos().size()); + assertEquals(jobStartTime, job.getReport().getStartTime(), "Job Start time not correct"); + assertEquals(task1StartTime, mapTask1.getReport().getStartTime(), + "Task Start time not correct"); + assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(), + "Task Finish time not correct"); + assertEquals(2, job.getAMInfos().size()); int attemptNum = 1; // Verify AMInfo for (AMInfo amInfo : job.getAMInfos()) { - Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId() - .getAttemptId()); - Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId() - .getApplicationAttemptId()); - Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); - Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); - Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); + assertEquals(attemptNum++, amInfo.getAppAttemptId().getAttemptId()); + assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId().getApplicationAttemptId()); + assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost()); + assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort()); + assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort()); } long am1StartTimeReal = job.getAMInfos().get(0).getStartTime(); long am2StartTimeReal = job.getAMInfos().get(1).getStartTime(); - Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst - && am1StartTimeReal <= am2StartTimeEst); - Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst - && am2StartTimeReal <= System.currentTimeMillis()); + assertTrue(am1StartTimeReal >= am1StartTimeEst && am1StartTimeReal <= am2StartTimeEst); + assertTrue(am2StartTimeReal >= am2StartTimeEst && + am2StartTimeReal <= System.currentTimeMillis()); } - @Test(timeout=30000) + @Test + @Timeout(value = 30) public void testRecoveryWithoutShuffleSecret() throws Exception { int runCount = 0; @@ -1504,8 +1483,7 @@ public void testRecoveryWithoutShuffleSecret() throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); Iterator it = job.getTasks().values().iterator(); Task mapTask1 = it.next(); Task mapTask2 = it.next(); @@ -1551,8 +1529,7 @@ public void testRecoveryWithoutShuffleSecret() throws Exception { job = app.submit(conf); app.waitForState(job, JobState.RUNNING); //all maps would be running - Assert.assertEquals("No of tasks not correct", - 3, job.getTasks().size()); + assertEquals(3, job.getTasks().size(), "No of tasks not correct"); it = job.getTasks().values().iterator(); mapTask1 = it.next(); mapTask2 = it.next(); @@ -1891,16 +1868,16 @@ private void recoveryChecker(MapTaskImpl checkTask, TaskState finalState, ArgumentCaptor arg, List expectedJobHistoryEvents, long expectedMapLaunches, long expectedFailedMaps) { - assertEquals("Final State of Task", finalState, checkTask.getState()); + assertEquals(finalState, checkTask.getState(), "Final State of Task"); Map recoveredAttempts = checkTask.getAttempts(); - assertEquals("Expected Number of Task Attempts", - finalAttemptStates.size(), recoveredAttempts.size()); + assertEquals(finalAttemptStates.size(), recoveredAttempts.size(), + "Expected Number of Task Attempts"); for (TaskAttemptID taID : finalAttemptStates.keySet()) { - assertEquals("Expected Task Attempt State", - finalAttemptStates.get(taID), - recoveredAttempts.get(TypeConverter.toYarn(taID)).getState()); + assertEquals(finalAttemptStates.get(taID), + recoveredAttempts.get(TypeConverter.toYarn(taID)).getState(), + "Expected Task Attempt State"); } Iterator ie = arg.getAllValues().iterator(); @@ -1948,12 +1925,10 @@ private void recoveryChecker(MapTaskImpl checkTask, TaskState finalState, } } assertTrue(jobTaskEventReceived || (finalState == TaskState.RUNNING)); - assertEquals("Did not process all expected JobHistoryEvents", - 0, expectedJobHistoryEvents.size()); - assertEquals("Expected Map Launches", - expectedMapLaunches, totalLaunchedMaps); - assertEquals("Expected Failed Maps", - expectedFailedMaps, totalFailedMaps); + assertEquals(0, expectedJobHistoryEvents.size(), + "Did not process all expected JobHistoryEvents"); + assertEquals(expectedMapLaunches, totalLaunchedMaps, "Expected Map Launches"); + assertEquals(expectedFailedMaps, totalFailedMaps, "Expected Failed Maps"); } private MapTaskImpl getMockMapTask(long clusterTimestamp, EventHandler eh) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java index 0031598da5b33..b7dae0479fb26 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java @@ -78,13 +78,13 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.offset; +import static org.junit.jupiter.api.Assertions.assertEquals; @SuppressWarnings({"unchecked", "rawtypes"}) public class TestRuntimeEstimators { @@ -152,16 +152,16 @@ public class TestRuntimeEstimators { conf.setDouble(MRJobConfig.SPECULATIVECAP_TOTAL_TASKS, 0.001); conf.setInt(MRJobConfig.SPECULATIVE_MINIMUM_ALLOWED_TASKS, 5); speculator = new DefaultSpeculator(conf, myAppContext, estimator, clock); - Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value", - 500L, speculator.getSoonestRetryAfterNoSpeculate()); - Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_SPECULATE value", - 5000L, speculator.getSoonestRetryAfterSpeculate()); + assertEquals(500L, speculator.getSoonestRetryAfterNoSpeculate(), + "wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value"); + assertEquals(5000L, speculator.getSoonestRetryAfterSpeculate(), + "wrong SPECULATIVE_RETRY_AFTER_SPECULATE value"); assertThat(speculator.getProportionRunningTasksSpeculatable()) .isCloseTo(0.1, offset(0.00001)); assertThat(speculator.getProportionTotalTasksSpeculatable()) .isCloseTo(0.001, offset(0.00001)); - Assert.assertEquals("wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value", - 5, speculator.getMinimumAllowedSpeculativeTasks()); + assertEquals(5, speculator.getMinimumAllowedSpeculativeTasks(), + "wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value"); dispatcher.register(Speculator.EventType.class, speculator); @@ -188,7 +188,7 @@ public class TestRuntimeEstimators { int undoneReduces = REDUCE_TASKS; // build a task sequence where all the maps precede any of the reduces - List allTasksSequence = new LinkedList(); + List allTasksSequence = new LinkedList<>(); allTasksSequence.addAll(myJob.getTasks(TaskType.MAP).values()); allTasksSequence.addAll(myJob.getTasks(TaskType.REDUCE).values()); @@ -244,8 +244,8 @@ public class TestRuntimeEstimators { } } - Assert.assertEquals("We got the wrong number of successful speculations.", - expectedSpeculations, successfulSpeculations.get()); + assertEquals(expectedSpeculations, successfulSpeculations.get(), + "We got the wrong number of successful speculations."); } @Test @@ -279,8 +279,8 @@ public void handle(TaskEvent event) { TaskId taskID = event.getTaskID(); Task task = myJob.getTask(taskID); - Assert.assertEquals - ("Wrong type event", TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType()); + assertEquals(TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType(), + "Wrong type event"); System.out.println("SpeculationRequestEventHandler.handle adds a speculation task for " + taskID); @@ -296,8 +296,7 @@ void addAttempt(Task task) { class MyTaskImpl implements Task { private final TaskId taskID; - private final Map attempts - = new ConcurrentHashMap(4); + private final Map attempts = new ConcurrentHashMap<>(4); MyTaskImpl(JobId jobID, int index, TaskType type) { taskID = recordFactory.newRecordInstance(TaskId.class); @@ -353,8 +352,7 @@ public TaskType getType() { @Override public Map getAttempts() { - Map result - = new HashMap(attempts.size()); + Map result = new HashMap<>(attempts.size()); result.putAll(attempts); return result; } @@ -389,9 +387,9 @@ public TaskState getState() { class MyJobImpl implements Job { private final JobId jobID; - private final Map allTasks = new HashMap(); - private final Map mapTasks = new HashMap(); - private final Map reduceTasks = new HashMap(); + private final Map allTasks = new HashMap<>(); + private final Map mapTasks = new HashMap<>(); + private final Map reduceTasks = new HashMap<>(); MyJobImpl(JobId jobID, int numMaps, int numReduces) { this.jobID = jobID; @@ -570,7 +568,7 @@ public int getKilledReduces() { /* * We follow the pattern of the real XxxImpl . We create a job and initialize * it with a full suite of tasks which in turn have one attempt each in the - * NEW state. Attempts transition only from NEW to RUNNING to SUCCEEDED . + * NEW state. Attempts transition only from NEW to RUNNING to SUCCEED . */ class MyTaskAttemptImpl implements TaskAttempt { private final TaskAttemptId myAttemptID; @@ -936,7 +934,6 @@ public String getHistoryUrl() { @Override public void setHistoryUrl(String historyUrl) { - return; } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java index 1f0ce2309e285..f8f7c2092f409 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java @@ -18,8 +18,9 @@ package org.apache.hadoop.mapreduce.v2.app; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.mock; @@ -61,9 +62,9 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** @@ -78,7 +79,7 @@ public class TestStagingCleanup { private final static RecordFactory recordFactory = RecordFactoryProvider. getRecordFactory(null); - @After + @AfterEach public void tearDown() { conf.setBoolean(MRJobConfig.PRESERVE_FAILED_TASK_FILES, false); } @@ -135,7 +136,7 @@ public void testDeletionofStaging() throws IOException { JobId jobid = recordFactory.newRecordInstance(JobId.class); jobid.setAppId(appId); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -146,7 +147,8 @@ public void testDeletionofStaging() throws IOException { verify(fs).delete(stagingJobPath, true); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testNoDeletionofStagingOnReboot() throws IOException { conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir); fs = mock(FileSystem.class); @@ -158,7 +160,7 @@ public void testNoDeletionofStagingOnReboot() throws IOException { 0); ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -197,7 +199,8 @@ public void testDeletionofStagingOnReboot() throws IOException { verify(fs).delete(stagingJobPath, true); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testDeletionofStagingOnKill() throws IOException { conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir); fs = mock(FileSystem.class); @@ -242,13 +245,13 @@ public void testDeletionofStagingOnKillLastTry() throws IOException { ContainerAllocator mockAlloc = mock(ContainerAllocator.class); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc); //no retry appMaster.init(conf); - assertTrue("appMaster.isLastAMRetry() is false", appMaster.isLastAMRetry()); + assertTrue(appMaster.isLastAMRetry(), "appMaster.isLastAMRetry() is false"); //simulate the process being killed MRAppMaster.MRAppMasterShutdownHook hook = new MRAppMaster.MRAppMasterShutdownHook(appMaster); hook.run(); - assertTrue("MRAppMaster isn't stopped", - appMaster.isInState(Service.STATE.STOPPED)); + assertTrue(appMaster.isInState(Service.STATE.STOPPED), + "MRAppMaster isn't stopped"); verify(fs).delete(stagingJobPath, true); } @@ -270,7 +273,7 @@ public void testByPreserveFailedStaging() throws IOException { JobId jobid = recordFactory.newRecordInstance(JobId.class); jobid.setAppId(appId); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.FAILED, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -298,7 +301,7 @@ public void testPreservePatternMatchedStaging() throws IOException { JobId jobid = recordFactory.newRecordInstance(JobId.class); jobid.setAppId(appId); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -324,7 +327,7 @@ public void testNotPreserveNotPatternMatchedStaging() throws IOException { JobId jobid = recordFactory.newRecordInstance(JobId.class); jobid.setAppId(appId); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -355,7 +358,7 @@ public void testPreservePatternMatchedAndFailedStaging() throws IOException { JobId jobid = recordFactory.newRecordInstance(JobId.class); jobid.setAppId(appId); ContainerAllocator mockAlloc = mock(ContainerAllocator.class); - Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); + assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1); MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS); appMaster.init(conf); @@ -583,7 +586,8 @@ public void runOnNextHeartbeat(Runnable callback) { }; } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testStagingCleanupOrder() throws Exception { MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true, this.getClass().getName(), true); @@ -598,7 +602,7 @@ public void testStagingCleanupOrder() throws Exception { } // assert ContainerAllocatorStopped and then tagingDirCleanedup - Assert.assertEquals(1, app.ContainerAllocatorStopped); - Assert.assertEquals(2, app.stagingDirCleanedup); + assertEquals(1, app.ContainerAllocatorStopped); + assertEquals(2, app.stagingDirCleanedup); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java index f5c30c2a8db54..d36b6d7d8ba1b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java @@ -18,14 +18,15 @@ package org.apache.hadoop.mapreduce.v2.app; -import static org.junit.Assert.assertFalse; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.v2.api.records.JobId; @@ -40,8 +41,7 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -214,20 +214,15 @@ public void testTaskUnregistered() throws Exception { JobId jobId = MRBuilderUtils.newJobId(appId, 4); TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP); final TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2); - Assert.assertFalse(hb.hasRecentlyUnregistered(taid)); + assertFalse(hb.hasRecentlyUnregistered(taid)); hb.register(taid); - Assert.assertFalse(hb.hasRecentlyUnregistered(taid)); + assertFalse(hb.hasRecentlyUnregistered(taid)); hb.unregister(taid); - Assert.assertTrue(hb.hasRecentlyUnregistered(taid)); + assertTrue(hb.hasRecentlyUnregistered(taid)); long unregisterTimeout = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT, MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT); clock.setTime(unregisterTimeout + 1); - GenericTestUtils.waitFor(new Supplier() { - @Override - public Boolean get() { - return !hb.hasRecentlyUnregistered(taid); - } - }, 10, 10000); + GenericTestUtils.waitFor(() -> !hb.hasRecentlyUnregistered(taid), 10, 10000); } finally { hb.stop(); } @@ -260,7 +255,7 @@ private static void verifyTaskTimeoutConfig(final Configuration conf, new TaskHeartbeatHandler(null, SystemClock.getInstance(), 1); hb.init(conf); - Assert.assertTrue("The value of the task timeout is incorrect.", - hb.getTaskTimeOut() == expectedTimeout); + assertEquals(hb.getTaskTimeOut(), expectedTimeout, + "The value of the task timeout is incorrect."); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java index a3e85aad841ae..61059c29e47a0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java @@ -18,7 +18,11 @@ package org.apache.hadoop.mapreduce.v2.app.commit; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -27,7 +31,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; -import org.junit.Assert; +import org.junit.jupiter.api.Assertions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobContext; @@ -39,9 +43,6 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.EventHandler; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - import java.io.File; import java.io.IOException; @@ -62,9 +63,9 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestCommitterEventHandler { public static class WaitForItHandler implements EventHandler { @@ -95,13 +96,13 @@ public synchronized Event getAndClearEvent() throws InterruptedException { static String stagingDir = "target/test-staging/"; - @BeforeClass + @BeforeAll public static void setup() { File dir = new File(stagingDir); stagingDir = dir.getAbsolutePath(); } - @Before + @BeforeEach public void cleanup() throws IOException { File dir = new File(stagingDir); if(dir.exists()) { @@ -146,11 +147,11 @@ public void testCommitWindow() throws Exception { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals("committer did not register a heartbeat callback", - 1, rmhh.getNumCallbacks()); + Assertions.assertEquals(1, rmhh.getNumCallbacks(), + "committer did not register a heartbeat callback"); verify(committer, never()).commitJob(any(JobContext.class)); - Assert.assertEquals("committer should not have committed", - 0, jeh.numCommitCompletedEvents); + Assertions.assertEquals(0, jeh.numCommitCompletedEvents, + "committer should not have committed"); // set a fresh heartbeat and verify commit completes rmhh.setLastHeartbeatTime(clock.getTime()); @@ -159,8 +160,8 @@ public void testCommitWindow() throws Exception { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals("committer did not complete commit after RM hearbeat", - 1, jeh.numCommitCompletedEvents); + Assertions.assertEquals(1, jeh.numCommitCompletedEvents, + "committer did not complete commit after RM heartbeat"); verify(committer, times(1)).commitJob(any()); //Clean up so we can try to commit again (Don't do this at home) @@ -174,8 +175,8 @@ public void testCommitWindow() throws Exception { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals("committer did not commit", - 2, jeh.numCommitCompletedEvents); + Assertions.assertEquals(2, jeh.numCommitCompletedEvents, + "committer did not commit"); verify(committer, times(2)).commitJob(any()); ceh.stop(); @@ -262,9 +263,9 @@ public void testBasic() throws Exception { assertNotNull(e); assertTrue(e instanceof JobCommitCompletedEvent); FileSystem fs = FileSystem.get(conf); - assertTrue(startCommitFile.toString(), fs.exists(startCommitFile)); - assertTrue(endCommitSuccessFile.toString(), fs.exists(endCommitSuccessFile)); - assertFalse(endCommitFailureFile.toString(), fs.exists(endCommitFailureFile)); + assertTrue(fs.exists(startCommitFile), startCommitFile.toString()); + assertTrue(fs.exists(endCommitSuccessFile), endCommitSuccessFile.toString()); + assertFalse(fs.exists(endCommitFailureFile), endCommitFailureFile.toString()); verify(mockCommitter).commitJob(any(JobContext.class)); } finally { handler.stop(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java index 5f378e4f9c3fa..8c891847e9c9e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java @@ -18,11 +18,20 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl; -import static org.mockito.ArgumentMatchers.any; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.io.File; @@ -105,11 +114,10 @@ import org.apache.hadoop.yarn.state.StateMachineFactory; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.mockito.Mockito; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; /** @@ -120,13 +128,13 @@ public class TestJobImpl { static String stagingDir = "target/test-staging/"; - @BeforeClass + @BeforeAll public static void setup() { File dir = new File(stagingDir); stagingDir = dir.getAbsolutePath(); } - @Before + @BeforeEach public void cleanup() throws IOException { File dir = new File(stagingDir); if(dir.exists()) { @@ -169,13 +177,14 @@ public void testJobNoTasks() { dispatcher.stop(); commitHandler.stop(); try { - Assert.assertTrue(jseHandler.getAssertValue()); + assertTrue(jseHandler.getAssertValue()); } catch (InterruptedException e) { - Assert.fail("Workflow related attributes are not tested properly"); + fail("Workflow related attributes are not tested properly"); } } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testCommitJobFailsJob() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -200,7 +209,8 @@ public void testCommitJobFailsJob() throws Exception { commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testCheckJobCompleteSuccess() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -252,7 +262,8 @@ public void testCheckJobCompleteSuccess() throws Exception { commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testRebootedDuringSetup() throws Exception{ Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -289,13 +300,14 @@ public synchronized void setupJob(JobContext jobContext) assertJobState(job, JobStateInternal.REBOOT); // return the external state as RUNNING since otherwise JobClient will // exit when it polls the AM for job state - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); dispatcher.stop(); commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testRebootedDuringCommit() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -321,15 +333,16 @@ public void testRebootedDuringCommit() throws Exception { job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT)); assertJobState(job, JobStateInternal.REBOOT); // return the external state as ERROR since this is last retry. - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true); - Assert.assertEquals(JobState.ERROR, job.getState()); + assertEquals(JobState.ERROR, job.getState()); dispatcher.stop(); commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testKilledDuringSetup() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -366,7 +379,8 @@ public synchronized void setupJob(JobContext jobContext) commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testKilledDuringCommit() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -399,7 +413,7 @@ public void testAbortJobCalledAfterKillingTasks() throws IOException { InlineDispatcher dispatcher = new InlineDispatcher(); dispatcher.init(conf); dispatcher.start(); - OutputCommitter committer = Mockito.mock(OutputCommitter.class); + OutputCommitter committer = mock(OutputCommitter.class); CommitterEventHandler commitHandler = createCommitterEventHandler(dispatcher, committer); commitHandler.init(conf); @@ -411,19 +425,20 @@ public void testAbortJobCalledAfterKillingTasks() throws IOException { MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.FAILED)); //Verify abort job hasn't been called - Mockito.verify(committer, Mockito.never()) - .abortJob((JobContext) Mockito.any(), (State) Mockito.any()); + verify(committer, never()) + .abortJob((JobContext) any(), (State) any()); assertJobState(job, JobStateInternal.FAIL_WAIT); //Verify abortJob is called once and the job failed - Mockito.verify(committer, Mockito.timeout(2000).times(1)) - .abortJob((JobContext) Mockito.any(), (State) Mockito.any()); + verify(committer, timeout(2000).times(1)) + .abortJob((JobContext) any(), (State) any()); assertJobState(job, JobStateInternal.FAILED); dispatcher.stop(); } - @Test (timeout=10000) + @Test + @Timeout(value = 10) public void testFailAbortDoesntHang() throws IOException { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -432,7 +447,7 @@ public void testFailAbortDoesntHang() throws IOException { DrainDispatcher dispatcher = new DrainDispatcher(); dispatcher.init(conf); dispatcher.start(); - OutputCommitter committer = Mockito.mock(OutputCommitter.class); + OutputCommitter committer = mock(OutputCommitter.class); CommitterEventHandler commitHandler = createCommitterEventHandler(dispatcher, committer); commitHandler.init(conf); @@ -454,14 +469,15 @@ public void testFailAbortDoesntHang() throws IOException { dispatcher.await(); //Verify abortJob is called once and the job failed - Mockito.verify(committer, Mockito.timeout(2000).times(1)) - .abortJob((JobContext) Mockito.any(), (State) Mockito.any()); + verify(committer, timeout(2000).times(1)) + .abortJob((JobContext) any(), (State) any()); assertJobState(job, JobStateInternal.FAILED); dispatcher.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testKilledDuringFailAbort() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -503,7 +519,8 @@ public synchronized void abortJob(JobContext jobContext, State state) commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testKilledDuringKillAbort() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -546,7 +563,8 @@ public synchronized void abortJob(JobContext jobContext, State state) commitHandler.stop(); } - @Test(timeout=20000) + @Test + @Timeout(value = 20) public void testUnusableNodeTransition() throws Exception { Configuration conf = new Configuration(); conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir); @@ -599,7 +617,7 @@ public void handle(TaskAttemptEvent event) { job.handle(new JobTaskAttemptCompletedEvent(tce)); // complete the task itself job.handle(new JobTaskEvent(taskId, TaskState.SUCCEEDED)); - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); } } @@ -699,13 +717,13 @@ public void handle(TaskEvent event) { * much value. Instead, we validate the T_KILL events. */ if (killMappers) { - Assert.assertEquals("Number of killed events", 2, killedEvents.size()); - Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000000", - killedEvents.get(0).getTaskID().toString()); - Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000001", - killedEvents.get(1).getTaskID().toString()); + assertEquals(2, killedEvents.size(), "Number of killed events"); + assertEquals("task_1234567890000_0001_m_000000", + killedEvents.get(0).getTaskID().toString(), "AttemptID"); + assertEquals("task_1234567890000_0001_m_000001", + killedEvents.get(1).getTaskID().toString(), "AttemptID"); } else { - Assert.assertEquals("Number of killed events", 0, killedEvents.size()); + assertEquals(0, killedEvents.size(), "Number of killed events"); } } @@ -738,8 +756,8 @@ public void testCheckAccess() { // Verify access JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null, null, null, null, true, user1, 0, null, null, null, null); - Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB)); - Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB)); + assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB)); + assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB)); // Setup configuration access to the user1 (owner) and user2 Configuration conf2 = new Configuration(); @@ -749,8 +767,8 @@ public void testCheckAccess() { // Verify access JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null, null, null, null, true, user1, 0, null, null, null, null); - Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB)); - Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB)); + assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB)); + assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB)); // Setup configuration access with security enabled and access to all Configuration conf3 = new Configuration(); @@ -760,8 +778,8 @@ public void testCheckAccess() { // Verify access JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null, null, null, null, true, user1, 0, null, null, null, null); - Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB)); - Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB)); + assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB)); + assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB)); // Setup configuration access without security enabled Configuration conf4 = new Configuration(); @@ -771,8 +789,8 @@ public void testCheckAccess() { // Verify access JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null, null, null, null, true, user1, 0, null, null, null, null); - Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB)); - Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB)); + assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB)); + assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB)); // Setup configuration access without security enabled Configuration conf5 = new Configuration(); @@ -782,8 +800,8 @@ public void testCheckAccess() { // Verify access JobImpl job5 = new JobImpl(jobId, null, conf5, null, null, null, null, null, null, null, null, true, user1, 0, null, null, null, null); - Assert.assertTrue(job5.checkAccess(ugi1, null)); - Assert.assertTrue(job5.checkAccess(ugi2, null)); + assertTrue(job5.checkAccess(ugi1, null)); + assertTrue(job5.checkAccess(ugi2, null)); } @Test @@ -804,8 +822,8 @@ null, mock(JobTokenSecretManager.class), null, mrAppMetrics, null, true, null, 0, null, mockContext, null, null); job.handle(diagUpdateEvent); String diagnostics = job.getReport().getDiagnostics(); - Assert.assertNotNull(diagnostics); - Assert.assertTrue(diagnostics.contains(diagMsg)); + assertNotNull(diagnostics); + assertTrue(diagnostics.contains(diagMsg)); job = new JobImpl(jobId, Records .newRecord(ApplicationAttemptId.class), new Configuration(), @@ -816,8 +834,8 @@ null, mock(JobTokenSecretManager.class), null, job.handle(new JobEvent(jobId, JobEventType.JOB_KILL)); job.handle(diagUpdateEvent); diagnostics = job.getReport().getDiagnostics(); - Assert.assertNotNull(diagnostics); - Assert.assertTrue(diagnostics.contains(diagMsg)); + assertNotNull(diagnostics); + assertTrue(diagnostics.contains(diagMsg)); } @Test @@ -826,13 +844,13 @@ public void testUberDecision() throws Exception { // with default values, no of maps is 2 Configuration conf = new Configuration(); boolean isUber = testUberDecision(conf); - Assert.assertFalse(isUber); + assertFalse(isUber); // enable uber mode, no of maps is 2 conf = new Configuration(); conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true); isUber = testUberDecision(conf); - Assert.assertTrue(isUber); + assertTrue(isUber); // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max // reduces is 0 @@ -841,7 +859,7 @@ public void testUberDecision() throws Exception { conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 0); conf.setInt(MRJobConfig.NUM_REDUCES, 1); isUber = testUberDecision(conf); - Assert.assertFalse(isUber); + assertFalse(isUber); // enable uber mode, no of maps is 2, no of reduces is 1 and uber task max // reduces is 1 @@ -850,14 +868,14 @@ public void testUberDecision() throws Exception { conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1); conf.setInt(MRJobConfig.NUM_REDUCES, 1); isUber = testUberDecision(conf); - Assert.assertTrue(isUber); + assertTrue(isUber); // enable uber mode, no of maps is 2 and uber task max maps is 0 conf = new Configuration(); conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true); conf.setInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 1); isUber = testUberDecision(conf); - Assert.assertFalse(isUber); + assertFalse(isUber); // enable uber mode of 0 reducer no matter how much memory assigned to reducer conf = new Configuration(); @@ -866,7 +884,7 @@ public void testUberDecision() throws Exception { conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 2048); conf.setInt(MRJobConfig.REDUCE_CPU_VCORES, 10); isUber = testUberDecision(conf); - Assert.assertTrue(isUber); + assertTrue(isUber); } private boolean testUberDecision(Configuration conf) { @@ -931,9 +949,9 @@ public void testTransitionsAtFailed() throws IOException { assertJobState(job, JobStateInternal.FAILED); job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE)); assertJobState(job, JobStateInternal.FAILED); - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true); - Assert.assertEquals(JobState.FAILED, job.getState()); + assertEquals(JobState.FAILED, job.getState()); dispatcher.stop(); commitHandler.stop(); @@ -960,12 +978,12 @@ protected TaskSplitMetaInfo[] createSplits(JobImpl job, JobId jobId) { JobEvent mockJobEvent = mock(JobEvent.class); JobStateInternal jobSI = initTransition.transition(job, mockJobEvent); - Assert.assertTrue("When init fails, return value from InitTransition.transition should equal NEW.", - jobSI.equals(JobStateInternal.NEW)); - Assert.assertTrue("Job diagnostics should contain YarnRuntimeException", - job.getDiagnostics().toString().contains("YarnRuntimeException")); - Assert.assertTrue("Job diagnostics should contain " + EXCEPTIONMSG, - job.getDiagnostics().toString().contains(EXCEPTIONMSG)); + assertEquals(jobSI, JobStateInternal.NEW, + "When init fails, return value from InitTransition.transition should equal NEW."); + assertTrue(job.getDiagnostics().toString().contains("YarnRuntimeException"), + "Job diagnostics should contain YarnRuntimeException"); + assertTrue(job.getDiagnostics().toString().contains(EXCEPTIONMSG), + "Job diagnostics should contain " + EXCEPTIONMSG); } @Test @@ -986,7 +1004,7 @@ public void testJobPriorityUpdate() throws Exception { assertJobState(job, JobStateInternal.SETUP); // Update priority of job to 5, and it will be updated job.setJobPriority(submittedPriority); - Assert.assertEquals(submittedPriority, job.getReport().getJobPriority()); + assertEquals(submittedPriority, job.getReport().getJobPriority()); job.handle(new JobSetupCompletedEvent(jobId)); assertJobState(job, JobStateInternal.RUNNING); @@ -996,10 +1014,10 @@ public void testJobPriorityUpdate() throws Exception { job.setJobPriority(updatedPriority); assertJobState(job, JobStateInternal.RUNNING); Priority jobPriority = job.getReport().getJobPriority(); - Assert.assertNotNull(jobPriority); + assertNotNull(jobPriority); // Verify whether changed priority is same as what is set in Job. - Assert.assertEquals(updatedPriority, jobPriority); + assertEquals(updatedPriority, jobPriority); } @Test @@ -1013,14 +1031,14 @@ public void testCleanupSharedCacheUploadPolicies() { filePolicies.put("file1", true); filePolicies.put("jar1", true); Job.setFileSharedCacheUploadPolicies(config, filePolicies); - Assert.assertEquals( + assertEquals( 2, Job.getArchiveSharedCacheUploadPolicies(config).size()); - Assert.assertEquals( + assertEquals( 2, Job.getFileSharedCacheUploadPolicies(config).size()); JobImpl.cleanupSharedCacheUploadPolicies(config); - Assert.assertEquals( + assertEquals( 0, Job.getArchiveSharedCacheUploadPolicies(config).size()); - Assert.assertEquals( + assertEquals( 0, Job.getFileSharedCacheUploadPolicies(config).size()); } @@ -1088,14 +1106,14 @@ private static void completeJobTasks(JobImpl job) { job.handle(new JobTaskEvent( MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.SUCCEEDED)); - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); } int numReduces = job.getTotalReduces(); for (int i = 0; i < numReduces; ++i) { job.handle(new JobTaskEvent( MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP), TaskState.SUCCEEDED)); - Assert.assertEquals(JobState.RUNNING, job.getState()); + assertEquals(JobState.RUNNING, job.getState()); } } @@ -1109,7 +1127,7 @@ private static void assertJobState(JobImpl job, JobStateInternal state) { break; } } - Assert.assertEquals(state, job.getInternalState()); + assertEquals(state, job.getInternalState()); } private void createSpiedMapTasks(Map diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java index 699afbf3ca8f3..7d824991b36e1 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java @@ -22,7 +22,6 @@ import java.util.Map; import org.apache.hadoop.mapreduce.TaskType; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; @@ -37,16 +36,22 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent; import org.apache.hadoop.mapreduce.v2.util.MRApps; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + public class TestMapReduceChildJVM { private static final Logger LOG = LoggerFactory.getLogger(TestMapReduceChildJVM.class); - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCommandLine() throws Exception { MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true); @@ -56,7 +61,7 @@ public void testCommandLine() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertEquals( + assertEquals( "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" + " -Djava.net.preferIPv4Stack=true" + " -Dhadoop.metrics.log.level=WARN " + @@ -72,23 +77,24 @@ public void testCommandLine() throws Exception { " 1>/stdout" + " 2>/stderr ]", app.launchCmdList.get(0)); - Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job", - app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER")); - Assert.assertEquals("INFO,console", - app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); - Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job", - app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS")); - Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"), + "HADOOP_ROOT_LOGGER not set for job"); + assertEquals("INFO,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"), + "HADOOP_CLIENT_OPTS not set for job"); + assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReduceCommandLineWithSeparateShuffle() throws Exception { final Configuration conf = new Configuration(); conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true); testReduceCommandLine(conf); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReduceCommandLineWithSeparateCRLAShuffle() throws Exception { final Configuration conf = new Configuration(); conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true); @@ -97,7 +103,8 @@ public void testReduceCommandLineWithSeparateCRLAShuffle() throws Exception { testReduceCommandLine(conf); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testReduceCommandLine() throws Exception { final Configuration conf = new Configuration(); testReduceCommandLine(conf); @@ -120,7 +127,7 @@ private void testReduceCommandLine(Configuration conf) ? "shuffleCRLA" : "shuffleCLA"; - Assert.assertEquals( + assertEquals( "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" + " -Djava.net.preferIPv4Stack=true" + " -Dhadoop.metrics.log.level=WARN " + @@ -140,16 +147,16 @@ private void testReduceCommandLine(Configuration conf) " 1>/stdout" + " 2>/stderr ]", app.launchCmdList.get(0)); - Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job", - app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER")); - Assert.assertEquals("INFO,console", - app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); - Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job", - app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS")); - Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"), + "HADOOP_ROOT_LOGGER not set for job"); + assertEquals("INFO,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"), + "HADOOP_CLIENT_OPTS not set for job"); + assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); } - @Test (timeout = 30000) + @Test + @Timeout(value = 30) public void testCommandLineWithLog4JConfig() throws Exception { MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true); @@ -162,7 +169,7 @@ public void testCommandLineWithLog4JConfig() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertEquals( + assertEquals( "[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" + " -Djava.net.preferIPv4Stack=true" + " -Dhadoop.metrics.log.level=WARN " + @@ -204,10 +211,8 @@ private void testAutoHeapSize(int mapMb, int redMb, String xmxArg) MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO); // Verify map and reduce java opts are not set by default - Assert.assertNull("Default map java opts!", - conf.get(MRJobConfig.MAP_JAVA_OPTS)); - Assert.assertNull("Default reduce java opts!", - conf.get(MRJobConfig.REDUCE_JAVA_OPTS)); + assertNull(conf.get(MRJobConfig.MAP_JAVA_OPTS), "Default map java opts!"); + assertNull(conf.get(MRJobConfig.REDUCE_JAVA_OPTS), "Default reduce java opts!"); // Set the memory-mbs and java-opts if (mapMb > 0) { conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMb); @@ -243,8 +248,8 @@ private void testAutoHeapSize(int mapMb, int redMb, String xmxArg) : MRJobConfig.REDUCE_JAVA_OPTS); heapMb = JobConf.parseMaximumHeapSizeMB(javaOpts); } - Assert.assertEquals("Incorrect heapsize in the command opts", - heapMb, JobConf.parseMaximumHeapSizeMB(cmd)); + assertEquals(heapMb, JobConf.parseMaximumHeapSizeMB(cmd), + "Incorrect heapsize in the command opts"); } } @@ -289,13 +294,12 @@ public void testEnvironmentVariables() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job", - app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER")); - Assert.assertEquals("WARN,console", - app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); - Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job", - app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS")); - Assert.assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"), + "HADOOP_ROOT_LOGGER not set for job"); + assertEquals("WARN,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"), + "HADOOP_CLIENT_OPTS not set for job"); + assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS")); // Try one more. app = new MyMRApp(1, 0, true, this.getClass().getName(), true); @@ -305,10 +309,9 @@ public void testEnvironmentVariables() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job", - app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER")); - Assert.assertEquals("trace", - app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"), + "HADOOP_ROOT_LOGGER not set for job"); + assertEquals("trace", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); // Try one using the mapreduce.task.env.var=value syntax app = new MyMRApp(1, 0, true, this.getClass().getName(), true); @@ -319,9 +322,8 @@ public void testEnvironmentVariables() throws Exception { app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); - Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job", - app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER")); - Assert.assertEquals("DEBUG,console", - app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); + assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"), + "HADOOP_ROOT_LOGGER not set for job"); + assertEquals("DEBUG,console", app.cmdEnvironment.get("HADOOP_ROOT_LOGGER")); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java index f44ff81079b86..3243488df9fb7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java @@ -18,6 +18,8 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -53,8 +55,7 @@ import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.junit.Test; -import org.junit.Assert; +import org.junit.jupiter.api.Test; public class TestShuffleProvider { @@ -110,9 +111,12 @@ public void testShuffleProviders() throws Exception { credentials); Map serviceDataMap = launchCtx.getServiceData(); - Assert.assertNotNull("TestShuffleHandler1 is missing", serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID)); - Assert.assertNotNull("TestShuffleHandler2 is missing", serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID)); - Assert.assertTrue("mismatch number of services in map", serviceDataMap.size() == 3); // 2 that we entered + 1 for the built-in shuffle-provider + assertNotNull(serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID), + "TestShuffleHandler1 is missing"); + assertNotNull(serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID), + "TestShuffleHandler2 is missing"); + // 2 that we entered + 1 for the built-in shuffle-provider + assertEquals(3, serviceDataMap.size(), "mismatch number of services in map"); } static public class StubbedFS extends RawLocalFileSystem { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java index 15682eeefc6c7..23e666cea1e5d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java @@ -20,9 +20,11 @@ import static org.apache.hadoop.test.GenericTestUtils.waitFor; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -41,10 +43,9 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent; import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -111,7 +112,7 @@ import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.apache.log4j.spi.LoggingEvent; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; @@ -151,17 +152,17 @@ private List getLogEvents() { } } - @BeforeClass + @BeforeAll public static void setupBeforeClass() { ResourceUtils.resetResourceTypes(new Configuration()); } - @Before + @BeforeEach public void before() { TaskAttemptImpl.RESOURCE_REQUEST_CACHE.clear(); } - @After + @AfterEach public void tearDown() { ResourceUtils.resetResourceTypes(new Configuration()); } @@ -289,7 +290,7 @@ public void testSingleRackRequest() throws Exception { ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); verify(eventHandler, times(2)).handle(arg.capture()); if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) { - Assert.fail("Second Event not of type ContainerRequestEvent"); + fail("Second Event not of type ContainerRequestEvent"); } ContainerRequestEvent cre = (ContainerRequestEvent) arg.getAllValues().get(1); @@ -323,7 +324,7 @@ public void testHostResolveAttempt() throws Exception { ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class); verify(eventHandler, times(2)).handle(arg.capture()); if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) { - Assert.fail("Second Event not of type ContainerRequestEvent"); + fail("Second Event not of type ContainerRequestEvent"); } Map expected = new HashMap(); expected.put("host1", true); @@ -361,16 +362,16 @@ public void verifyMillisCounters(Resource containerResource, Job job = app.submit(conf); app.waitForState(job, JobState.RUNNING); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 2, tasks.size()); + assertEquals(2, tasks.size(), "Num tasks is not correct"); Iterator taskIter = tasks.values().iterator(); Task mTask = taskIter.next(); app.waitForState(mTask, TaskState.RUNNING); Task rTask = taskIter.next(); app.waitForState(rTask, TaskState.RUNNING); Map mAttempts = mTask.getAttempts(); - Assert.assertEquals("Num attempts is not correct", 1, mAttempts.size()); + assertEquals(1, mAttempts.size(), "Num attempts is not correct"); Map rAttempts = rTask.getAttempts(); - Assert.assertEquals("Num attempts is not correct", 1, rAttempts.size()); + assertEquals(1, rAttempts.size(), "Num attempts is not correct"); TaskAttempt mta = mAttempts.values().iterator().next(); TaskAttempt rta = rAttempts.values().iterator().next(); app.waitForState(mta, TaskAttemptState.RUNNING); @@ -392,21 +393,21 @@ public void verifyMillisCounters(Resource containerResource, int memoryMb = (int) containerResource.getMemorySize(); int vcores = containerResource.getVirtualCores(); - Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize), + assertEquals((int) Math.ceil((float) memoryMb / minContainerSize), counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue()); - Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize), + assertEquals((int) Math.ceil((float) memoryMb / minContainerSize), counters.findCounter(JobCounter.SLOTS_MILLIS_REDUCES).getValue()); - Assert.assertEquals(1, + assertEquals(1, counters.findCounter(JobCounter.MILLIS_MAPS).getValue()); - Assert.assertEquals(1, + assertEquals(1, counters.findCounter(JobCounter.MILLIS_REDUCES).getValue()); - Assert.assertEquals(memoryMb, + assertEquals(memoryMb, counters.findCounter(JobCounter.MB_MILLIS_MAPS).getValue()); - Assert.assertEquals(memoryMb, + assertEquals(memoryMb, counters.findCounter(JobCounter.MB_MILLIS_REDUCES).getValue()); - Assert.assertEquals(vcores, + assertEquals(vcores, counters.findCounter(JobCounter.VCORES_MILLIS_MAPS).getValue()); - Assert.assertEquals(vcores, + assertEquals(vcores, counters.findCounter(JobCounter.VCORES_MILLIS_REDUCES).getValue()); } @@ -452,23 +453,23 @@ private void testMRAppHistory(MRApp app) throws Exception { app.waitForState(job, JobState.FAILED); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + assertEquals(1, tasks.size(), "Num tasks is not correct"); Task task = tasks.values().iterator().next(); - Assert.assertEquals("Task state not correct", TaskState.FAILED, task - .getReport().getTaskState()); + assertEquals(TaskState.FAILED, task + .getReport().getTaskState(), "Task state not correct"); Map attempts = tasks.values().iterator().next() .getAttempts(); - Assert.assertEquals("Num attempts is not correct", 4, attempts.size()); + assertEquals(4, attempts.size(), "Num attempts is not correct"); Iterator it = attempts.values().iterator(); TaskAttemptReport report = it.next().getReport(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED, - report.getTaskAttemptState()); - Assert.assertEquals("Diagnostic Information is not Correct", - "Test Diagnostic Event", report.getDiagnosticInfo()); + assertEquals(TaskAttemptState.FAILED, + report.getTaskAttemptState(), "Attempt state not correct"); + assertEquals("Test Diagnostic Event", report.getDiagnosticInfo(), + "Diagnostic Information is not Correct"); report = it.next().getReport(); - Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED, - report.getTaskAttemptState()); + assertEquals(TaskAttemptState.FAILED, + report.getTaskAttemptState(), "Attempt state not correct"); } private void testTaskAttemptAssignedFailHistory @@ -477,8 +478,8 @@ private void testMRAppHistory(MRApp app) throws Exception { Job job = app.submit(conf); app.waitForState(job, JobState.FAILED); Map tasks = job.getTasks(); - Assert.assertTrue("No Ta Started JH Event", app.getTaStartJHEvent()); - Assert.assertTrue("No Ta Failed JH Event", app.getTaFailedJHEvent()); + assertTrue(app.getTaStartJHEvent(), "No Ta Started JH Event"); + assertTrue(app.getTaFailedJHEvent(), "No Ta Failed JH Event"); } private void testTaskAttemptAssignedKilledHistory @@ -518,8 +519,8 @@ public void handle(JobHistoryEvent event) { if (event.getType() == org.apache.hadoop.mapreduce.jobhistory.EventType.MAP_ATTEMPT_FAILED) { TaskAttemptUnsuccessfulCompletion datum = (TaskAttemptUnsuccessfulCompletion) event .getHistoryEvent().getDatum(); - Assert.assertEquals("Diagnostic Information is not Correct", - "Test Diagnostic Event", datum.get(8).toString()); + assertEquals("Test Diagnostic Event", datum.get(8).toString(), + "Diagnostic Information is not Correct"); } } }; @@ -638,8 +639,8 @@ public void testLaunchFailedWhileKilling() throws Exception { taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED)); assertFalse(eventHandler.internalError); - assertEquals("Task attempt is not assigned on the local node", - Locality.NODE_LOCAL, taImpl.getLocality()); + assertEquals(Locality.NODE_LOCAL, taImpl.getLocality(), + "Task attempt is not assigned on the local node"); } @Test @@ -695,10 +696,10 @@ public void testContainerCleanedWhileRunning() throws Exception { .isEqualTo(TaskAttemptState.RUNNING); taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_CONTAINER_CLEANED)); - assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", - eventHandler.internalError); - assertEquals("Task attempt is not assigned on the local rack", - Locality.RACK_LOCAL, taImpl.getLocality()); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_CONTAINER_CLEANED"); + assertEquals(Locality.RACK_LOCAL, taImpl.getLocality(), + "Task attempt is not assigned on the local rack"); } @Test @@ -757,10 +758,10 @@ public void testContainerCleanedWhileCommitting() throws Exception { .isEqualTo(TaskAttemptState.COMMIT_PENDING); taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_CONTAINER_CLEANED)); - assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", - eventHandler.internalError); - assertEquals("Task attempt is assigned locally", Locality.OFF_SWITCH, - taImpl.getLocality()); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_CONTAINER_CLEANED"); + assertEquals(Locality.OFF_SWITCH, + taImpl.getLocality(), "Task attempt is assigned locally"); } @Test @@ -832,8 +833,8 @@ public void testDoubleTooManyFetchFailure() throws Exception { assertThat(taImpl.getState()) .withFailMessage("Task attempt is not in FAILED state, still") .isEqualTo(TaskAttemptState.FAILED); - assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_CONTAINER_CLEANED"); } @@ -883,16 +884,14 @@ public void testAppDiagnosticEventOnUnassignedTask() { TaskAttemptEventType.TA_SCHEDULE)); taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId, "Task got killed")); - assertFalse( - "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task"); try { taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_KILL)); - Assert.assertTrue("No exception on UNASSIGNED STATE KILL event", true); + assertTrue(true, "No exception on UNASSIGNED STATE KILL event"); } catch (Exception e) { - Assert.assertFalse( - "Exception not expected for UNASSIGNED STATE KILL event", true); + fail("Exception not expected for UNASSIGNED STATE KILL event"); } } @@ -962,8 +961,8 @@ public void testTooManyFetchFailureAfterKill() throws Exception { assertThat(taImpl.getState()) .withFailMessage("Task attempt is not in KILLED state, still") .isEqualTo(TaskAttemptState.KILLED); - assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_CONTAINER_CLEANED"); } @Test @@ -1009,9 +1008,8 @@ public void testAppDiagnosticEventOnNewTask() { when(container.getNodeHttpAddress()).thenReturn("localhost:0"); taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId, "Task got killed")); - assertFalse( - "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task"); } @Test @@ -1072,8 +1070,8 @@ public void testFetchFailureAttemptFinishTime() throws Exception{ .withFailMessage("Task attempt is not in SUCCEEDED state") .isEqualTo(TaskAttemptState.SUCCEEDED); - assertTrue("Task Attempt finish time is not greater than 0", - taImpl.getFinishTime() > 0); + assertTrue(taImpl.getFinishTime() > 0, + "Task Attempt finish time is not greater than 0"); Long finishTime = taImpl.getFinishTime(); Thread.sleep(5); @@ -1084,9 +1082,9 @@ public void testFetchFailureAttemptFinishTime() throws Exception{ .withFailMessage("Task attempt is not in FAILED state") .isEqualTo(TaskAttemptState.FAILED); - assertEquals("After TA_TOO_MANY_FETCH_FAILURE," - + " Task attempt finish time is not the same ", - finishTime, Long.valueOf(taImpl.getFinishTime())); + assertEquals(finishTime, Long.valueOf(taImpl.getFinishTime()), + "After TA_TOO_MANY_FETCH_FAILURE," + + " Task attempt finish time is not the same "); } private void containerKillBeforeAssignment(boolean scheduleAttempt) @@ -1114,7 +1112,7 @@ private void containerKillBeforeAssignment(boolean scheduleAttempt) assertThat(taImpl.getInternalState()) .withFailMessage("Task attempt's internal state is not KILLED") .isEqualTo(TaskAttemptStateInternal.KILLED); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); TaskEvent event = eventHandler.lastTaskEvent; assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType()); // In NEW state, new map attempt should not be rescheduled. @@ -1238,8 +1236,8 @@ public void testContainerKillWhileRunning() throws Exception { .isEqualTo(TaskAttemptState.RUNNING); taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_KILL)); - assertFalse("InternalError occurred trying to handle TA_KILL", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_KILL"); assertThat(taImpl.getInternalState()) .withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state") .isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP); @@ -1301,8 +1299,8 @@ public void testContainerKillWhileCommitPending() throws Exception { .isEqualTo(TaskAttemptStateInternal.COMMIT_PENDING); taImpl.handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_KILL)); - assertFalse("InternalError occurred trying to handle TA_KILL", - eventHandler.internalError); + assertFalse(eventHandler.internalError, + "InternalError occurred trying to handle TA_KILL"); assertThat(taImpl.getInternalState()) .withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state") .isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP); @@ -1348,7 +1346,7 @@ public void testKillMapTaskWhileSuccessFinishing() throws Exception { .withFailMessage("Task attempt is not in KILLED state") .isEqualTo(TaskAttemptState.KILLED); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1359,32 +1357,30 @@ public void testKillMapOnlyTaskWhileSuccessFinishing() throws Exception { taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_DONE)); - assertEquals("Task attempt is not in SUCCEEDED state", - TaskAttemptState.SUCCEEDED, taImpl.getState()); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_FINISHING_CONTAINER", - TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, - taImpl.getInternalState()); + assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(), + "Task attempt is not in SUCCEEDED state"); + assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_FINISHING_CONTAINER"); // If the map only task is killed when it is in SUCCESS_FINISHING_CONTAINER // state, the state will move to SUCCESS_CONTAINER_CLEANUP taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_KILL)); - assertEquals("Task attempt is not in SUCCEEDED state", - TaskAttemptState.SUCCEEDED, taImpl.getState()); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_CONTAINER_CLEANUP", - TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, - taImpl.getInternalState()); + assertEquals(TaskAttemptState.SUCCEEDED, + taImpl.getState(), "Task attempt is not in SUCCEEDED state"); + assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_CONTAINER_CLEANUP"); taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_CONTAINER_CLEANED)); - assertEquals("Task attempt is not in SUCCEEDED state", - TaskAttemptState.SUCCEEDED, taImpl.getState()); - assertEquals("Task attempt's internal state is not SUCCEEDED state", - TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState()); + assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(), + "Task attempt is not in SUCCEEDED state"); + assertEquals(TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState(), + "Task attempt's internal state is not SUCCEEDED state"); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1414,7 +1410,7 @@ public void testKillMapTaskAfterSuccess() throws Exception { assertThat(taImpl.getInternalState()) .withFailMessage("Task attempt's internal state is not KILLED") .isEqualTo(TaskAttemptStateInternal.KILLED); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); TaskEvent event = eventHandler.lastTaskEvent; assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType()); // Send an attempt killed event to TaskImpl forwarding the same reschedule @@ -1430,22 +1426,21 @@ public void testKillMapOnlyTaskAfterSuccess() throws Exception { taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_DONE)); - assertEquals("Task attempt is not in SUCCEEDED state", - TaskAttemptState.SUCCEEDED, taImpl.getState()); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_FINISHING_CONTAINER", - TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, - taImpl.getInternalState()); + assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(), + "Task attempt is not in SUCCEEDED state"); + assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_FINISHING_CONTAINER"); taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_CONTAINER_CLEANED)); // Succeeded taImpl.handle(new TaskAttemptKillEvent(taImpl.getID(),"", true)); - assertEquals("Task attempt is not in SUCCEEDED state", - TaskAttemptState.SUCCEEDED, taImpl.getState()); - assertEquals("Task attempt's internal state is not SUCCEEDED", - TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState()); - assertFalse("InternalError occurred", eventHandler.internalError); + assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(), + "Task attempt is not in SUCCEEDED state"); + assertEquals(TaskAttemptStateInternal.SUCCEEDED, + taImpl.getInternalState(), "Task attempt's internal state is not SUCCEEDED"); + assertFalse(eventHandler.internalError, "InternalError occurred"); TaskEvent event = eventHandler.lastTaskEvent; assertEquals(TaskEventType.T_ATTEMPT_SUCCEEDED, event.getType()); } @@ -1498,7 +1493,7 @@ public void testKillMapTaskWhileFailFinishing() throws Exception { .withFailMessage("Task attempt is not in FAILED state") .isEqualTo(TaskAttemptState.FAILED); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1531,7 +1526,7 @@ public void testFailMapTaskByClient() throws Exception { .withFailMessage("Task attempt is not in FAILED state") .isEqualTo(TaskAttemptState.FAILED); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1561,7 +1556,7 @@ public void testTaskAttemptDiagnosticEventOnFinishing() throws Exception { "SUCCESS_FINISHING_CONTAINER") .isEqualTo(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1592,7 +1587,7 @@ public void testTimeoutWhileSuccessFinishing() throws Exception { "SUCCESS_CONTAINER_CLEANUP") .isEqualTo(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1619,7 +1614,7 @@ public void testTimeoutWhileFailFinishing() throws Exception { "FAIL_CONTAINER_CLEANUP") .isEqualTo(TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP); - assertFalse("InternalError occurred", eventHandler.internalError); + assertFalse(eventHandler.internalError, "InternalError occurred"); } @Test @@ -1636,8 +1631,8 @@ public void testMapperCustomResourceTypes() { ResourceInformation resourceInfo = getResourceInfoFromContainerRequest(taImpl, eventHandler). getResourceInformation(CUSTOM_RESOURCE_NAME); - assertEquals("Expecting the default unit (G)", - "G", resourceInfo.getUnits()); + assertEquals("G", resourceInfo.getUnits(), + "Expecting the default unit (G)"); assertEquals(7L, resourceInfo.getValue()); } @@ -1654,8 +1649,8 @@ public void testReducerCustomResourceTypes() { ResourceInformation resourceInfo = getResourceInfoFromContainerRequest(taImpl, eventHandler). getResourceInformation(CUSTOM_RESOURCE_NAME); - assertEquals("Expecting the specified unit (m)", - "m", resourceInfo.getUnits()); + assertEquals("m", resourceInfo.getUnits(), + "Expecting the specified unit (m)"); assertEquals(3L, resourceInfo.getValue()); } @@ -1752,18 +1747,20 @@ public void testReducerMemoryRequestOverriding() { } } - @Test(expected=IllegalArgumentException.class) + @Test public void testReducerMemoryRequestMultipleName() { - EventHandler eventHandler = mock(EventHandler.class); - Clock clock = SystemClock.getInstance(); - JobConf jobConf = new JobConf(); - for (String memoryName : ImmutableList.of( - MRJobConfig.RESOURCE_TYPE_NAME_MEMORY, - MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) { - jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName, - "3Gi"); - } - createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf); + assertThrows(IllegalArgumentException.class, ()->{ + EventHandler eventHandler = mock(EventHandler.class); + Clock clock = SystemClock.getInstance(); + JobConf jobConf = new JobConf(); + for (String memoryName : ImmutableList.of( + MRJobConfig.RESOURCE_TYPE_NAME_MEMORY, + MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) { + jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName, + "3Gi"); + } + createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf); + }); } @Test @@ -1853,21 +1850,23 @@ private Resource getResourceInfoFromContainerRequest( containerRequestEvents.add((ContainerRequestEvent) e); } } - assertEquals("Expected one ContainerRequestEvent after scheduling " - + "task attempt", 1, containerRequestEvents.size()); + assertEquals(1, containerRequestEvents.size(), + "Expected one ContainerRequestEvent after scheduling task attempt"); return containerRequestEvents.get(0).getCapability(); } - @Test(expected=IllegalArgumentException.class) + @Test public void testReducerCustomResourceTypeWithInvalidUnit() { - initResourceTypes(); - EventHandler eventHandler = mock(EventHandler.class); - Clock clock = SystemClock.getInstance(); - JobConf jobConf = new JobConf(); - jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX - + CUSTOM_RESOURCE_NAME, "3z"); - createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf); + assertThrows(IllegalArgumentException.class, () -> { + initResourceTypes(); + EventHandler eventHandler = mock(EventHandler.class); + Clock clock = SystemClock.getInstance(); + JobConf jobConf = new JobConf(); + jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + + CUSTOM_RESOURCE_NAME, "3z"); + createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf); + }); } @Test @@ -1882,22 +1881,19 @@ public void testKillingTaskWhenContainerCleanup() { // move in two steps to the desired state (cannot get there directly) taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_DONE)); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_FINISHING_CONTAINER", - TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, - taImpl.getInternalState()); + assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_FINISHING_CONTAINER"); taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_TIMED_OUT)); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_CONTAINER_CLEANUP", - TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, - taImpl.getInternalState()); + assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_CONTAINER_CLEANUP"); taImpl.handle(new TaskAttemptKillEvent(mapTAId, "", true)); - assertEquals("Task attempt is not in KILLED state", - TaskAttemptState.KILLED, - taImpl.getState()); + assertEquals(TaskAttemptState.KILLED, + taImpl.getState(), "Task attempt is not in KILLED state"); } @Test @@ -1912,24 +1908,21 @@ public void testTooManyFetchFailureWhileContainerCleanup() { // move in two steps to the desired state (cannot get there directly) taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_DONE)); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_FINISHING_CONTAINER", - TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, - taImpl.getInternalState()); + assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_FINISHING_CONTAINER"); taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_TIMED_OUT)); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_CONTAINER_CLEANUP", - TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, - taImpl.getInternalState()); + assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_CONTAINER_CLEANUP"); taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(), reduceTAId, "Host")); - assertEquals("Task attempt is not in FAILED state", - TaskAttemptState.FAILED, - taImpl.getState()); - assertFalse("InternalError occurred", eventHandler.internalError); + assertEquals(TaskAttemptState.FAILED, + taImpl.getState(), "Task attempt is not in FAILED state"); + assertFalse(eventHandler.internalError, "InternalError occurred"); } private void initResourceTypes() { @@ -1951,17 +1944,15 @@ public void testTooManyFetchFailureWhileSuccessFinishing() throws Exception { taImpl.handle(new TaskAttemptEvent(taImpl.getID(), TaskAttemptEventType.TA_DONE)); - assertEquals("Task attempt's internal state is not " + - "SUCCESS_FINISHING_CONTAINER", - TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, - taImpl.getInternalState()); + assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER, + taImpl.getInternalState(), "Task attempt's internal state is not " + + "SUCCESS_FINISHING_CONTAINER"); taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(), reduceTAId, "Host")); - assertEquals("Task attempt is not in FAILED state", - TaskAttemptState.FAILED, - taImpl.getState()); - assertFalse("InternalError occurred", eventHandler.internalError); + assertEquals(TaskAttemptState.FAILED, + taImpl.getState(), "Task attempt is not in FAILED state"); + assertFalse(eventHandler.internalError, "InternalError occurred"); } private void setupTaskAttemptFinishingMonitor( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java index 585b949d7f9ea..fc9dcc83fa005 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java @@ -18,6 +18,9 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -27,8 +30,7 @@ import java.util.HashMap; import java.util.Map; -import org.junit.After; -import org.junit.Assert; +import org.junit.jupiter.api.AfterEach; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileStatus; @@ -58,12 +60,12 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.Test; +import org.junit.jupiter.api.Test; @SuppressWarnings({"rawtypes"}) public class TestTaskAttemptContainerRequest { - @After + @AfterEach public void cleanup() { UserGroupInformation.reset(); } @@ -72,8 +74,7 @@ public void cleanup() { public void testAttemptContainerRequest() throws Exception { final Text SECRET_KEY_ALIAS = new Text("secretkeyalias"); final byte[] SECRET_KEY = ("secretkey").getBytes(); - Map acls = - new HashMap(1); + Map acls = new HashMap<>(1); acls.put(ApplicationAccessType.VIEW_APP, "otheruser"); ApplicationId appId = ApplicationId.newInstance(1, 1); JobId jobId = MRBuilderUtils.newJobId(appId, 1); @@ -95,7 +96,7 @@ public void testAttemptContainerRequest() throws Exception { Credentials credentials = new Credentials(); credentials.addSecretKey(SECRET_KEY_ALIAS, SECRET_KEY); - Token jobToken = new Token( + Token jobToken = new Token<>( ("tokenid").getBytes(), ("tokenpw").getBytes(), new Text("tokenkind"), new Text("tokenservice")); @@ -114,7 +115,7 @@ public void testAttemptContainerRequest() throws Exception { mock(WrappedJvmID.class), taListener, credentials); - Assert.assertEquals("ACLs mismatch", acls, launchCtx.getApplicationACLs()); + assertEquals(acls, launchCtx.getApplicationACLs(), "ACLs mismatch"); Credentials launchCredentials = new Credentials(); DataInputByteBuffer dibb = new DataInputByteBuffer(); @@ -125,17 +126,14 @@ public void testAttemptContainerRequest() throws Exception { for (Token token : credentials.getAllTokens()) { Token launchToken = launchCredentials.getToken(token.getService()); - Assert.assertNotNull("Token " + token.getService() + " is missing", - launchToken); - Assert.assertEquals("Token " + token.getService() + " mismatch", - token, launchToken); + assertNotNull(launchToken, "Token " + token.getService() + " is missing"); + assertEquals(token, launchToken, "Token " + token.getService() + " mismatch"); } // verify the secret key is in the launch context - Assert.assertNotNull("Secret key missing", - launchCredentials.getSecretKey(SECRET_KEY_ALIAS)); - Assert.assertTrue("Secret key mismatch", Arrays.equals(SECRET_KEY, - launchCredentials.getSecretKey(SECRET_KEY_ALIAS))); + assertNotNull(launchCredentials.getSecretKey(SECRET_KEY_ALIAS), "Secret key missing"); + assertTrue(Arrays.equals(SECRET_KEY, + launchCredentials.getSecretKey(SECRET_KEY_ALIAS)), "Secret key mismatch"); } static public class StubbedFS extends RawLocalFileSystem { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java index 1225c4308cc84..da4d22b8b29ae 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.mapreduce.v2.app.job.impl; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -65,9 +65,9 @@ import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.SystemClock; -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; @@ -234,7 +234,7 @@ public boolean isMapTask() { } - @Before + @BeforeEach @SuppressWarnings("unchecked") public void setup() { dispatcher = new InlineDispatcher(); @@ -273,7 +273,7 @@ private MockTaskImpl createMockTask(TaskType taskType) { startCount, metrics, appContext, taskType); } - @After + @AfterEach public void teardown() { taskAttempts.clear(); } @@ -587,10 +587,10 @@ public void testFailureDuringTaskAttemptCommit() { mockTask.handle(new TaskTAttemptEvent(getLastAttempt().getAttemptId(), TaskEventType.T_ATTEMPT_SUCCEEDED)); - assertFalse("First attempt should not commit", - mockTask.canCommit(taskAttempts.get(0).getAttemptId())); - assertTrue("Second attempt should commit", - mockTask.canCommit(getLastAttempt().getAttemptId())); + assertFalse(mockTask.canCommit(taskAttempts.get(0).getAttemptId()), + "First attempt should not commit"); + assertTrue(mockTask.canCommit(getLastAttempt().getAttemptId()), + "Second attempt should commit"); assertTaskSucceededState(); } @@ -879,7 +879,7 @@ protected int getMaxAttempts() { baseAttempt.setProgress(1.0f); Counters taskCounters = mockTask.getCounters(); - assertEquals("wrong counters for task", specAttemptCounters, taskCounters); + assertEquals(specAttemptCounters, taskCounters, "wrong counters for task"); } public static class MockTaskAttemptEventHandler implements EventHandler { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java index dda93b682b3fc..15cd9e5b90d76 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java @@ -19,6 +19,9 @@ package org.apache.hadoop.mapreduce.v2.app.launcher; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import java.io.IOException; @@ -44,7 +47,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse; import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse; import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.ipc.Server; @@ -93,7 +95,8 @@ import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; import org.apache.hadoop.yarn.util.Records; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +110,8 @@ public class TestContainerLauncher { static final Logger LOG = LoggerFactory.getLogger(TestContainerLauncher.class); - @Test (timeout = 10000) + @Test + @Timeout(value = 10) public void testPoolSize() throws InterruptedException { ApplicationId appId = ApplicationId.newInstance(12345, 67); @@ -127,10 +131,10 @@ public void testPoolSize() throws InterruptedException { // No events yet assertThat(containerLauncher.initialPoolSize).isEqualTo( MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE); - Assert.assertEquals(0, threadPool.getPoolSize()); - Assert.assertEquals(containerLauncher.initialPoolSize, + assertEquals(0, threadPool.getPoolSize()); + assertEquals(containerLauncher.initialPoolSize, threadPool.getCorePoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertNull(containerLauncher.foundErrors); containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize; for (int i = 0; i < 10; i++) { @@ -141,8 +145,8 @@ public void testPoolSize() throws InterruptedException { ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH)); } waitForEvents(containerLauncher, 10); - Assert.assertEquals(10, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(10, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); // Same set of hosts, so no change containerLauncher.finishEventHandling = true; @@ -153,7 +157,7 @@ public void testPoolSize() throws InterruptedException { + ". Timeout is " + timeOut); Thread.sleep(1000); } - Assert.assertEquals(10, containerLauncher.numEventsProcessed.get()); + assertEquals(10, containerLauncher.numEventsProcessed.get()); containerLauncher.finishEventHandling = false; for (int i = 0; i < 10; i++) { ContainerId containerId = ContainerId.newContainerId(appAttemptId, @@ -165,8 +169,8 @@ public void testPoolSize() throws InterruptedException { ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH)); } waitForEvents(containerLauncher, 20); - Assert.assertEquals(10, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(10, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); // Different hosts, there should be an increase in core-thread-pool size to // 21(11hosts+10buffer) @@ -179,8 +183,8 @@ public void testPoolSize() throws InterruptedException { containerId, "host11:1234", null, ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH)); waitForEvents(containerLauncher, 21); - Assert.assertEquals(11, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(11, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); containerLauncher.stop(); @@ -194,7 +198,8 @@ public void testPoolSize() throws InterruptedException { assertThat(containerLauncher.initialPoolSize).isEqualTo(20); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testPoolLimits() throws InterruptedException { ApplicationId appId = ApplicationId.newInstance(12345, 67); ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance( @@ -222,8 +227,8 @@ public void testPoolLimits() throws InterruptedException { ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH)); } waitForEvents(containerLauncher, 10); - Assert.assertEquals(10, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(10, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); // 4 more different hosts, but thread pool size should be capped at 12 containerLauncher.expectedCorePoolSize = 12 ; @@ -233,14 +238,14 @@ public void testPoolLimits() throws InterruptedException { ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH)); } waitForEvents(containerLauncher, 12); - Assert.assertEquals(12, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(12, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); // Make some threads ideal so that remaining events are also done. containerLauncher.finishEventHandling = true; waitForEvents(containerLauncher, 14); - Assert.assertEquals(12, threadPool.getPoolSize()); - Assert.assertNull(containerLauncher.foundErrors); + assertEquals(12, threadPool.getPoolSize()); + assertNull(containerLauncher.foundErrors); containerLauncher.stop(); } @@ -254,11 +259,11 @@ private void waitForEvents(CustomContainerLauncher containerLauncher, + ". It is now " + containerLauncher.numEventsProcessing.get()); Thread.sleep(1000); } - Assert.assertEquals(expectedNumEvents, - containerLauncher.numEventsProcessing.get()); + assertEquals(expectedNumEvents, containerLauncher.numEventsProcessing.get()); } - @Test(timeout = 15000) + @Test + @Timeout(value = 15) public void testSlowNM() throws Exception { conf = new Configuration(); @@ -290,15 +295,14 @@ public void testSlowNM() throws Exception { app.waitForState(job, JobState.RUNNING); Map tasks = job.getTasks(); - Assert.assertEquals("Num tasks is not correct", 1, tasks.size()); + assertEquals(1, tasks.size(), "Num tasks is not correct"); Task task = tasks.values().iterator().next(); app.waitForState(task, TaskState.SCHEDULED); Map attempts = tasks.values().iterator() .next().getAttempts(); - Assert.assertEquals("Num attempts is not correct", maxAttempts, - attempts.size()); + assertEquals(maxAttempts, attempts.size(), "Num attempts is not correct"); TaskAttempt attempt = attempts.values().iterator().next(); app.waitForInternalState((TaskAttemptImpl) attempt, @@ -309,11 +313,10 @@ public void testSlowNM() throws Exception { String diagnostics = attempt.getDiagnostics().toString(); LOG.info("attempt.getDiagnostics: " + diagnostics); - Assert.assertTrue(diagnostics.contains("Container launch failed for " + assertTrue(diagnostics.contains("Container launch failed for " + "container_0_0000_01_000000 : ")); - Assert - .assertTrue(diagnostics - .contains("java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel")); + assertTrue(diagnostics.contains( + "java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel")); } finally { server.stop(); @@ -440,7 +443,7 @@ public StartContainersResponse startContainers(StartContainersRequest requests) MRApp.newContainerTokenIdentifier(request.getContainerToken()); // Validate that the container is what RM is giving. - Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT, + assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT, containerTokenIdentifier.getNmHostAddress()); StartContainersResponse response = recordFactory diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java index 88ba8943ceb3f..d98b8031ec99f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java @@ -79,8 +79,9 @@ import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.ArgumentCaptor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,7 +95,7 @@ public class TestContainerLauncherImpl { private Map serviceResponse = new HashMap(); - @Before + @BeforeEach public void setup() throws IOException { serviceResponse.clear(); serviceResponse.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID, @@ -168,7 +169,8 @@ public static TaskAttemptId makeTaskAttemptId(long ts, int appId, int taskId, return MRBuilderUtils.newTaskAttemptId(tID, id); } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testHandle() throws Exception { LOG.info("STARTING testHandle"); AppContext mockContext = mock(AppContext.class); @@ -226,7 +228,8 @@ public void testHandle() throws Exception { } } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testOutOfOrder() throws Exception { LOG.info("STARTING testOutOfOrder"); AppContext mockContext = mock(AppContext.class); @@ -300,7 +303,8 @@ public void testOutOfOrder() throws Exception { } } - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testMyShutdown() throws Exception { LOG.info("in test Shutdown"); @@ -352,7 +356,8 @@ public void testMyShutdown() throws Exception { } @SuppressWarnings({ "rawtypes", "unchecked" }) - @Test(timeout = 5000) + @Test + @Timeout(value = 5) public void testContainerCleaned() throws Exception { LOG.info("STARTING testContainerCleaned"); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java index de4977205b045..eef1a4a10835f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java @@ -18,6 +18,10 @@ package org.apache.hadoop.mapreduce.v2.app.local; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -69,8 +73,7 @@ import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; public class TestLocalContainerAllocator { @@ -90,7 +93,7 @@ public void testRMConnectionRetry() throws Exception { lca.start(); try { lca.heartbeat(); - Assert.fail("heartbeat was supposed to throw"); + fail("heartbeat was supposed to throw"); } catch (YarnException e) { // YarnException is expected } finally { @@ -104,7 +107,7 @@ public void testRMConnectionRetry() throws Exception { lca.start(); try { lca.heartbeat(); - Assert.fail("heartbeat was supposed to throw"); + fail("heartbeat was supposed to throw"); } catch (YarnRuntimeException e) { // YarnRuntimeException is expected } finally { @@ -172,14 +175,13 @@ public Void run() throws Exception { } } - Assert.assertEquals("too many AMRM tokens", 1, tokenCount); - Assert.assertArrayEquals("token identifier not updated", - newToken.getIdentifier(), ugiToken.getIdentifier()); - Assert.assertArrayEquals("token password not updated", - newToken.getPassword(), ugiToken.getPassword()); - Assert.assertEquals("AMRM token service not updated", - new Text(ClientRMProxy.getAMRMTokenService(conf)), - ugiToken.getService()); + assertEquals(1, tokenCount, "too many AMRM tokens"); + assertArrayEquals(newToken.getIdentifier(), ugiToken.getIdentifier(), + "token identifier not updated"); + assertArrayEquals(newToken.getPassword(), ugiToken.getPassword(), + "token password not updated"); + assertEquals(new Text(ClientRMProxy.getAMRMTokenService(conf)), + ugiToken.getService(), "AMRM token service not updated"); } @Test @@ -202,7 +204,7 @@ public void testAllocatedContainerResourceIsNotNull() { verify(eventHandler, times(1)).handle(containerAssignedCaptor.capture()); Container container = containerAssignedCaptor.getValue().getContainer(); Resource containerResource = container.getResource(); - Assert.assertNotNull(containerResource); + assertNotNull(containerResource); assertThat(containerResource.getMemorySize()).isEqualTo(0); assertThat(containerResource.getVirtualCores()).isEqualTo(0); } @@ -282,8 +284,7 @@ public FinishApplicationMasterResponse finishApplicationMaster( @Override public AllocateResponse allocate(AllocateRequest request) throws YarnException, IOException { - Assert.assertEquals("response ID mismatch", - responseId, request.getResponseId()); + assertEquals(responseId, request.getResponseId(), "response ID mismatch"); ++responseId; org.apache.hadoop.yarn.api.records.Token yarnToken = null; if (amToken != null) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java index 3fd4cb028a5a8..6f50c611a01d8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java @@ -25,14 +25,15 @@ import static org.apache.hadoop.test.MetricsAsserts.*; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class TestMRAppMetrics { - @After + @AfterEach public void tearDown() { DefaultMetricsSystem.shutdown(); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java index bbfac68ae9eaa..a81c32f724eaf 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java @@ -23,7 +23,8 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator.AllocatorRunnable; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.util.Clock; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.stubbing.Answer; import static org.mockito.Mockito.doThrow; @@ -45,7 +46,8 @@ protected void heartbeat() throws Exception { } } - @Test(timeout = 6000) + @Test + @Timeout(value = 6) public void testRMContainerAllocatorExceptionIsHandled() throws Exception { ClientService mockClientService = mock(ClientService.class); AppContext mockContext = mock(AppContext.class); @@ -66,7 +68,8 @@ public void testRMContainerAllocatorExceptionIsHandled() throws Exception { testRunnable.run(); } - @Test(timeout = 2000) + @Test + @Timeout(value = 2) public void testRMContainerAllocatorYarnRuntimeExceptionIsHandled() throws Exception { ClientService mockClientService = mock(ClientService.class); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java index 4c64b11372cc5..7742aa978f720 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java @@ -20,7 +20,15 @@ import static org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestCreator.createRequest; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyFloat; import static org.mockito.ArgumentMatchers.anyInt; @@ -149,10 +157,10 @@ import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.function.Supplier; import org.mockito.InOrder; @@ -167,7 +175,7 @@ public class TestRMContainerAllocator { static final RecordFactory recordFactory = RecordFactoryProvider .getRecordFactory(null); - @Before + @BeforeEach public void setup() { MyContainerAllocator.getJobUpdatedNodeEvents().clear(); MyContainerAllocator.getTaskAttemptKillEvents().clear(); @@ -176,7 +184,7 @@ public void setup() { UserGroupInformation.setLoginUser(null); } - @After + @AfterEach public void tearDown() { DefaultMetricsSystem.shutdown(); } @@ -231,8 +239,8 @@ public void testSimple() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - Assert.assertEquals(4, rm.getMyFifoScheduler().lastAsk.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); + assertEquals(4, rm.getMyFifoScheduler().lastAsk.size()); // send another request with different resource and priority ContainerRequestEvent event3 = ContainerRequestCreator.createRequest(jobId, @@ -243,8 +251,8 @@ public void testSimple() throws Exception { // as nodes are not added, no allocations assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); - Assert.assertEquals(3, rm.getMyFifoScheduler().lastAsk.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); + assertEquals(3, rm.getMyFifoScheduler().lastAsk.size()); // update resources in scheduler nodeManager1.nodeHeartbeat(true); // Node heartbeat @@ -254,14 +262,14 @@ public void testSimple() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0, rm.getMyFifoScheduler().lastAsk.size()); + assertEquals(0, rm.getMyFifoScheduler().lastAsk.size()); checkAssignments(new ContainerRequestEvent[] {event1, event2, event3}, assigned, false); // check that the assigned container requests are cancelled allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(5, rm.getMyFifoScheduler().lastAsk.size()); + assertEquals(5, rm.getMyFifoScheduler().lastAsk.size()); } @Test @@ -323,7 +331,7 @@ public void testMapNodeLocality() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // update resources in scheduler // Node heartbeat from rack-local first. This makes node h3 the first in the @@ -342,7 +350,7 @@ public void testMapNodeLocality() throws Exception { for(TaskAttemptContainerAssignedEvent event : assigned) { if(event.getTaskAttemptID().equals(event3.getAttemptID())) { assigned.remove(event); - Assert.assertEquals("h3", event.getContainer().getNodeId().getHost()); + assertEquals("h3", event.getContainer().getNodeId().getHost()); break; } } @@ -402,7 +410,7 @@ public void testResource() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // update resources in scheduler nodeManager1.nodeHeartbeat(true); // Node heartbeat @@ -416,7 +424,8 @@ public void testResource() throws Exception { assigned, false); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testReducerRampdownDiagnostics() throws Exception { LOG.info("Running tesReducerRampdownDiagnostics"); @@ -467,11 +476,12 @@ public void testReducerRampdownDiagnostics() throws Exception { } final String killEventMessage = allocator.getTaskAttemptKillEvents().get(0) .getMessage(); - Assert.assertTrue("No reducer rampDown preemption message", - killEventMessage.contains(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC)); + assertTrue(killEventMessage.contains(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC), + "No reducer rampDown preemption message"); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testPreemptReducers() throws Exception { LOG.info("Running testPreemptReducers"); @@ -514,11 +524,12 @@ public void testPreemptReducers() throws Exception { mock(Container.class)); allocator.preemptReducesIfNeeded(); - Assert.assertEquals("The reducer is not preempted", - 1, assignedRequests.preemptionWaitingReduces.size()); + assertEquals(1, assignedRequests.preemptionWaitingReduces.size(), + "The reducer is not preempted"); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testNonAggressivelyPreemptReducers() throws Exception { LOG.info("Running testNonAggressivelyPreemptReducers"); @@ -571,16 +582,17 @@ public void testNonAggressivelyPreemptReducers() throws Exception { clock.setTime(clock.getTime() + 1); allocator.preemptReducesIfNeeded(); - Assert.assertEquals("The reducer is aggressively preeempted", 0, - assignedRequests.preemptionWaitingReduces.size()); + assertEquals(0, assignedRequests.preemptionWaitingReduces.size(), + "The reducer is aggressively preempted"); clock.setTime(clock.getTime() + (preemptThreshold) * 1000); allocator.preemptReducesIfNeeded(); - Assert.assertEquals("The reducer is not preeempted", 1, - assignedRequests.preemptionWaitingReduces.size()); + assertEquals(1, assignedRequests.preemptionWaitingReduces.size(), + "The reducer is not preempted"); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testUnconditionalPreemptReducers() throws Exception { LOG.info("Running testForcePreemptReducers"); @@ -635,16 +647,17 @@ public void testUnconditionalPreemptReducers() throws Exception { clock.setTime(clock.getTime() + 1); allocator.preemptReducesIfNeeded(); - Assert.assertEquals("The reducer is preeempted too soon", 0, - assignedRequests.preemptionWaitingReduces.size()); + assertEquals(0, assignedRequests.preemptionWaitingReduces.size(), + "The reducer is preempted too soon"); clock.setTime(clock.getTime() + 1000 * forcePreemptThresholdSecs); allocator.preemptReducesIfNeeded(); - Assert.assertEquals("The reducer is not preeempted", 1, - assignedRequests.preemptionWaitingReduces.size()); + assertEquals(1, assignedRequests.preemptionWaitingReduces.size(), + "The reducer is not preempted"); } - @Test(timeout = 30000) + @Test + @Timeout(value = 30) public void testExcessReduceContainerAssign() throws Exception { final Configuration conf = new Configuration(); conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART, 0.0f); @@ -743,7 +756,7 @@ protected ApplicationMasterProtocol createSchedulerProxy() { allocator.schedule(); // verify all of the host-specific asks were sent plus one for the // default rack and one for the ANY request - Assert.assertEquals(3, mockScheduler.lastAsk.size()); + assertEquals(3, mockScheduler.lastAsk.size()); // verify ResourceRequest sent for MAP have appropriate node // label expression as per the configuration validateLabelsRequests(mockScheduler.lastAsk.get(0), false); @@ -754,7 +767,7 @@ protected ApplicationMasterProtocol createSchedulerProxy() { ContainerId cid0 = mockScheduler.assignContainer("map", false); allocator.schedule(); // default rack and one for the ANY request - Assert.assertEquals(3, mockScheduler.lastAsk.size()); + assertEquals(3, mockScheduler.lastAsk.size()); validateLabelsRequests(mockScheduler.lastAsk.get(0), true); validateLabelsRequests(mockScheduler.lastAsk.get(1), true); validateLabelsRequests(mockScheduler.lastAsk.get(2), true); @@ -769,14 +782,14 @@ private void validateLabelsRequests(ResourceRequest resourceRequest, case "map": case "reduce": case NetworkTopology.DEFAULT_RACK: - Assert.assertNull(resourceRequest.getNodeLabelExpression()); + assertNull(resourceRequest.getNodeLabelExpression()); break; case "*": - Assert.assertEquals(isReduce ? "ReduceNodes" : "MapNodes", + assertEquals(isReduce ? "ReduceNodes" : "MapNodes", resourceRequest.getNodeLabelExpression()); break; default: - Assert.fail("Invalid resource location " + fail("Invalid resource location " + resourceRequest.getResourceName()); } } @@ -930,7 +943,7 @@ public void testMapReduceScheduling() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // update resources in scheduler nodeManager1.nodeHeartbeat(true); // Node heartbeat @@ -945,8 +958,8 @@ public void testMapReduceScheduling() throws Exception { // validate that no container is assigned to h1 as it doesn't have 2048 for (TaskAttemptContainerAssignedEvent assig : assigned) { - Assert.assertFalse("Assigned count not correct", "h1".equals(assig - .getContainer().getNodeId().getHost())); + assertNotEquals("h1", assig.getContainer().getNodeId().getHost(), + "Assigned count not correct"); } } @@ -1037,7 +1050,7 @@ protected ContainerAllocator createContainerAllocator( }; }; - Assert.assertEquals(0.0, rmApp.getProgress(), 0.0); + assertEquals(0.0, rmApp.getProgress(), 0.0); mrApp.submit(conf); Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next() @@ -1076,23 +1089,23 @@ protected ContainerAllocator createContainerAllocator( allocator.schedule(); // Send heartbeat rm.drainEvents(); - Assert.assertEquals(0.05f, job.getProgress(), 0.001f); - Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f); + assertEquals(0.05f, job.getProgress(), 0.001f); + assertEquals(0.05f, rmApp.getProgress(), 0.001f); // Finish off 1 map. Iterator it = job.getTasks().values().iterator(); finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 1); allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.095f, job.getProgress(), 0.001f); - Assert.assertEquals(0.095f, rmApp.getProgress(), 0.001f); + assertEquals(0.095f, job.getProgress(), 0.001f); + assertEquals(0.095f, rmApp.getProgress(), 0.001f); // Finish off 7 more so that map-progress is 80% finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 7); allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.41f, job.getProgress(), 0.001f); - Assert.assertEquals(0.41f, rmApp.getProgress(), 0.001f); + assertEquals(0.41f, job.getProgress(), 0.001f); + assertEquals(0.41f, rmApp.getProgress(), 0.001f); // Finish off the 2 remaining maps finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 2); @@ -1116,16 +1129,16 @@ protected ContainerAllocator createContainerAllocator( allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.59f, job.getProgress(), 0.001f); - Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f); + assertEquals(0.59f, job.getProgress(), 0.001f); + assertEquals(0.59f, rmApp.getProgress(), 0.001f); // Finish off the remaining 8 reduces. finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 8); allocator.schedule(); rm.drainEvents(); // Remaining is JobCleanup - Assert.assertEquals(0.95f, job.getProgress(), 0.001f); - Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f); + assertEquals(0.95f, job.getProgress(), 0.001f); + assertEquals(0.95f, rmApp.getProgress(), 0.001f); } private void finishNextNTasks(DrainDispatcher rmDispatcher, MockNM node, @@ -1189,7 +1202,7 @@ protected ContainerAllocator createContainerAllocator( }; }; - Assert.assertEquals(0.0, rmApp.getProgress(), 0.0); + assertEquals(0.0, rmApp.getProgress(), 0.0); mrApp.submit(conf); Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next() @@ -1224,8 +1237,8 @@ protected ContainerAllocator createContainerAllocator( allocator.schedule(); // Send heartbeat rm.drainEvents(); - Assert.assertEquals(0.05f, job.getProgress(), 0.001f); - Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f); + assertEquals(0.05f, job.getProgress(), 0.001f); + assertEquals(0.05f, rmApp.getProgress(), 0.001f); Iterator it = job.getTasks().values().iterator(); @@ -1233,22 +1246,22 @@ protected ContainerAllocator createContainerAllocator( finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 1); allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.14f, job.getProgress(), 0.001f); - Assert.assertEquals(0.14f, rmApp.getProgress(), 0.001f); + assertEquals(0.14f, job.getProgress(), 0.001f); + assertEquals(0.14f, rmApp.getProgress(), 0.001f); // Finish off 5 more map so that map-progress is 60% finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 5); allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.59f, job.getProgress(), 0.001f); - Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f); + assertEquals(0.59f, job.getProgress(), 0.001f); + assertEquals(0.59f, rmApp.getProgress(), 0.001f); // Finish off remaining map so that map-progress is 100% finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 4); allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0.95f, job.getProgress(), 0.001f); - Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f); + assertEquals(0.95f, job.getProgress(), 0.001f); + assertEquals(0.95f, rmApp.getProgress(), 0.001f); } @Test @@ -1299,17 +1312,17 @@ public void testUpdatedNodes() throws Exception { nm1.nodeHeartbeat(true); rm.drainEvents(); - Assert.assertEquals(1, allocator.getJobUpdatedNodeEvents().size()); - Assert.assertEquals(3, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size()); + assertEquals(1, allocator.getJobUpdatedNodeEvents().size()); + assertEquals(3, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size()); allocator.getJobUpdatedNodeEvents().clear(); // get the assignment assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(1, assigned.size()); - Assert.assertEquals(nm1.getNodeId(), assigned.get(0).getContainer().getNodeId()); + assertEquals(1, assigned.size()); + assertEquals(nm1.getNodeId(), assigned.get(0).getContainer().getNodeId()); // no updated nodes reported - Assert.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty()); - Assert.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty()); + assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty()); + assertTrue(allocator.getTaskAttemptKillEvents().isEmpty()); // mark nodes bad nm1.nodeHeartbeat(false); @@ -1319,23 +1332,23 @@ public void testUpdatedNodes() throws Exception { // schedule response returns updated nodes assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0, assigned.size()); + assertEquals(0, assigned.size()); // updated nodes are reported - Assert.assertEquals(1, allocator.getJobUpdatedNodeEvents().size()); - Assert.assertEquals(1, allocator.getTaskAttemptKillEvents().size()); - Assert.assertEquals(2, + assertEquals(1, allocator.getJobUpdatedNodeEvents().size()); + assertEquals(1, allocator.getTaskAttemptKillEvents().size()); + assertEquals(2, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size()); - Assert.assertEquals(attemptId, + assertEquals(attemptId, allocator.getTaskAttemptKillEvents().get(0).getTaskAttemptID()); allocator.getJobUpdatedNodeEvents().clear(); allocator.getTaskAttemptKillEvents().clear(); assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals(0, assigned.size()); + assertEquals(0, assigned.size()); // no updated nodes reported - Assert.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty()); - Assert.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty()); + assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty()); + assertTrue(allocator.getTaskAttemptKillEvents().isEmpty()); } @Test @@ -1404,7 +1417,7 @@ public void testBlackListedNodes() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // Send events to blacklist nodes h1 and h2 ContainerFailedEvent f1 = createFailEvent(jobId, 1, "h1", false); @@ -1418,9 +1431,9 @@ public void testBlackListedNodes() throws Exception { rm.drainEvents(); assigned = allocator.schedule(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); assertBlacklistAdditionsAndRemovals(2, 0, rm); // mark h1/h2 as bad nodes @@ -1431,7 +1444,7 @@ public void testBlackListedNodes() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); assertBlacklistAdditionsAndRemovals(0, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); nodeManager3.nodeHeartbeat(true); // Node heartbeat rm.drainEvents(); @@ -1439,12 +1452,12 @@ public void testBlackListedNodes() throws Exception { rm.drainEvents(); assertBlacklistAdditionsAndRemovals(0, 0, rm); - Assert.assertTrue("No of assignments must be 3", assigned.size() == 3); + assertEquals(3, assigned.size(), "No of assignments must be 3"); // validate that all containers are assigned to h3 for (TaskAttemptContainerAssignedEvent assig : assigned) { - Assert.assertTrue("Assigned container host not correct", "h3".equals(assig - .getContainer().getNodeId().getHost())); + assertEquals("h3", assig.getContainer().getNodeId().getHost(), + "Assigned container host not correct"); } } @@ -1489,7 +1502,7 @@ public void testIgnoreBlacklisting() throws Exception { assigned = getContainerOnHost(jobId, 1, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); LOG.info("Failing container _1 on H1 (Node should be blacklisted and" + " ignore blacklisting enabled"); @@ -1504,47 +1517,47 @@ public void testIgnoreBlacklisting() throws Exception { assigned = getContainerOnHost(jobId, 2, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 1, 0, 0, 1, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // Known=1, blacklisted=1, ignore should be true - assign 1 assigned = getContainerOnHost(jobId, 2, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); nodeManagers[nmNum] = registerNodeManager(nmNum++, rm); // Known=2, blacklisted=1, ignore should be true - assign 1 anyway. assigned = getContainerOnHost(jobId, 3, 1024, new String[] {"h2"}, nodeManagers[1], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); nodeManagers[nmNum] = registerNodeManager(nmNum++, rm); // Known=3, blacklisted=1, ignore should be true - assign 1 anyway. assigned = getContainerOnHost(jobId, 4, 1024, new String[] {"h3"}, nodeManagers[2], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); // Known=3, blacklisted=1, ignore should be true - assign 1 assigned = getContainerOnHost(jobId, 5, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); nodeManagers[nmNum] = registerNodeManager(nmNum++, rm); // Known=4, blacklisted=1, ignore should be false - assign 1 anyway assigned = getContainerOnHost(jobId, 6, 1024, new String[] {"h4"}, nodeManagers[3], allocator, 0, 0, 1, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); // Test blacklisting re-enabled. // Known=4, blacklisted=1, ignore should be false - no assignment on h1 assigned = getContainerOnHost(jobId, 7, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // RMContainerRequestor would have created a replacement request. // Blacklist h2 @@ -1557,20 +1570,20 @@ public void testIgnoreBlacklisting() throws Exception { assigned = getContainerOnHost(jobId, 8, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 1, 0, 0, 2, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // Known=4, blacklisted=2, ignore should be true. Should assign 2 // containers. assigned = getContainerOnHost(jobId, 8, 1024, new String[] {"h1"}, nodeManagers[0], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 2", 2, assigned.size()); + assertEquals(2, assigned.size(), "No of assignments must be 2"); // Known=4, blacklisted=2, ignore should be true. assigned = getContainerOnHost(jobId, 9, 1024, new String[] {"h2"}, nodeManagers[1], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); // Test blacklist while ignore blacklisting enabled ContainerFailedEvent f3 = createFailEvent(jobId, 4, "h3", false); @@ -1581,7 +1594,7 @@ public void testIgnoreBlacklisting() throws Exception { assigned = getContainerOnHost(jobId, 10, 1024, new String[] {"h3"}, nodeManagers[2], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); // Assign on 5 more nodes - to re-enable blacklisting for (int i = 0; i < 5; i++) { @@ -1590,14 +1603,14 @@ public void testIgnoreBlacklisting() throws Exception { getContainerOnHost(jobId, 11 + i, 1024, new String[] {String.valueOf(5 + i)}, nodeManagers[4 + i], allocator, 0, 0, (i == 4 ? 3 : 0), 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); } // Test h3 (blacklisted while ignoring blacklisting) is blacklisted. assigned = getContainerOnHost(jobId, 20, 1024, new String[] {"h3"}, nodeManagers[2], allocator, 0, 0, 0, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); } private MockNM registerNodeManager(int i, MyResourceManager rm) @@ -1624,7 +1637,7 @@ List getContainerOnHost(JobId jobId, rm.drainEvents(); assertBlacklistAdditionsAndRemovals( expectedAdditions1, expectedRemovals1, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // Heartbeat from the required nodeManager mockNM.nodeHeartbeat(true); @@ -1689,7 +1702,7 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { // as nodes are not added, no allocations List assigned = allocator.schedule(); rm.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); LOG.info("h1 Heartbeat (To actually schedule the containers)"); // update resources in scheduler @@ -1700,7 +1713,7 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); assertBlacklistAdditionsAndRemovals(0, 0, rm); - Assert.assertEquals("No of assignments must be 1", 1, assigned.size()); + assertEquals(1, assigned.size(), "No of assignments must be 1"); LOG.info("Failing container _1 on H1 (should blacklist the node)"); // Send events to blacklist nodes h1 and h2 @@ -1718,7 +1731,7 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); assertBlacklistAdditionsAndRemovals(1, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); // send another request with different resource and priority ContainerRequestEvent event3 = @@ -1739,7 +1752,7 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); assertBlacklistAdditionsAndRemovals(0, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); //RMContainerAllocator gets assigned a p:5 on a blacklisted node. @@ -1748,7 +1761,7 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { assigned = allocator.schedule(); rm.drainEvents(); assertBlacklistAdditionsAndRemovals(0, 0, rm); - Assert.assertEquals("No of assignments must be 0", 0, assigned.size()); + assertEquals(0, assigned.size(), "No of assignments must be 0"); //Hearbeat from H3 to schedule on this host. LOG.info("h3 Heartbeat (To re-schedule the containers)"); @@ -1767,27 +1780,27 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception { " with priority " + assig.getContainer().getPriority()); } - Assert.assertEquals("No of assignments must be 2", 2, assigned.size()); + assertEquals(2, assigned.size(), "No of assignments must be 2"); // validate that all containers are assigned to h3 for (TaskAttemptContainerAssignedEvent assig : assigned) { - Assert.assertEquals("Assigned container " + assig.getContainer().getId() - + " host not correct", "h3", assig.getContainer().getNodeId().getHost()); + assertEquals("h3", assig.getContainer().getNodeId().getHost(), + "Assigned container " + assig.getContainer().getId() + " host not correct"); } } private static void assertBlacklistAdditionsAndRemovals( int expectedAdditions, int expectedRemovals, MyResourceManager rm) { - Assert.assertEquals(expectedAdditions, + assertEquals(expectedAdditions, rm.getMyFifoScheduler().lastBlacklistAdditions.size()); - Assert.assertEquals(expectedRemovals, + assertEquals(expectedRemovals, rm.getMyFifoScheduler().lastBlacklistRemovals.size()); } private static void assertAsksAndReleases(int expectedAsk, int expectedRelease, MyResourceManager rm) { - Assert.assertEquals(expectedAsk, rm.getMyFifoScheduler().lastAsk.size()); - Assert.assertEquals(expectedRelease, + assertEquals(expectedAsk, rm.getMyFifoScheduler().lastAsk.size()); + assertEquals(expectedRelease, rm.getMyFifoScheduler().lastRelease.size()); } @@ -1930,17 +1943,16 @@ private ContainerAllocatorEvent createDeallocateEvent(JobId jobId, private void checkAssignments(ContainerRequestEvent[] requests, List assignments, boolean checkHostMatch) { - Assert.assertNotNull("Container not assigned", assignments); - Assert.assertEquals("Assigned count not correct", requests.length, - assignments.size()); + assertNotNull(assignments, "Container not assigned"); + assertEquals(requests.length, assignments.size(), "Assigned count not correct"); // check for uniqueness of containerIDs - Set containerIds = new HashSet(); + Set containerIds = new HashSet<>(); for (TaskAttemptContainerAssignedEvent assigned : assignments) { containerIds.add(assigned.getContainer().getId()); } - Assert.assertEquals("Assigned containers must be different", assignments - .size(), containerIds.size()); + assertEquals(assignments.size(), containerIds.size(), + "Assigned containers must be different"); // check for all assignment for (ContainerRequestEvent req : requests) { @@ -1957,14 +1969,14 @@ private void checkAssignments(ContainerRequestEvent[] requests, private void checkAssignment(ContainerRequestEvent request, TaskAttemptContainerAssignedEvent assigned, boolean checkHostMatch) { - Assert.assertNotNull("Nothing assigned to attempt " - + request.getAttemptID(), assigned); - Assert.assertEquals("assigned to wrong attempt", request.getAttemptID(), - assigned.getTaskAttemptID()); + assertNotNull(assigned, "Nothing assigned to attempt " + + request.getAttemptID()); + assertEquals(request.getAttemptID(), assigned.getTaskAttemptID(), + "assigned to wrong attempt"); if (checkHostMatch) { - Assert.assertTrue("Not assigned to requested host", Arrays.asList( + assertTrue(Arrays.asList( request.getHosts()).contains( - assigned.getContainer().getNodeId().getHost())); + assigned.getContainer().getNodeId().getHost()), "Not assigned to requested host"); } } @@ -2351,13 +2363,13 @@ public void testCompletedTasksRecalculateSchedule() throws Exception { allocator.recalculatedReduceSchedule = false; allocator.schedule(); - Assert.assertFalse("Unexpected recalculate of reduce schedule", - allocator.recalculatedReduceSchedule); + assertFalse(allocator.recalculatedReduceSchedule, + "Unexpected recalculate of reduce schedule"); doReturn(1).when(job).getCompletedMaps(); allocator.schedule(); - Assert.assertTrue("Expected recalculate of reduce schedule", - allocator.recalculatedReduceSchedule); + assertTrue(allocator.recalculatedReduceSchedule, + "Expected recalculate of reduce schedule"); } @Test @@ -2395,14 +2407,14 @@ protected synchronized void heartbeat() throws Exception { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals(5, allocator.getLastHeartbeatTime()); + assertEquals(5, allocator.getLastHeartbeatTime()); clock.setTime(7); timeToWaitMs = 5000; while (allocator.getLastHeartbeatTime() != 7 && timeToWaitMs > 0) { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals(7, allocator.getLastHeartbeatTime()); + assertEquals(7, allocator.getLastHeartbeatTime()); final AtomicBoolean callbackCalled = new AtomicBoolean(false); allocator.runOnNextHeartbeat(new Runnable() { @@ -2417,8 +2429,8 @@ public void run() { Thread.sleep(10); timeToWaitMs -= 10; } - Assert.assertEquals(8, allocator.getLastHeartbeatTime()); - Assert.assertTrue(callbackCalled.get()); + assertEquals(8, allocator.getLastHeartbeatTime()); + assertTrue(callbackCalled.get()); } @Test @@ -2446,12 +2458,12 @@ public void testCompletedContainerEvent() { TaskAttemptEvent event = allocator.createContainerFinishedEvent(status, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, + assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, event.getType()); TaskAttemptEvent abortedEvent = allocator.createContainerFinishedEvent( abortedStatus, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType()); + assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType()); // PREEMPTED ContainerId containerId2 = @@ -2464,12 +2476,12 @@ public void testCompletedContainerEvent() { TaskAttemptEvent event2 = allocator.createContainerFinishedEvent(status2, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, + assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, event2.getType()); TaskAttemptEvent abortedEvent2 = allocator.createContainerFinishedEvent( preemptedStatus, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent2.getType()); + assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent2.getType()); // KILLED_BY_CONTAINER_SCHEDULER ContainerId containerId3 = @@ -2483,12 +2495,12 @@ public void testCompletedContainerEvent() { TaskAttemptEvent event3 = allocator.createContainerFinishedEvent(status3, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, + assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED, event3.getType()); TaskAttemptEvent abortedEvent3 = allocator.createContainerFinishedEvent( killedByContainerSchedulerStatus, attemptId); - Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent3.getType()); + assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent3.getType()); } @Test @@ -2529,9 +2541,9 @@ protected ContainerAllocator createContainerAllocator( MyContainerAllocator allocator = (MyContainerAllocator) mrApp.getContainerAllocator(); amDispatcher.await(); - Assert.assertTrue(allocator.isApplicationMasterRegistered()); + assertTrue(allocator.isApplicationMasterRegistered()); mrApp.stop(); - Assert.assertTrue(allocator.isUnregistered()); + assertTrue(allocator.isUnregistered()); } // Step-1 : AM send allocate request for 2 ContainerRequests and 1 @@ -2611,8 +2623,8 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() List assignedContainers = allocator.schedule(); rm1.drainEvents(); - Assert.assertEquals("No of assignments must be 0", 0, - assignedContainers.size()); + assertEquals(0, + assignedContainers.size(), "No of assignments must be 0"); // Why ask is 3, not 4? --> ask from blacklisted node h2 is removed assertAsksAndReleases(3, 0, rm1); assertBlacklistAdditionsAndRemovals(1, 0, rm1); @@ -2623,14 +2635,14 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() // Step-2 : 2 containers are allocated by RM. assignedContainers = allocator.schedule(); rm1.drainEvents(); - Assert.assertEquals("No of assignments must be 2", 2, - assignedContainers.size()); + assertEquals(2, + assignedContainers.size(), "No of assignments must be 2"); assertAsksAndReleases(0, 0, rm1); assertBlacklistAdditionsAndRemovals(0, 0, rm1); assignedContainers = allocator.schedule(); - Assert.assertEquals("No of assignments must be 0", 0, - assignedContainers.size()); + assertEquals(0, + assignedContainers.size(), "No of assignments must be 0"); assertAsksAndReleases(3, 0, rm1); assertBlacklistAdditionsAndRemovals(0, 0, rm1); @@ -2649,8 +2661,8 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() allocator.sendDeallocate(deallocate1); assignedContainers = allocator.schedule(); - Assert.assertEquals("No of assignments must be 0", 0, - assignedContainers.size()); + assertEquals(0, + assignedContainers.size(), "No of assignments must be 0"); assertAsksAndReleases(3, 1, rm1); assertBlacklistAdditionsAndRemovals(0, 0, rm1); @@ -2662,7 +2674,7 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() // NM should be rebooted on heartbeat, even first heartbeat for nm2 NodeHeartbeatResponse hbResponse = nm1.nodeHeartbeat(true); - Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); + assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); // new NM to represent NM re-register nm1 = new MockNM("h1:1234", 10240, rm2.getResourceTrackerService()); @@ -2689,12 +2701,12 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() new String[]{"h1", "h2"}); allocator.sendRequest(event4); - // send allocate request to 2nd RM and get resync command + // send allocate request to 2nd RM and get rsync command allocator.schedule(); rm2.drainEvents(); - // Step-5 : On Resync,AM sends all outstanding - // asks,release,blacklistAaddition + // Step-5 : On Rsync,AM sends all outstanding + // asks,release,blacklistAddition // and another containerRequest(event5) ContainerRequestEvent event5 = ContainerRequestCreator.createRequest(jobId, 5, @@ -2715,12 +2727,11 @@ public void testRMContainerAllocatorResendsRequestsOnRMRestart() assignedContainers = allocator.schedule(); rm2.drainEvents(); - Assert.assertEquals("Number of container should be 3", 3, - assignedContainers.size()); + assertEquals(3, assignedContainers.size(), + "Number of container should be 3"); for (TaskAttemptContainerAssignedEvent assig : assignedContainers) { - Assert.assertTrue("Assigned count not correct", - "h1".equals(assig.getContainer().getNodeId().getHost())); + assertEquals("h1", assig.getContainer().getNodeId().getHost(), "Assigned count not correct"); } rm1.stop(); @@ -2764,7 +2775,7 @@ protected Resource getMaxContainerCapability() { allocator.sendRequests(Arrays.asList(mapRequestEvt)); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskMap); + assertEquals(0, mockScheduler.lastAnyAskMap); } @Test @@ -2807,7 +2818,7 @@ protected Resource getMaxContainerCapability() { allocator.scheduleAllReduces(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskReduce); + assertEquals(0, mockScheduler.lastAnyAskReduce); } @Test @@ -2842,19 +2853,18 @@ public void testRMUnavailable() allocator.jobEvents.clear(); try { allocator.schedule(); - Assert.fail("Should Have Exception"); + fail("Should Have Exception"); } catch (RMContainerAllocationException e) { - Assert.assertTrue(e.getMessage().contains("Could not contact RM after")); + assertTrue(e.getMessage().contains("Could not contact RM after")); } rm1.drainEvents(); - Assert.assertEquals("Should Have 1 Job Event", 1, - allocator.jobEvents.size()); + assertEquals(1, allocator.jobEvents.size(), "Should Have 1 Job Event"); JobEvent event = allocator.jobEvents.get(0); - Assert.assertTrue("Should Reboot", - event.getType().equals(JobEventType.JOB_AM_REBOOT)); + assertEquals(event.getType(), JobEventType.JOB_AM_REBOOT, "Should Reboot"); } - @Test(timeout=60000) + @Test + @Timeout(value = 60) public void testAMRMTokenUpdate() throws Exception { LOG.info("Running testAMRMTokenUpdate"); @@ -2892,7 +2902,7 @@ public void testAMRMTokenUpdate() throws Exception { final Token oldToken = rm.getRMContext().getRMApps() .get(appId).getRMAppAttempt(appAttemptId).getAMRMToken(); - Assert.assertNotNull("app should have a token", oldToken); + assertNotNull(oldToken, "app should have a token"); UserGroupInformation testUgi = UserGroupInformation.createUserForTesting( "someuser", new String[0]); Token newToken = testUgi.doAs( @@ -2907,7 +2917,7 @@ public Token run() throws Exception { long startTime = Time.monotonicNow(); while (currentToken == oldToken) { if (Time.monotonicNow() - startTime > 20000) { - Assert.fail("Took to long to see AMRM token change"); + fail("Took to long to see AMRM token change"); } Thread.sleep(100); allocator.schedule(); @@ -2930,13 +2940,12 @@ public Token run() throws Exception { } } - Assert.assertEquals("too many AMRM tokens", 1, tokenCount); - Assert.assertArrayEquals("token identifier not updated", - newToken.getIdentifier(), ugiToken.getIdentifier()); - Assert.assertArrayEquals("token password not updated", - newToken.getPassword(), ugiToken.getPassword()); - Assert.assertEquals("AMRM token service not updated", - new Text(rmAddr), ugiToken.getService()); + assertEquals(1, tokenCount, "too many AMRM tokens"); + assertArrayEquals(newToken.getIdentifier(), ugiToken.getIdentifier(), + "token identifier not updated"); + assertArrayEquals(newToken.getPassword(), ugiToken.getPassword(), + "token password not updated"); + assertEquals(new Text(rmAddr), ugiToken.getService(), "AMRM token service not updated"); } @Test @@ -2976,7 +2985,7 @@ protected ApplicationMasterProtocol createSchedulerProxy() { @Override protected void setRequestLimit(Priority priority, Resource capability, int limit) { - Assert.fail("setRequestLimit() should not be invoked"); + fail("setRequestLimit() should not be invoked"); } }; @@ -3058,22 +3067,22 @@ protected ApplicationMasterProtocol createSchedulerProxy() { // verify all of the host-specific asks were sent plus one for the // default rack and one for the ANY request - Assert.assertEquals(reqMapEvents.length + 2, mockScheduler.lastAsk.size()); + assertEquals(reqMapEvents.length + 2, mockScheduler.lastAsk.size()); // verify AM is only asking for the map limit overall - Assert.assertEquals(MAP_LIMIT, mockScheduler.lastAnyAskMap); + assertEquals(MAP_LIMIT, mockScheduler.lastAnyAskMap); // assign a map task and verify we do not ask for any more maps ContainerId cid0 = mockScheduler.assignContainer("h0", false); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(2, mockScheduler.lastAnyAskMap); + assertEquals(2, mockScheduler.lastAnyAskMap); // complete the map task and verify that we ask for one more mockScheduler.completeContainer(cid0); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(3, mockScheduler.lastAnyAskMap); + assertEquals(3, mockScheduler.lastAnyAskMap); // assign three more maps and verify we ask for no more maps ContainerId cid1 = mockScheduler.assignContainer("h1", false); @@ -3081,7 +3090,7 @@ protected ApplicationMasterProtocol createSchedulerProxy() { ContainerId cid3 = mockScheduler.assignContainer("h3", false); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskMap); + assertEquals(0, mockScheduler.lastAnyAskMap); // complete two containers and verify we only asked for one more // since at that point all maps should be scheduled/completed @@ -3089,7 +3098,7 @@ protected ApplicationMasterProtocol createSchedulerProxy() { mockScheduler.completeContainer(cid3); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(1, mockScheduler.lastAnyAskMap); + assertEquals(1, mockScheduler.lastAnyAskMap); // allocate the last container and complete the first one // and verify there are no more map asks. @@ -3097,76 +3106,78 @@ protected ApplicationMasterProtocol createSchedulerProxy() { ContainerId cid4 = mockScheduler.assignContainer("h4", false); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskMap); + assertEquals(0, mockScheduler.lastAnyAskMap); // complete the last map mockScheduler.completeContainer(cid4); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskMap); + assertEquals(0, mockScheduler.lastAnyAskMap); // verify only reduce limit being requested - Assert.assertEquals(REDUCE_LIMIT, mockScheduler.lastAnyAskReduce); + assertEquals(REDUCE_LIMIT, mockScheduler.lastAnyAskReduce); // assign a reducer and verify ask goes to zero cid0 = mockScheduler.assignContainer("h0", true); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskReduce); + assertEquals(0, mockScheduler.lastAnyAskReduce); // complete the reducer and verify we ask for another mockScheduler.completeContainer(cid0); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(1, mockScheduler.lastAnyAskReduce); + assertEquals(1, mockScheduler.lastAnyAskReduce); // assign a reducer and verify ask goes to zero cid0 = mockScheduler.assignContainer("h0", true); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskReduce); + assertEquals(0, mockScheduler.lastAnyAskReduce); // complete the reducer and verify no more reducers mockScheduler.completeContainer(cid0); allocator.schedule(); allocator.schedule(); - Assert.assertEquals(0, mockScheduler.lastAnyAskReduce); + assertEquals(0, mockScheduler.lastAnyAskReduce); allocator.close(); } - @Test(expected = RMContainerAllocationException.class) + @Test public void testAttemptNotFoundCausesRMCommunicatorException() throws Exception { - Configuration conf = new Configuration(); - MyResourceManager rm = new MyResourceManager(conf); - rm.start(); - - // Submit the application - RMApp app = MockRMAppSubmitter.submitWithMemory(1024, rm); - rm.drainEvents(); + assertThrows(RMContainerAllocationException.class, () -> { + Configuration conf = new Configuration(); + MyResourceManager rm = new MyResourceManager(conf); + rm.start(); - MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); - amNodeManager.nodeHeartbeat(true); - rm.drainEvents(); + // Submit the application + RMApp app = MockRMAppSubmitter.submitWithMemory(1024, rm); + rm.drainEvents(); - ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() - .getAppAttemptId(); - rm.sendAMLaunched(appAttemptId); - rm.drainEvents(); + MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); + amNodeManager.nodeHeartbeat(true); + rm.drainEvents(); - JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0); - Job mockJob = mock(Job.class); - when(mockJob.getReport()).thenReturn( - MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0, - 0, 0, 0, 0, 0, 0, "jobfile", null, false, "")); - MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, - appAttemptId, mockJob); + ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt() + .getAppAttemptId(); + rm.sendAMLaunched(appAttemptId); + rm.drainEvents(); - // Now kill the application - rm.killApp(app.getApplicationId()); - rm.waitForState(app.getApplicationId(), RMAppState.KILLED); - allocator.schedule(); + JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0); + Job mockJob = mock(Job.class); + when(mockJob.getReport()).thenReturn( + MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0, + 0, 0, 0, 0, 0, 0, "jobfile", null, false, "")); + MyContainerAllocator allocator = new MyContainerAllocator(rm, conf, + appAttemptId, mockJob); + + // Now kill the application + rm.killApp(app.getApplicationId()); + rm.waitForState(app.getApplicationId(), RMAppState.KILLED); + allocator.schedule(); + }); } @Test @@ -3247,29 +3258,29 @@ public void testUpdateAskOnRampDownAllReduces() throws Exception { rm.drainEvents(); // One map is assigned. - Assert.assertEquals(1, allocator.getAssignedRequests().maps.size()); + assertEquals(1, allocator.getAssignedRequests().maps.size()); // Send deallocate request for map so that no maps are assigned after this. ContainerAllocatorEvent deallocate = createDeallocateEvent(jobId, 1, false); allocator.sendDeallocate(deallocate); // Now one reducer should be scheduled and one should be pending. - Assert.assertEquals(1, allocator.getScheduledRequests().reduces.size()); - Assert.assertEquals(1, allocator.getNumOfPendingReduces()); + assertEquals(1, allocator.getScheduledRequests().reduces.size()); + assertEquals(1, allocator.getNumOfPendingReduces()); // No map should be assigned and one should be scheduled. - Assert.assertEquals(1, allocator.getScheduledRequests().maps.size()); - Assert.assertEquals(0, allocator.getAssignedRequests().maps.size()); + assertEquals(1, allocator.getScheduledRequests().maps.size()); + assertEquals(0, allocator.getAssignedRequests().maps.size()); - Assert.assertEquals(6, allocator.getAsk().size()); + assertEquals(6, allocator.getAsk().size()); for (ResourceRequest req : allocator.getAsk()) { boolean isReduce = req.getPriority().equals(RMContainerAllocator.PRIORITY_REDUCE); if (isReduce) { // 1 reducer each asked on h2, * and default-rack - Assert.assertTrue((req.getResourceName().equals("*") || + assertTrue((req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack") || req.getResourceName().equals("h2")) && req.getNumContainers() == 1); } else { //map // 0 mappers asked on h1 and 1 each on * and default-rack - Assert.assertTrue(((req.getResourceName().equals("*") || + assertTrue(((req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack")) && req.getNumContainers() == 1) || (req.getResourceName().equals("h1") && req.getNumContainers() == 0)); @@ -3282,17 +3293,17 @@ public void testUpdateAskOnRampDownAllReduces() throws Exception { // After allocate response from scheduler, all scheduled reduces are ramped // down and move to pending. 3 asks are also updated with 0 containers to // indicate ramping down of reduces to scheduler. - Assert.assertEquals(0, allocator.getScheduledRequests().reduces.size()); - Assert.assertEquals(2, allocator.getNumOfPendingReduces()); - Assert.assertEquals(3, allocator.getAsk().size()); + assertEquals(0, allocator.getScheduledRequests().reduces.size()); + assertEquals(2, allocator.getNumOfPendingReduces()); + assertEquals(3, allocator.getAsk().size()); for (ResourceRequest req : allocator.getAsk()) { - Assert.assertEquals( + assertEquals( RMContainerAllocator.PRIORITY_REDUCE, req.getPriority()); - Assert.assertTrue(req.getResourceName().equals("*") || + assertTrue(req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack") || req.getResourceName().equals("h2")); - Assert.assertEquals(Resource.newInstance(1024, 1), req.getCapability()); - Assert.assertEquals(0, req.getNumContainers()); + assertEquals(Resource.newInstance(1024, 1), req.getCapability()); + assertEquals(0, req.getNumContainers()); } } @@ -3417,29 +3428,29 @@ public void testAvoidAskMoreReducersWhenReducerPreemptionIsRequired() rm.drainEvents(); // One map is assigned. - Assert.assertEquals(1, allocator.getAssignedRequests().maps.size()); + assertEquals(1, allocator.getAssignedRequests().maps.size()); // Send deallocate request for map so that no maps are assigned after this. ContainerAllocatorEvent deallocate = createDeallocateEvent(jobId, 1, false); allocator.sendDeallocate(deallocate); // Now one reducer should be scheduled and one should be pending. - Assert.assertEquals(1, allocator.getScheduledRequests().reduces.size()); - Assert.assertEquals(1, allocator.getNumOfPendingReduces()); + assertEquals(1, allocator.getScheduledRequests().reduces.size()); + assertEquals(1, allocator.getNumOfPendingReduces()); // No map should be assigned and one should be scheduled. - Assert.assertEquals(1, allocator.getScheduledRequests().maps.size()); - Assert.assertEquals(0, allocator.getAssignedRequests().maps.size()); + assertEquals(1, allocator.getScheduledRequests().maps.size()); + assertEquals(0, allocator.getAssignedRequests().maps.size()); - Assert.assertEquals(6, allocator.getAsk().size()); + assertEquals(6, allocator.getAsk().size()); for (ResourceRequest req : allocator.getAsk()) { boolean isReduce = req.getPriority().equals(RMContainerAllocator.PRIORITY_REDUCE); if (isReduce) { // 1 reducer each asked on h2, * and default-rack - Assert.assertTrue((req.getResourceName().equals("*") || + assertTrue((req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack") || req.getResourceName().equals("h2")) && req.getNumContainers() == 1); } else { //map // 0 mappers asked on h1 and 1 each on * and default-rack - Assert.assertTrue(((req.getResourceName().equals("*") || + assertTrue(((req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack")) && req.getNumContainers() == 1) || (req.getResourceName().equals("h1") && req.getNumContainers() == 0)); @@ -3455,17 +3466,17 @@ public void testAvoidAskMoreReducersWhenReducerPreemptionIsRequired() // After allocate response from scheduler, all scheduled reduces are ramped // down and move to pending. 3 asks are also updated with 0 containers to // indicate ramping down of reduces to scheduler. - Assert.assertEquals(0, allocator.getScheduledRequests().reduces.size()); - Assert.assertEquals(2, allocator.getNumOfPendingReduces()); - Assert.assertEquals(3, allocator.getAsk().size()); + assertEquals(0, allocator.getScheduledRequests().reduces.size()); + assertEquals(2, allocator.getNumOfPendingReduces()); + assertEquals(3, allocator.getAsk().size()); for (ResourceRequest req : allocator.getAsk()) { - Assert.assertEquals( + assertEquals( RMContainerAllocator.PRIORITY_REDUCE, req.getPriority()); - Assert.assertTrue(req.getResourceName().equals("*") || + assertTrue(req.getResourceName().equals("*") || req.getResourceName().equals("/default-rack") || req.getResourceName().equals("h2")); - Assert.assertEquals(Resource.newInstance(1024, 1), req.getCapability()); - Assert.assertEquals(0, req.getNumContainers()); + assertEquals(Resource.newInstance(1024, 1), req.getCapability()); + assertEquals(0, req.getNumContainers()); } } @@ -3553,14 +3564,14 @@ public void testExcludeSchedReducesFromHeadroom() throws Exception { rm.drainEvents(); // Two maps are assigned. - Assert.assertEquals(2, allocator.getAssignedRequests().maps.size()); + assertEquals(2, allocator.getAssignedRequests().maps.size()); // Send deallocate request for map so that no maps are assigned after this. ContainerAllocatorEvent deallocate1 = createDeallocateEvent(jobId, 1, false); allocator.sendDeallocate(deallocate1); ContainerAllocatorEvent deallocate2 = createDeallocateEvent(jobId, 2, false); allocator.sendDeallocate(deallocate2); // No map should be assigned. - Assert.assertEquals(0, allocator.getAssignedRequests().maps.size()); + assertEquals(0, allocator.getAssignedRequests().maps.size()); nodeManager.nodeHeartbeat(true); rm.drainEvents(); @@ -3584,18 +3595,18 @@ public void testExcludeSchedReducesFromHeadroom() throws Exception { allocator.schedule(); rm.drainEvents(); // One reducer is assigned and one map is scheduled - Assert.assertEquals(1, allocator.getScheduledRequests().maps.size()); - Assert.assertEquals(1, allocator.getAssignedRequests().reduces.size()); + assertEquals(1, allocator.getScheduledRequests().maps.size()); + assertEquals(1, allocator.getAssignedRequests().reduces.size()); // Headroom enough to run a mapper if headroom is taken as it is but wont be // enough if scheduled reducers resources are deducted. rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(1260, 2)); allocator.schedule(); rm.drainEvents(); // After allocate response, the one assigned reducer is preempted and killed - Assert.assertEquals(1, MyContainerAllocator.getTaskAttemptKillEvents().size()); - Assert.assertEquals(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC, + assertEquals(1, MyContainerAllocator.getTaskAttemptKillEvents().size()); + assertEquals(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC, MyContainerAllocator.getTaskAttemptKillEvents().get(0).getMessage()); - Assert.assertEquals(1, allocator.getNumOfPendingReduces()); + assertEquals(1, allocator.getNumOfPendingReduces()); } private static class MockScheduler implements ApplicationMasterProtocol { @@ -3718,7 +3729,7 @@ public AllocateResponse allocate(AllocateRequest request) } } - public static void main(String[] args) throws Exception { + /*public static void main(String[] args) throws Exception { TestRMContainerAllocator t = new TestRMContainerAllocator(); t.testSimple(); t.testResource(); @@ -3728,6 +3739,6 @@ public static void main(String[] args) throws Exception { t.testBlackListedNodes(); t.testCompletedTasksRecalculateSchedule(); t.testAMRMTokenUpdate(); - } + }*/ } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java index cab8f54441675..0a6dc7c001bef 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java @@ -19,12 +19,12 @@ package org.apache.hadoop.mapreduce.v2.app.rm; import org.apache.hadoop.yarn.api.records.Resource; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.EnumSet; import static org.apache.hadoop.yarn.proto.YarnServiceProtos.*; +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestResourceCalculatorUtils { @Test @@ -59,17 +59,16 @@ private void verifyDifferentResourceTypes(Resource clusterAvailableResources, Resource nonZeroResource, int expectedNumberOfContainersForMemoryOnly, int expectedNumberOfContainersOverall) { - Assert.assertEquals("Incorrect number of available containers for Memory", - expectedNumberOfContainersForMemoryOnly, + assertEquals(expectedNumberOfContainersForMemoryOnly, ResourceCalculatorUtils.computeAvailableContainers( - clusterAvailableResources, nonZeroResource, - EnumSet.of(SchedulerResourceTypes.MEMORY))); + clusterAvailableResources, nonZeroResource, + EnumSet.of(SchedulerResourceTypes.MEMORY)), + "Incorrect number of available containers for Memory"); - Assert.assertEquals("Incorrect number of available containers overall", - expectedNumberOfContainersOverall, + assertEquals(expectedNumberOfContainersOverall, ResourceCalculatorUtils.computeAvailableContainers( - clusterAvailableResources, nonZeroResource, - EnumSet.of(SchedulerResourceTypes.CPU, - SchedulerResourceTypes.MEMORY))); + clusterAvailableResources, nonZeroResource, + EnumSet.of(SchedulerResourceTypes.CPU, SchedulerResourceTypes.MEMORY)), + "Incorrect number of available containers overall"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java index d5b817c48288d..16cf064ad7e82 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java @@ -18,8 +18,9 @@ package org.apache.hadoop.mapreduce.v2.app.speculate; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; public class TestDataStatistics { @@ -28,46 +29,46 @@ public class TestDataStatistics { @Test public void testEmptyDataStatistics() throws Exception { DataStatistics statistics = new DataStatistics(); - Assert.assertEquals(0, statistics.count(), TOL); - Assert.assertEquals(0, statistics.mean(), TOL); - Assert.assertEquals(0, statistics.var(), TOL); - Assert.assertEquals(0, statistics.std(), TOL); - Assert.assertEquals(0, statistics.outlier(1.0f), TOL); + assertEquals(0, statistics.count(), TOL); + assertEquals(0, statistics.mean(), TOL); + assertEquals(0, statistics.var(), TOL); + assertEquals(0, statistics.std(), TOL); + assertEquals(0, statistics.outlier(1.0f), TOL); } @Test public void testSingleEntryDataStatistics() throws Exception { DataStatistics statistics = new DataStatistics(17.29); - Assert.assertEquals(1, statistics.count(), TOL); - Assert.assertEquals(17.29, statistics.mean(), TOL); - Assert.assertEquals(0, statistics.var(), TOL); - Assert.assertEquals(0, statistics.std(), TOL); - Assert.assertEquals(17.29, statistics.outlier(1.0f), TOL); + assertEquals(1, statistics.count(), TOL); + assertEquals(17.29, statistics.mean(), TOL); + assertEquals(0, statistics.var(), TOL); + assertEquals(0, statistics.std(), TOL); + assertEquals(17.29, statistics.outlier(1.0f), TOL); } @Test - public void testMutiEntryDataStatistics() throws Exception { + public void testMultiEntryDataStatistics() throws Exception { DataStatistics statistics = new DataStatistics(); statistics.add(17); statistics.add(29); - Assert.assertEquals(2, statistics.count(), TOL); - Assert.assertEquals(23.0, statistics.mean(), TOL); - Assert.assertEquals(36.0, statistics.var(), TOL); - Assert.assertEquals(6.0, statistics.std(), TOL); - Assert.assertEquals(29.0, statistics.outlier(1.0f), TOL); + assertEquals(2, statistics.count(), TOL); + assertEquals(23.0, statistics.mean(), TOL); + assertEquals(36.0, statistics.var(), TOL); + assertEquals(6.0, statistics.std(), TOL); + assertEquals(29.0, statistics.outlier(1.0f), TOL); } @Test public void testUpdateStatistics() throws Exception { DataStatistics statistics = new DataStatistics(17); statistics.add(29); - Assert.assertEquals(2, statistics.count(), TOL); - Assert.assertEquals(23.0, statistics.mean(), TOL); - Assert.assertEquals(36.0, statistics.var(), TOL); + assertEquals(2, statistics.count(), TOL); + assertEquals(23.0, statistics.mean(), TOL); + assertEquals(36.0, statistics.var(), TOL); statistics.updateStatistics(17, 29); - Assert.assertEquals(2, statistics.count(), TOL); - Assert.assertEquals(29.0, statistics.mean(), TOL); - Assert.assertEquals(0.0, statistics.var(), TOL); + assertEquals(2, statistics.count(), TOL); + assertEquals(29.0, statistics.mean(), TOL); + assertEquals(0.0, statistics.var(), TOL); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java index b799d8c2ccdd7..b887aacecabbe 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java @@ -23,8 +23,9 @@ import org.apache.hadoop.yarn.util.ControlledClock; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Testing the statistical model of simple exponential estimator. @@ -101,21 +102,18 @@ private int zeroTestSimpleExponentialForecast() { @Test public void testSimpleExponentialForecastLinearInc() throws Exception { int res = incTestSimpleExponentialForecast(); - Assert.assertEquals("We got the wrong estimate from simple exponential.", - res, 0); + assertEquals(res, 0, "We got the wrong estimate from simple exponential."); } @Test public void testSimpleExponentialForecastLinearDec() throws Exception { int res = decTestSimpleExponentialForecast(); - Assert.assertEquals("We got the wrong estimate from simple exponential.", - res, 0); + assertEquals(res, 0, "We got the wrong estimate from simple exponential."); } @Test public void testSimpleExponentialForecastZeros() throws Exception { int res = zeroTestSimpleExponentialForecast(); - Assert.assertEquals("We got the wrong estimate from simple exponential.", - res, 0); + assertEquals(res, 0, "We got the wrong estimate from simple exponential."); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java index de9e7a2692e2b..575c69c85d6f9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID; -import static org.junit.Assert.assertEquals; import java.io.ByteArrayOutputStream; import java.io.File; @@ -45,7 +44,6 @@ import org.glassfish.jersey.internal.inject.AbstractBinder; import org.glassfish.jersey.jettison.JettisonFeature; import org.glassfish.jersey.server.ResourceConfig; -import org.junit.Assert; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.HttpConfig.Policy; @@ -70,12 +68,14 @@ import org.apache.hadoop.yarn.webapp.test.WebAppTests; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; import org.apache.http.HttpStatus; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; import com.google.inject.Injector; -import org.junit.contrib.java.lang.system.EnvironmentVariables; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; public class TestAMWebApp { @@ -84,7 +84,7 @@ public class TestAMWebApp { System.getProperty("java.io.tmpdir")), TestAMWebApp.class.getName()); - @After + @AfterEach public void tearDown() { TEST_DIR.delete(); } @@ -217,7 +217,7 @@ protected ClientService createClientService(AppContext context) { InputStream in = conn.getInputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream(); IOUtils.copyBytes(in, out, 1024); - Assert.assertTrue(out.toString().contains("MapReduce Application")); + assertTrue(out.toString().contains("MapReduce Application")); // https:// is not accessible. URL httpsUrl = new URL("https://" + hostPort + "/mapreduce/"); @@ -225,7 +225,7 @@ protected ClientService createClientService(AppContext context) { HttpURLConnection httpsConn = (HttpURLConnection) httpsUrl.openConnection(); httpsConn.getInputStream(); - Assert.fail("https:// is not accessible, expected to fail"); + fail("https:// is not accessible, expected to fail"); } catch (SSLException e) { // expected } @@ -234,10 +234,6 @@ protected ClientService createClientService(AppContext context) { app.verifyCompleted(); } - @Rule - public final EnvironmentVariables environmentVariables - = new EnvironmentVariables(); - @Test public void testMRWebAppSSLEnabled() throws Exception { MRApp app = new MRApp(2, 2, true, this.getClass().getName(), true) { @@ -256,9 +252,8 @@ protected ClientService createClientService(AppContext context) { keystoreFile.getParentFile().mkdirs(); KeyStoreTestUtil.createKeyStore(keystoreFile.getAbsolutePath(), "password", "server", keyPair.getPrivate(), cert); - environmentVariables.set("KEYSTORE_FILE_LOCATION", - keystoreFile.getAbsolutePath()); - environmentVariables.set("KEYSTORE_PASSWORD", "password"); + System.setProperty("KEYSTORE_FILE_LOCATION", keystoreFile.getAbsolutePath()); + System.setProperty("KEYSTORE_PASSWORD", "password"); Job job = app.submit(conf); @@ -274,7 +269,7 @@ protected ClientService createClientService(AppContext context) { InputStream in = httpsConn.getInputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream(); IOUtils.copyBytes(in, out, 1024); - Assert.assertTrue(out.toString().contains("MapReduce Application")); + assertTrue(out.toString().contains("MapReduce Application")); // http:// is not accessible. URL httpUrl = new URL("http://" + hostPort + "/mapreduce/"); @@ -282,7 +277,7 @@ protected ClientService createClientService(AppContext context) { HttpURLConnection httpConn = (HttpURLConnection) httpUrl.openConnection(); httpConn.getResponseCode(); - Assert.fail("http:// is not accessible, expected to fail"); + fail("http:// is not accessible, expected to fail"); } catch (SocketException e) { // expected } @@ -291,6 +286,8 @@ protected ClientService createClientService(AppContext context) { app.verifyCompleted(); keystoreFile.delete(); + System.clearProperty("KEYSTORE_FILE_LOCATION"); + System.clearProperty("KEYSTORE_PASSWORD"); } @Test @@ -312,9 +309,8 @@ protected ClientService createClientService(AppContext context) { keystoreFile.getParentFile().mkdirs(); KeyStoreTestUtil.createKeyStore(keystoreFile.getAbsolutePath(), "password", "server", keyPair.getPrivate(), cert); - environmentVariables.set("KEYSTORE_FILE_LOCATION", - keystoreFile.getAbsolutePath()); - environmentVariables.set("KEYSTORE_PASSWORD", "password"); + System.setProperty("KEYSTORE_FILE_LOCATION", keystoreFile.getAbsolutePath()); + System.setProperty("KEYSTORE_PASSWORD", "password"); KeyPair clientKeyPair = KeyStoreTestUtil.generateKeyPair("RSA"); X509Certificate clientCert = KeyStoreTestUtil.generateCertificate( @@ -323,9 +319,8 @@ protected ClientService createClientService(AppContext context) { truststoreFile.getParentFile().mkdirs(); KeyStoreTestUtil.createTrustStore(truststoreFile.getAbsolutePath(), "password", "client", clientCert); - environmentVariables.set("TRUSTSTORE_FILE_LOCATION", - truststoreFile.getAbsolutePath()); - environmentVariables.set("TRUSTSTORE_PASSWORD", "password"); + System.setProperty("TRUSTSTORE_FILE_LOCATION", truststoreFile.getAbsolutePath()); + System.setProperty("TRUSTSTORE_PASSWORD", "password"); Job job = app.submit(conf); @@ -341,7 +336,7 @@ protected ClientService createClientService(AppContext context) { InputStream in = httpsConn.getInputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream(); IOUtils.copyBytes(in, out, 1024); - Assert.assertTrue(out.toString().contains("MapReduce Application")); + assertTrue(out.toString().contains("MapReduce Application")); // Try with wrong client cert KeyPair otherClientKeyPair = KeyStoreTestUtil.generateKeyPair("RSA"); @@ -353,7 +348,7 @@ protected ClientService createClientService(AppContext context) { HttpURLConnection httpConn = (HttpURLConnection) httpsUrl.openConnection(); httpConn.getResponseCode(); - Assert.fail("Wrong client certificate, expected to fail"); + fail("Wrong client certificate, expected to fail"); } catch (SSLException e) { // expected } @@ -363,6 +358,11 @@ protected ClientService createClientService(AppContext context) { keystoreFile.delete(); truststoreFile.delete(); + + System.clearProperty("KEYSTORE_FILE_LOCATION"); + System.clearProperty("KEYSTORE_PASSWORD"); + System.clearProperty("TRUSTSTORE_FILE_LOCATION"); + System.clearProperty("TRUSTSTORE_PASSWORD"); } static String webProxyBase = null; @@ -408,9 +408,9 @@ protected ClientService createClientService(AppContext context) { String expectedURL = scheme + conf.get(YarnConfiguration.PROXY_ADDRESS) + ProxyUriUtils.getPath(app.getAppID(), "/mapreduce", true); - Assert.assertEquals(expectedURL, + assertEquals(expectedURL, conn.getHeaderField(HttpHeaders.LOCATION)); - Assert.assertEquals(HttpStatus.SC_MOVED_TEMPORARILY, + assertEquals(HttpStatus.SC_MOVED_TEMPORARILY, conn.getResponseCode()); app.waitForState(job, JobState.SUCCEEDED); app.verifyCompleted(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java index 560bc75156099..89b6dead1e764 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java @@ -19,9 +19,9 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import java.io.StringReader; @@ -52,7 +52,7 @@ import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -109,7 +109,7 @@ public void testAM() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -121,7 +121,7 @@ public void testAMSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -133,7 +133,7 @@ public void testAMDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -157,7 +157,7 @@ public void testInfo() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -170,7 +170,7 @@ public void testInfoSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -182,7 +182,7 @@ public void testInfoDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyAMInfo(json.getJSONObject("info"), appContext); } @@ -255,7 +255,7 @@ public void testBlacklistedNodes() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); verifyBlacklistedNodesInfo(json, appContext); } @@ -273,7 +273,7 @@ public void testBlacklistedNodesXML() throws Exception { public void verifyAMInfo(JSONObject info, AppContext ctx) throws JSONException { - assertEquals("incorrect number of elements", 5, info.length()); + assertEquals(5, info.length(), "incorrect number of elements"); verifyAMInfoGeneric(ctx, info.getString("appId"), info.getString("user"), info.getString("name"), info.getLong("startedOn"), @@ -288,7 +288,7 @@ public void verifyAMInfoXML(String xml, AppContext ctx) is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList nodes = dom.getElementsByTagName("info"); - assertEquals("incorrect number of elements", 1, nodes.getLength()); + assertEquals(1, nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); @@ -311,8 +311,8 @@ public void verifyAMInfoGeneric(AppContext ctx, String id, String user, WebServicesTestUtils.checkStringMatch("name", ctx.getApplicationName(), name); - assertEquals("startedOn incorrect", ctx.getStartTime(), startedOn); - assertTrue("elapsedTime not greater then 0", (elapsedTime > 0)); + assertEquals(ctx.getStartTime(), startedOn, "startedOn incorrect"); + assertTrue((elapsedTime > 0), "elapsedTime not greater then 0"); } @@ -334,11 +334,11 @@ public void verifyBlacklistedNodesInfoXML(String xml, AppContext ctx) is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList infonodes = dom.getElementsByTagName("blacklistednodesinfo"); - assertEquals("incorrect number of elements", 1, infonodes.getLength()); + assertEquals(1, infonodes.getLength(), "incorrect number of elements"); NodeList nodes = dom.getElementsByTagName("blacklistedNodes"); Set blacklistedNodes = ctx.getBlacklistedNodes(); - assertEquals("incorrect number of elements", blacklistedNodes.size(), - nodes.getLength()); + assertEquals(blacklistedNodes.size(), + nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); assertTrue( diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java index 2d802e52cafae..e44901b1d80d7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java @@ -18,7 +18,7 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; -import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -57,7 +57,7 @@ import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.apache.hadoop.yarn.webapp.WebServicesTestUtils; import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -128,11 +128,6 @@ protected Properties getConfiguration(String configPrefix, } } - @Override - public void setUp() throws Exception { - super.setUp(); - } - public TestAMWebServicesAttempt() { } @@ -160,7 +155,7 @@ public void testGetTaskAttemptIdState() throws Exception { + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); JSONObject jobState = json.getJSONObject("jobTaskAttemptState"); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); assertEquals(att.getState().toString(), jobState.get("state")); } } @@ -229,7 +224,7 @@ public void testPutTaskAttemptIdState() throws Exception { + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); JSONObject jobState = json.getJSONObject("jobTaskAttemptState"); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); assertEquals(TaskAttemptState.KILLED.toString(), jobState.get("state")); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java index 7ecb3e9ec774e..2a3e13f3c0c74 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java @@ -19,11 +19,11 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import java.io.StringReader; @@ -59,7 +59,7 @@ import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -190,7 +190,7 @@ public void testTaskAttemptsXML() throws JSONException, Exception { is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList attempts = dom.getElementsByTagName("taskAttempts"); - assertEquals("incorrect number of elements", 1, attempts.getLength()); + assertEquals(1, attempts.getLength(), "incorrect number of elements"); NodeList nodes = dom.getElementsByTagName("taskAttempt"); verifyAMTaskAttemptsXML(nodes, task); @@ -220,7 +220,7 @@ public void testTaskAttemptId() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("taskAttempt"); verifyAMTaskAttempt(info, att, task.getType()); } @@ -250,7 +250,7 @@ public void testTaskAttemptIdSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("taskAttempt"); verifyAMTaskAttempt(info, att, task.getType()); } @@ -280,7 +280,7 @@ public void testTaskAttemptIdDefault() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("taskAttempt"); verifyAMTaskAttempt(info, att, task.getType()); } @@ -385,7 +385,7 @@ private void testTaskAttemptIdErrorGeneric(String attid, String error) String entity = response.readEntity(String.class); JSONObject msg = new JSONObject(entity); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -426,9 +426,9 @@ public void verifyAMTaskAttemptXML(Element element, TaskAttempt att, public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att, TaskType ttype) throws JSONException { if (ttype == TaskType.REDUCE) { - assertEquals("incorrect number of elements", 17, info.length()); + assertEquals(17, info.length(), "incorrect number of elements"); } else { - assertEquals("incorrect number of elements", 12, info.length()); + assertEquals(12, info.length(), "incorrect number of elements"); } verifyTaskAttemptGeneric(att, ttype, info.getString("id"), @@ -447,9 +447,9 @@ public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att, public void verifyAMTaskAttempts(JSONObject json, Task task) throws JSONException { - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject attempts = json.getJSONObject("taskAttempts"); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject taskAttempt = attempts.getJSONObject("taskAttempt"); JSONArray arr = new JSONArray(); arr.put(taskAttempt); @@ -465,13 +465,13 @@ public void verifyAMTaskAttempts(JSONObject json, Task task) verifyAMTaskAttempt(info, att, task.getType()); } } - assertTrue("task attempt with id: " + attid - + " not in web service output", found); + assertTrue(found, "task attempt with id: " + attid + + " not in web service output"); } } public void verifyAMTaskAttemptsXML(NodeList nodes, Task task) { - assertEquals("incorrect number of elements", 1, nodes.getLength()); + assertEquals(1, nodes.getLength(), "incorrect number of elements"); for (TaskAttempt att : task.getAttempts().values()) { TaskAttemptId id = att.getID(); @@ -479,15 +479,15 @@ public void verifyAMTaskAttemptsXML(NodeList nodes, Task task) { boolean found = false; for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); - assertFalse("task attempt should not contain any attributes, it can lead to incorrect JSON marshaling", - element.hasAttributes()); + assertFalse(element.hasAttributes(), "task attempt should not contain any attributes, " + + "it can lead to incorrect JSON marshaling"); if (attid.matches(WebServicesTestUtils.getXmlString(element, "id"))) { found = true; verifyAMTaskAttemptXML(element, att, task.getType()); } } - assertTrue("task with id: " + attid + " not in web service output", found); + assertTrue(found, "task with id: " + attid + " not in web service output"); } } @@ -522,26 +522,24 @@ public void verifyTaskAttemptGeneric(TaskAttempt ta, TaskType ttype, ta.getAssignedContainerID().toString(), assignedContainerId); - assertEquals("startTime wrong", ta.getLaunchTime(), startTime); - assertEquals("finishTime wrong", ta.getFinishTime(), finishTime); - assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime); - assertEquals("progress wrong", ta.getProgress() * 100, progress, 1e-3f); + assertEquals(ta.getLaunchTime(), startTime, "startTime wrong"); + assertEquals(ta.getFinishTime(), finishTime, "finishTime wrong"); + assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong"); + assertEquals(ta.getProgress() * 100, progress, 1e-3f, "progress wrong"); } public void verifyReduceTaskAttemptGeneric(TaskAttempt ta, long shuffleFinishTime, long mergeFinishTime, long elapsedShuffleTime, long elapsedMergeTime, long elapsedReduceTime) { - assertEquals("shuffleFinishTime wrong", ta.getShuffleFinishTime(), - shuffleFinishTime); - assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(), - mergeFinishTime); - assertEquals("elapsedShuffleTime wrong", - ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime); - assertEquals("elapsedMergeTime wrong", - ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime); - assertEquals("elapsedReduceTime wrong", - ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime); + assertEquals(ta.getShuffleFinishTime(), shuffleFinishTime, "shuffleFinishTime wrong"); + assertEquals(ta.getSortFinishTime(), mergeFinishTime, "mergeFinishTime wrong"); + assertEquals(ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime, + "elapsedShuffleTime wrong"); + assertEquals(ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime, + "elapsedMergeTime wrong"); + assertEquals(ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime, + "elapsedReduceTime wrong"); } @Test @@ -566,7 +564,7 @@ public void testTaskAttemptIdCounters() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobTaskAttemptCounters"); verifyAMJobTaskAttemptCounters(info, att); } @@ -611,7 +609,7 @@ public void testTaskAttemptIdXMLCounters() throws Exception { public void verifyAMJobTaskAttemptCounters(JSONObject info, TaskAttempt att) throws JSONException { - assertEquals("incorrect number of elements", 2, info.length()); + assertEquals(2, info.length(), "incorrect number of elements"); WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()), info.getString("id")); @@ -622,15 +620,14 @@ public void verifyAMJobTaskAttemptCounters(JSONObject info, TaskAttempt att) for (int i = 0; i < counterGroups.length(); i++) { JSONObject counterGroup = counterGroups.getJSONObject(i); String name = counterGroup.getString("counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); JSONArray counters = counterGroup.getJSONArray("counter"); for (int j = 0; j < counters.length(); j++) { JSONObject counter = counters.getJSONObject(j); String counterName = counter.getString("name"); - assertTrue("name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), "name not set"); long value = counter.getLong("value"); - assertTrue("value >= 0", value >= 0); + assertTrue(value >= 0, "value >= 0"); } } } @@ -648,20 +645,19 @@ public void verifyAMTaskCountersXML(NodeList nodes, TaskAttempt att) { for (int j = 0; j < groups.getLength(); j++) { Element counters = (Element) groups.item(j); - assertNotNull("should have counters in the web service info", counters); + assertNotNull(counters, "should have counters in the web service info"); String name = WebServicesTestUtils.getXmlString(counters, "counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); NodeList counterArr = counters.getElementsByTagName("counter"); for (int z = 0; z < counterArr.getLength(); z++) { Element counter = (Element) counterArr.item(z); String counterName = WebServicesTestUtils.getXmlString(counter, "name"); - assertTrue("counter name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), "counter name not set"); long value = WebServicesTestUtils.getXmlLong(counter, "value"); - assertTrue("value not >= 0", value >= 0); + assertTrue(value >= 0, "value not >= 0"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java index e9ae85f8ced6b..ecebf1403487e 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java @@ -18,10 +18,10 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; -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.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import java.io.File; @@ -57,8 +57,9 @@ import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.AfterClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -130,13 +131,13 @@ protected void configure() { } } - @Override - public void setUp() throws Exception { - super.setUp(); + + @BeforeAll + public static void setTestConfDir() throws Exception { testConfDir.mkdir(); } - @AfterClass + @AfterAll static public void stop() { FileUtil.fullyDelete(testConfDir); } @@ -154,7 +155,7 @@ public void testJobConf() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("conf"); verifyAMJobConf(info, jobsMap.get(id)); } @@ -172,7 +173,7 @@ public void testJobConfSlash() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("conf"); verifyAMJobConf(info, jobsMap.get(id)); } @@ -190,7 +191,7 @@ public void testJobConfDefault() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("conf"); verifyAMJobConf(info, jobsMap.get(id)); } @@ -221,7 +222,7 @@ public void testJobConfXML() throws Exception { public void verifyAMJobConf(JSONObject info, Job job) throws JSONException { - assertEquals("incorrect number of elements", 2, info.length()); + assertEquals(2, info.length(), "incorrect number of elements"); WebServicesTestUtils.checkStringMatch("path", job.getConfFile().toString(), info.getString("path")); @@ -232,14 +233,14 @@ public void verifyAMJobConf(JSONObject info, Job job) throws JSONException { JSONObject prop = properties.getJSONObject(i); String name = prop.getString("name"); String value = prop.getString("value"); - assertTrue("name not set", (name != null && !name.isEmpty())); - assertTrue("value not set", (value != null && !value.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); + assertTrue((value != null && !value.isEmpty()), "value not set"); } } public void verifyAMJobConfXML(NodeList nodes, Job job) { - assertEquals("incorrect number of elements", 1, nodes.getLength()); + assertEquals(1, nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); @@ -252,11 +253,11 @@ public void verifyAMJobConfXML(NodeList nodes, Job job) { for (int j = 0; j < properties.getLength(); j++) { Element property = (Element) properties.item(j); - assertNotNull("should have counters in the web service info", property); + assertNotNull(property, "should have counters in the web service info"); String name = WebServicesTestUtils.getXmlString(property, "name"); String value = WebServicesTestUtils.getXmlString(property, "value"); - assertTrue("name not set", (name != null && !name.isEmpty())); - assertTrue("name not set", (value != null && !value.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); + assertTrue((value != null && !value.isEmpty()), "name not set"); } } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java index ba6b9606bc484..c4f24e2fa504c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java @@ -20,10 +20,10 @@ import static org.apache.hadoop.yarn.util.StringHelper.ujoin; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; -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.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import java.io.StringReader; @@ -60,7 +60,7 @@ import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -119,7 +119,7 @@ public void testJobs() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject jobs = json.getJSONObject("jobs"); JSONObject jobObject = jobs.getJSONObject("job"); JSONArray arr = new JSONArray(); @@ -139,7 +139,7 @@ public void testJobsSlash() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject jobs = json.getJSONObject("jobs"); JSONObject jobObject = jobs.getJSONObject("job"); JSONArray arr = new JSONArray(); @@ -158,7 +158,7 @@ public void testJobsDefault() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject jobs = json.getJSONObject("jobs"); JSONObject jobObject = jobs.getJSONObject("job"); JSONArray arr = new JSONArray(); @@ -184,9 +184,9 @@ public void testJobsXML() throws Exception { is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList jobs = dom.getElementsByTagName("jobs"); - assertEquals("incorrect number of elements", 1, jobs.getLength()); + assertEquals(1, jobs.getLength(), "incorrect number of elements"); NodeList job = dom.getElementsByTagName("job"); - assertEquals("incorrect number of elements", 1, job.getLength()); + assertEquals(1, job.getLength(), "incorrect number of elements"); verifyAMJobXML(job, appContext); } @@ -204,7 +204,7 @@ public void testJobId() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("job"); verifyAMJob(info, jobsMap.get(id)); } @@ -224,7 +224,7 @@ public void testJobIdSlash() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("job"); verifyAMJob(info, jobsMap.get(id)); } @@ -242,7 +242,7 @@ public void testJobIdDefault() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("job"); verifyAMJob(info, jobsMap.get(id)); } @@ -264,7 +264,7 @@ public void testJobIdNonExist() throws Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -292,7 +292,7 @@ public void testJobIdInvalid() throws Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -316,7 +316,7 @@ public void testJobIdInvalidDefault() throws Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -379,7 +379,7 @@ public void testJobIdInvalidBogus() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -419,7 +419,7 @@ public void testJobIdXML() throws Exception { public void verifyAMJob(JSONObject info, Job job) throws JSONException { - assertEquals("incorrect number of elements", 31, info.length()); + assertEquals(31, info.length(), "incorrect number of elements"); // everyone access fields verifyAMJobGeneric(job, info.getString("id"), info.getString("user"), @@ -470,8 +470,7 @@ public void verifyAMJob(JSONObject info, Job job) throws JSONException { } else { fail("should have acls in the web service info"); } - assertTrue("acl: " + expectName + " not found in webservice output", - found); + assertTrue(found, "acl: " + expectName + " not found in webservice output"); } } @@ -479,14 +478,14 @@ public void verifyAMJob(JSONObject info, Job job) throws JSONException { public void verifyAMJobXML(NodeList nodes, AppContext appContext) { - assertEquals("incorrect number of elements", 1, nodes.getLength()); + assertEquals(1, nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); Job job = appContext.getJob(MRApps.toJobID(WebServicesTestUtils .getXmlString(element, "id"))); - assertNotNull("Job not found - output incorrect", job); + assertNotNull(job, "Job not found - output incorrect"); verifyAMJobGeneric(job, WebServicesTestUtils.getXmlString(element, "id"), WebServicesTestUtils.getXmlString(element, "user"), @@ -545,8 +544,7 @@ public void verifyAMJobXML(NodeList nodes, AppContext appContext) { } else { fail("should have acls in the web service info"); } - assertTrue("acl: " + expectName + " not found in webservice output", - found); + assertTrue(found, "acl: " + expectName + " not found in webservice output"); } } } @@ -560,27 +558,25 @@ public void verifyAMJobGeneric(Job job, String id, String user, String name, WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()), id); - WebServicesTestUtils.checkStringMatch("user", job.getUserName().toString(), + WebServicesTestUtils.checkStringMatch("user", job.getUserName(), user); WebServicesTestUtils.checkStringMatch("name", job.getName(), name); WebServicesTestUtils.checkStringMatch("state", job.getState().toString(), state); - assertEquals("startTime incorrect", report.getStartTime(), startTime); - assertEquals("finishTime incorrect", report.getFinishTime(), finishTime); - assertEquals("elapsedTime incorrect", - Times.elapsed(report.getStartTime(), report.getFinishTime()), - elapsedTime); - assertEquals("mapsTotal incorrect", job.getTotalMaps(), mapsTotal); - assertEquals("mapsCompleted incorrect", job.getCompletedMaps(), - mapsCompleted); - assertEquals("reducesTotal incorrect", job.getTotalReduces(), reducesTotal); - assertEquals("reducesCompleted incorrect", job.getCompletedReduces(), - reducesCompleted); - assertEquals("mapProgress incorrect", report.getMapProgress() * 100, - mapProgress, 0); - assertEquals("reduceProgress incorrect", report.getReduceProgress() * 100, - reduceProgress, 0); + assertEquals(report.getStartTime(), startTime, "startTime incorrect"); + assertEquals(report.getFinishTime(), finishTime, "finishTime incorrect"); + assertEquals(Times.elapsed(report.getStartTime(), report.getFinishTime()), + elapsedTime, "elapsedTime incorrect"); + assertEquals(job.getTotalMaps(), mapsTotal, "mapsTotal incorrect"); + assertEquals(job.getCompletedMaps(), + mapsCompleted, "mapsCompleted incorrect"); + assertEquals(job.getTotalReduces(), reducesTotal, "reducesTotal incorrect"); + assertEquals(job.getCompletedReduces(), + reducesCompleted, "reducesCompleted incorrect"); + assertEquals(report.getMapProgress() * 100, mapProgress, 0, "mapProgress incorrect"); + assertEquals(report.getReduceProgress() * 100, + reduceProgress, 0, "reduceProgress incorrect"); } public void verifyAMJobGenericSecure(Job job, int mapsPending, @@ -603,28 +599,27 @@ public void verifyAMJobGenericSecure(Job job, int mapsPending, WebServicesTestUtils.checkStringMatch("diagnostics", diagString, diagnostics); - assertEquals("isUber incorrect", job.isUber(), uberized); + assertEquals(job.isUber(), uberized, "isUber incorrect"); // unfortunately the following fields are all calculated in JobInfo // so not easily accessible without doing all the calculations again. // For now just make sure they are present. - assertTrue("mapsPending not >= 0", mapsPending >= 0); - assertTrue("mapsRunning not >= 0", mapsRunning >= 0); - assertTrue("reducesPending not >= 0", reducesPending >= 0); - assertTrue("reducesRunning not >= 0", reducesRunning >= 0); + assertTrue(mapsPending >= 0, "mapsPending not >= 0"); + assertTrue(mapsRunning >= 0, "mapsRunning not >= 0"); + assertTrue(reducesPending >= 0, "reducesPending not >= 0"); + assertTrue(reducesRunning >= 0, "reducesRunning not >= 0"); - assertTrue("newReduceAttempts not >= 0", newReduceAttempts >= 0); - assertTrue("runningReduceAttempts not >= 0", runningReduceAttempts >= 0); - assertTrue("failedReduceAttempts not >= 0", failedReduceAttempts >= 0); - assertTrue("killedReduceAttempts not >= 0", killedReduceAttempts >= 0); - assertTrue("successfulReduceAttempts not >= 0", - successfulReduceAttempts >= 0); + assertTrue(newReduceAttempts >= 0, "newReduceAttempts not >= 0"); + assertTrue(runningReduceAttempts >= 0, "runningReduceAttempts not >= 0"); + assertTrue(failedReduceAttempts >= 0, "failedReduceAttempts not >= 0"); + assertTrue(killedReduceAttempts >= 0, "killedReduceAttempts not >= 0"); + assertTrue(successfulReduceAttempts >= 0, "successfulReduceAttempts not >= 0"); - assertTrue("newMapAttempts not >= 0", newMapAttempts >= 0); - assertTrue("runningMapAttempts not >= 0", runningMapAttempts >= 0); - assertTrue("failedMapAttempts not >= 0", failedMapAttempts >= 0); - assertTrue("killedMapAttempts not >= 0", killedMapAttempts >= 0); - assertTrue("successfulMapAttempts not >= 0", successfulMapAttempts >= 0); + assertTrue(newMapAttempts >= 0, "newMapAttempts not >= 0"); + assertTrue(runningMapAttempts >= 0, "runningMapAttempts not >= 0"); + assertTrue(failedMapAttempts >= 0, "failedMapAttempts not >= 0"); + assertTrue(killedMapAttempts >= 0, "killedMapAttempts not >= 0"); + assertTrue(successfulMapAttempts >= 0, "successfulMapAttempts not >= 0"); } @@ -640,7 +635,7 @@ public void testJobCounters() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobCounters"); verifyAMJobCounters(info, jobsMap.get(id)); } @@ -659,7 +654,7 @@ public void testJobCountersSlash() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobCounters"); verifyAMJobCounters(info, jobsMap.get(id)); } @@ -677,7 +672,7 @@ public void testJobCountersDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobCounters"); verifyAMJobCounters(info, jobsMap.get(id)); } @@ -709,7 +704,7 @@ public void testJobCountersXML() throws Exception { public void verifyAMJobCounters(JSONObject info, Job job) throws JSONException { - assertEquals("incorrect number of elements", 2, info.length()); + assertEquals(2, info.length(), "incorrect number of elements"); WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()), info.getString("id")); @@ -719,22 +714,22 @@ public void verifyAMJobCounters(JSONObject info, Job job) for (int i = 0; i < counterGroups.length(); i++) { JSONObject counterGroup = counterGroups.getJSONObject(i); String name = counterGroup.getString("counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); JSONArray counters = counterGroup.getJSONArray("counter"); for (int j = 0; j < counters.length(); j++) { JSONObject counter = counters.getJSONObject(j); String counterName = counter.getString("name"); - assertTrue("counter name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), + "counter name not set"); long mapValue = counter.getLong("mapCounterValue"); - assertTrue("mapCounterValue >= 0", mapValue >= 0); + assertTrue(mapValue >= 0, "mapCounterValue >= 0"); long reduceValue = counter.getLong("reduceCounterValue"); - assertTrue("reduceCounterValue >= 0", reduceValue >= 0); + assertTrue(reduceValue >= 0, "reduceCounterValue >= 0"); long totalValue = counter.getLong("totalCounterValue"); - assertTrue("totalCounterValue >= 0", totalValue >= 0); + assertTrue(totalValue >= 0, "totalCounterValue >= 0"); } } @@ -745,7 +740,7 @@ public void verifyAMJobCountersXML(NodeList nodes, Job job) { for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); - assertNotNull("Job not found - output incorrect", job); + assertNotNull(job, "Job not found - output incorrect"); WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()), WebServicesTestUtils.getXmlString(element, "id")); @@ -755,29 +750,29 @@ public void verifyAMJobCountersXML(NodeList nodes, Job job) { for (int j = 0; j < groups.getLength(); j++) { Element counters = (Element) groups.item(j); - assertNotNull("should have counters in the web service info", counters); + assertNotNull(counters, "should have counters in the web service info"); String name = WebServicesTestUtils.getXmlString(counters, "counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); NodeList counterArr = counters.getElementsByTagName("counter"); for (int z = 0; z < counterArr.getLength(); z++) { Element counter = (Element) counterArr.item(z); String counterName = WebServicesTestUtils.getXmlString(counter, "name"); - assertTrue("counter name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), + "counter name not set"); long mapValue = WebServicesTestUtils.getXmlLong(counter, "mapCounterValue"); - assertTrue("mapCounterValue not >= 0", mapValue >= 0); + assertTrue(mapValue >= 0, "mapCounterValue not >= 0"); long reduceValue = WebServicesTestUtils.getXmlLong(counter, "reduceCounterValue"); - assertTrue("reduceCounterValue >= 0", reduceValue >= 0); + assertTrue(reduceValue >= 0, "reduceCounterValue >= 0"); long totalValue = WebServicesTestUtils.getXmlLong(counter, "totalCounterValue"); - assertTrue("totalCounterValue >= 0", totalValue >= 0); + assertTrue(totalValue >= 0, "totalCounterValue >= 0"); } } } @@ -796,7 +791,7 @@ public void testJobAttempts() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobAttempts"); verifyJobAttempts(info, jobsMap.get(id)); } @@ -814,7 +809,7 @@ public void testJobAttemptsSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobAttempts"); verifyJobAttempts(info, jobsMap.get(id)); } @@ -833,7 +828,7 @@ public void testJobAttemptsDefault() throws Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobAttempts"); verifyJobAttempts(info, jobsMap.get(id)); } @@ -858,7 +853,7 @@ public void testJobAttemptsXML() throws Exception { is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList attempts = dom.getElementsByTagName("jobAttempts"); - assertEquals("incorrect number of elements", 1, attempts.getLength()); + assertEquals(1, attempts.getLength(), "incorrect number of elements"); NodeList info = dom.getElementsByTagName("jobAttempt"); verifyJobAttemptsXML(info, jobsMap.get(id)); } @@ -868,7 +863,7 @@ public void verifyJobAttempts(JSONObject info, Job job) throws JSONException { JSONArray attempts = info.getJSONArray("jobAttempt"); - assertEquals("incorrect number of elements", 2, attempts.length()); + assertEquals(2, attempts.length(), "incorrect number of elements"); for (int i = 0; i < attempts.length(); i++) { JSONObject attempt = attempts.getJSONObject(i); verifyJobAttemptsGeneric(job, attempt.getString("nodeHttpAddress"), @@ -880,7 +875,7 @@ public void verifyJobAttempts(JSONObject info, Job job) public void verifyJobAttemptsXML(NodeList nodes, Job job) { - assertEquals("incorrect number of elements", 2, nodes.getLength()); + assertEquals(2, nodes.getLength(), "incorrect number of elements"); for (int i = 0; i < nodes.getLength(); i++) { Element element = (Element) nodes.item(i); verifyJobAttemptsGeneric(job, @@ -906,17 +901,17 @@ public void verifyJobAttemptsGeneric(Job job, String nodeHttpAddress, + nmHttpPort, nodeHttpAddress); WebServicesTestUtils.checkStringMatch("nodeId", NodeId.newInstance(nmHost, nmPort).toString(), nodeId); - assertTrue("startime not greater than 0", startTime > 0); + assertTrue(startTime > 0, "start time not greater than 0"); WebServicesTestUtils.checkStringMatch("containerId", amInfo .getContainerId().toString(), containerId); String localLogsLink =ujoin("node", "containerlogs", containerId, job.getUserName()); - assertTrue("logsLink", logsLink.contains(localLogsLink)); + assertTrue(logsLink.contains(localLogsLink), "logsLink"); } } - assertTrue("attempt: " + id + " was not found", attemptFound); + assertTrue(attemptFound, "attempt: " + id + " was not found"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java index a8903c8cada10..3e99c98c9393a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java @@ -19,9 +19,9 @@ package org.apache.hadoop.mapreduce.v2.app.webapp; import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import java.io.StringReader; @@ -56,7 +56,7 @@ import org.codehaus.jettison.json.JSONArray; import org.codehaus.jettison.json.JSONException; import org.codehaus.jettison.json.JSONObject; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.w3c.dom.Document; import org.w3c.dom.Element; import org.w3c.dom.NodeList; @@ -119,10 +119,10 @@ public void testTasks() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject tasks = json.getJSONObject("tasks"); JSONArray arr = tasks.getJSONArray("task"); - assertEquals("incorrect number of elements", 2, arr.length()); + assertEquals(2, arr.length(), "incorrect number of elements"); verifyAMTask(arr, jobsMap.get(id), null); } @@ -139,10 +139,10 @@ public void testTasksDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject tasks = json.getJSONObject("tasks"); JSONArray arr = tasks.getJSONArray("task"); - assertEquals("incorrect number of elements", 2, arr.length()); + assertEquals(2, arr.length(), "incorrect number of elements"); verifyAMTask(arr, jobsMap.get(id), null); } @@ -160,10 +160,10 @@ public void testTasksSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject tasks = json.getJSONObject("tasks"); JSONArray arr = tasks.getJSONArray("task"); - assertEquals("incorrect number of elements", 2, arr.length()); + assertEquals(2, arr.length(), "incorrect number of elements"); verifyAMTask(arr, jobsMap.get(id), null); } @@ -188,7 +188,7 @@ public void testTasksXML() throws JSONException, Exception { is.setCharacterStream(new StringReader(xml)); Document dom = db.parse(is); NodeList tasks = dom.getElementsByTagName("tasks"); - assertEquals("incorrect number of elements", 1, tasks.getLength()); + assertEquals(1, tasks.getLength(), "incorrect number of elements"); NodeList task = dom.getElementsByTagName("task"); verifyAMTaskXML(task, jobsMap.get(id)); } @@ -207,12 +207,12 @@ public void testTasksQueryMap() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject tasks = json.getJSONObject("tasks"); JSONObject task = tasks.getJSONObject("task"); JSONArray arr = new JSONArray(); arr.put(task); - assertEquals("incorrect number of elements", 1, arr.length()); + assertEquals(1, arr.length(), "incorrect number of elements"); verifyAMTask(arr, jobsMap.get(id), type); } } @@ -230,12 +230,12 @@ public void testTasksQueryReduce() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject tasks = json.getJSONObject("tasks"); JSONObject task = tasks.getJSONObject("task"); JSONArray arr = new JSONArray(); arr.put(task); - assertEquals("incorrect number of elements", 1, arr.length()); + assertEquals(1, arr.length(), "incorrect number of elements"); verifyAMTask(arr, jobsMap.get(id), type); } } @@ -261,7 +261,7 @@ public void testTasksQueryInvalid() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -290,7 +290,7 @@ public void testTaskId() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("task"); verifyAMSingleTask(info, task); } @@ -312,7 +312,7 @@ public void testTaskIdSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("task"); verifyAMSingleTask(info, task); } @@ -334,7 +334,7 @@ public void testTaskIdDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("task"); verifyAMSingleTask(info, task); } @@ -360,7 +360,7 @@ public void testTaskIdBogus() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -396,7 +396,7 @@ public void testTaskIdNonExist() throws JSONException, Exception { String entity = response.readEntity(String.class); JSONObject msg = new JSONObject(entity); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -427,7 +427,7 @@ public void testTaskIdInvalid() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -462,7 +462,7 @@ public void testTaskIdInvalid2() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -496,7 +496,7 @@ public void testTaskIdInvalid3() throws JSONException, Exception { response.getMediaType().toString()); JSONObject msg = response.readEntity(JSONObject.class); JSONObject exception = msg.getJSONObject("RemoteException"); - assertEquals("incorrect number of elements", 3, exception.length()); + assertEquals(3, exception.length(), "incorrect number of elements"); String message = exception.getString("message"); String type = exception.getString("exception"); String classname = exception.getString("javaClassName"); @@ -545,7 +545,7 @@ public void testTaskIdXML() throws Exception { public void verifyAMSingleTask(JSONObject info, Task task) throws JSONException { - assertEquals("incorrect number of elements", 9, info.length()); + assertEquals(9, info.length(), "incorrect number of elements"); verifyTaskGeneric(task, info.getString("id"), info.getString("state"), info.getString("type"), info.getString("successfulAttempt"), @@ -569,7 +569,7 @@ public void verifyAMTask(JSONArray arr, Job job, String type) verifyAMSingleTask(info, task); } } - assertTrue("task with id: " + tid + " not in web service output", found); + assertTrue(found, "task with id: " + tid + " not in web service output"); } } } @@ -588,12 +588,12 @@ public void verifyTaskGeneric(Task task, String id, String state, WebServicesTestUtils.checkStringMatch("state", report.getTaskState() .toString(), state); // not easily checked without duplicating logic, just make sure its here - assertNotNull("successfulAttempt null", successfulAttempt); - assertEquals("startTime wrong", report.getStartTime(), startTime); - assertEquals("finishTime wrong", report.getFinishTime(), finishTime); - assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime); - assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f); - assertEquals("status wrong", report.getStatus(), status); + assertNotNull(successfulAttempt, "successfulAttempt null"); + assertEquals(report.getStartTime(), startTime, "startTime wrong"); + assertEquals(report.getFinishTime(), finishTime, "finishTime wrong"); + assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong"); + assertEquals(report.getProgress() * 100, progress, 1e-3f, "progress wrong"); + assertEquals(report.getStatus(), status, "status wrong"); } public void verifyAMSingleTaskXML(Element element, Task task) { @@ -610,7 +610,7 @@ public void verifyAMSingleTaskXML(Element element, Task task) { public void verifyAMTaskXML(NodeList nodes, Job job) { - assertEquals("incorrect number of elements", 2, nodes.getLength()); + assertEquals(2, nodes.getLength(), "incorrect number of elements"); for (Task task : job.getTasks().values()) { TaskId id = task.getID(); @@ -624,7 +624,7 @@ public void verifyAMTaskXML(NodeList nodes, Job job) { verifyAMSingleTaskXML(element, task); } } - assertTrue("task with id: " + tid + " not in web service output", found); + assertTrue(found, "task with id: " + tid + " not in web service output"); } } @@ -643,7 +643,7 @@ public void testTaskIdCounters() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobTaskCounters"); verifyAMJobTaskCounters(info, task); } @@ -665,7 +665,7 @@ public void testTaskIdCountersSlash() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobTaskCounters"); verifyAMJobTaskCounters(info, task); } @@ -687,7 +687,7 @@ public void testTaskIdCountersDefault() throws JSONException, Exception { assertEquals(MediaType.APPLICATION_JSON_TYPE + ";" + JettyUtils.UTF_8, response.getMediaType().toString()); JSONObject json = response.readEntity(JSONObject.class); - assertEquals("incorrect number of elements", 1, json.length()); + assertEquals(1, json.length(), "incorrect number of elements"); JSONObject info = json.getJSONObject("jobTaskCounters"); verifyAMJobTaskCounters(info, task); } @@ -723,7 +723,7 @@ public void testJobTaskCountersXML() throws Exception { public void verifyAMJobTaskCounters(JSONObject info, Task task) throws JSONException { - assertEquals("incorrect number of elements", 2, info.length()); + assertEquals(2, info.length(), "incorrect number of elements"); WebServicesTestUtils.checkStringMatch("id", MRApps.toString(task.getID()), info.getString("id")); @@ -733,15 +733,14 @@ public void verifyAMJobTaskCounters(JSONObject info, Task task) for (int i = 0; i < counterGroups.length(); i++) { JSONObject counterGroup = counterGroups.getJSONObject(i); String name = counterGroup.getString("counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); JSONArray counters = counterGroup.getJSONArray("counter"); for (int j = 0; j < counters.length(); j++) { JSONObject counter = counters.getJSONObject(j); String counterName = counter.getString("name"); - assertTrue("name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), "name not set"); long value = counter.getLong("value"); - assertTrue("value >= 0", value >= 0); + assertTrue(value >= 0, "value >= 0"); } } } @@ -760,20 +759,19 @@ public void verifyAMTaskCountersXML(NodeList nodes, Task task) { for (int j = 0; j < groups.getLength(); j++) { Element counters = (Element) groups.item(j); - assertNotNull("should have counters in the web service info", counters); + assertNotNull(counters, "should have counters in the web service info"); String name = WebServicesTestUtils.getXmlString(counters, "counterGroupName"); - assertTrue("name not set", (name != null && !name.isEmpty())); + assertTrue((name != null && !name.isEmpty()), "name not set"); NodeList counterArr = counters.getElementsByTagName("counter"); for (int z = 0; z < counterArr.getLength(); z++) { Element counter = (Element) counterArr.item(z); String counterName = WebServicesTestUtils.getXmlString(counter, "name"); - assertTrue("counter name not set", - (counterName != null && !counterName.isEmpty())); + assertTrue((counterName != null && !counterName.isEmpty()), "counter name not set"); long value = WebServicesTestUtils.getXmlLong(counter, "value"); - assertTrue("value not >= 0", value >= 0); + assertTrue(value >= 0, "value not >= 0"); } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java index 473681c3e4241..ad0f09b284648 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java @@ -17,7 +17,13 @@ */ package org.apache.hadoop.mapreduce.v2.app.webapp; -import static org.mockito.Mockito.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.Iterator; @@ -37,9 +43,8 @@ import org.apache.hadoop.yarn.webapp.Controller.RequestContext; import org.apache.hadoop.yarn.webapp.MimeType; import org.apache.hadoop.yarn.webapp.ResponseInfo; -import org.junit.Before; -import org.junit.Test; -import static org.junit.Assert.*; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestAppController { @@ -48,7 +53,7 @@ public class TestAppController { private Job job; private static final String taskId = "task_01_01_m_01"; - @Before + @BeforeEach public void setUp() throws IOException { AppContext context = mock(AppContext.class); when(context.getApplicationID()).thenReturn( @@ -275,8 +280,7 @@ public void testConfiguration() { public void testDownloadConfiguration() { appController.downloadConf(); String jobConfXml = appController.getData(); - assertTrue("Error downloading the job configuration file.", - !jobConfXml.contains("Error")); + assertTrue(!jobConfXml.contains("Error"), "Error downloading the job configuration file."); } /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java index 82b8a37dbea8c..793924975eace 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java @@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.util.MRJobConfUtil; import org.apache.hadoop.yarn.webapp.View; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.v2.api.records.JobId; @@ -51,8 +51,12 @@ import org.apache.hadoop.yarn.webapp.view.HtmlBlock; import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block; -import static org.mockito.Mockito.*; -import static org.junit.Assert.*; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; public class TestBlocks { private ByteArrayOutputStream data = new ByteArrayOutputStream(); diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index a4a9b94c863d0..6c6b45ed1688c 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1321,6 +1321,11 @@ + + org.mockito + mockito-junit-jupiter + 4.11.0 + org.apache.avro avro diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java index 6aed17982c900..63e3679a5d1d0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java @@ -369,9 +369,15 @@ public void setup() { if (httpScheme.equals(WebAppUtils.HTTPS_PREFIX)) { String amKeystoreLoc = System.getenv("KEYSTORE_FILE_LOCATION"); + if (StringUtils.isBlank(amKeystoreLoc)) { + amKeystoreLoc = System.getProperty("KEYSTORE_FILE_LOCATION"); + } if (amKeystoreLoc != null) { LOG.info("Setting keystore location to " + amKeystoreLoc); String password = System.getenv("KEYSTORE_PASSWORD"); + if (StringUtils.isBlank(password)) { + password = System.getProperty("KEYSTORE_PASSWORD"); + } builder.keyStore(amKeystoreLoc, password, "jks"); } else { LOG.info("Loading standard ssl config"); @@ -380,9 +386,15 @@ public void setup() { builder.needsClientAuth(needsClientAuth); if (needsClientAuth) { String amTruststoreLoc = System.getenv("TRUSTSTORE_FILE_LOCATION"); + if (StringUtils.isBlank(amTruststoreLoc)) { + amTruststoreLoc = System.getProperty("TRUSTSTORE_FILE_LOCATION"); + } if (amTruststoreLoc != null) { LOG.info("Setting truststore location to " + amTruststoreLoc); String password = System.getenv("TRUSTSTORE_PASSWORD"); + if (StringUtils.isBlank(password)) { + password = System.getProperty("TRUSTSTORE_PASSWORD"); + } builder.trustStore(amTruststoreLoc, password, "jks"); } }