|
@@ -43,14 +43,12 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.server.timeline.EntityGroupFSTimelineStore.AppState;
|
|
|
import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
-import org.junit.After;
|
|
|
-import org.junit.Assert;
|
|
|
-import org.junit.AfterClass;
|
|
|
-import org.junit.Before;
|
|
|
-import org.junit.BeforeClass;
|
|
|
-import org.junit.Rule;
|
|
|
-import org.junit.Test;
|
|
|
-import org.junit.rules.TestName;
|
|
|
+import org.junit.jupiter.api.AfterAll;
|
|
|
+import org.junit.jupiter.api.AfterEach;
|
|
|
+import org.junit.jupiter.api.BeforeAll;
|
|
|
+import org.junit.jupiter.api.BeforeEach;
|
|
|
+import org.junit.jupiter.api.Test;
|
|
|
+import org.junit.jupiter.api.TestInfo;
|
|
|
|
|
|
import com.fasterxml.jackson.annotation.JsonInclude;
|
|
|
import com.fasterxml.jackson.core.JsonFactory;
|
|
@@ -78,11 +76,12 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.function.Supplier;
|
|
|
|
|
|
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.assertNotEquals;
|
|
|
-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.assertFalse;
|
|
|
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
|
|
+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 static org.mockito.Mockito.when;
|
|
|
|
|
@@ -118,12 +117,10 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
private EntityGroupFSTimelineStore store;
|
|
|
private TimelineEntity entityNew;
|
|
|
|
|
|
- @Rule
|
|
|
- public TestName currTestName = new TestName();
|
|
|
private File rootDir;
|
|
|
private File testJar;
|
|
|
|
|
|
- @BeforeClass
|
|
|
+ @BeforeAll
|
|
|
public static void setupClass() throws Exception {
|
|
|
config.setBoolean(YarnConfiguration.TIMELINE_SERVICE_TTL_ENABLE, false);
|
|
|
config.set(
|
|
@@ -163,8 +160,8 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
testActiveDirPath.toString());
|
|
|
}
|
|
|
|
|
|
- @Before
|
|
|
- public void setup() throws Exception {
|
|
|
+ @BeforeEach
|
|
|
+ public void setup(TestInfo testInfo) throws Exception {
|
|
|
for (ApplicationId appId : sampleAppIds) {
|
|
|
Path attemotDirPath =
|
|
|
new Path(new Path(testActiveDirPath, appId.toString()),
|
|
@@ -173,7 +170,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
store = new EntityGroupFSTimelineStore();
|
|
|
- if (currTestName.getMethodName().contains("Plugin")) {
|
|
|
+ if (testInfo.getTestMethod().get().getName().contains("Plugin")) {
|
|
|
rootDir = GenericTestUtils.getTestDir(getClass()
|
|
|
.getSimpleName());
|
|
|
if (!rootDir.exists()) {
|
|
@@ -201,7 +198,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
store.start();
|
|
|
}
|
|
|
|
|
|
- @After
|
|
|
+ @AfterEach
|
|
|
public void tearDown() throws Exception {
|
|
|
store.stop();
|
|
|
for (ApplicationId appId : sampleAppIds) {
|
|
@@ -213,7 +210,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @AfterClass
|
|
|
+ @AfterAll
|
|
|
public static void tearDownClass() throws Exception {
|
|
|
hdfsCluster.shutdown();
|
|
|
FileContext fileContext = FileContext.getLocalFSFileContext();
|
|
@@ -222,10 +219,10 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testAppLogsScanLogs() throws Exception {
|
|
|
+ void testAppLogsScanLogs() throws Exception {
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ AppState.COMPLETED);
|
|
|
appLogs.scanForLogs();
|
|
|
List<LogInfo> summaryLogs = appLogs.getSummaryLogs();
|
|
|
List<LogInfo> detailLogs = appLogs.getDetailLogs();
|
|
@@ -245,13 +242,13 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testAppLogsDomainLogLastlyScanned() throws Exception {
|
|
|
+ void testAppLogsDomainLogLastlyScanned() throws Exception {
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
- store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
+ AppState.COMPLETED);
|
|
|
Path attemptDirPath = new Path(new Path(testActiveDirPath,
|
|
|
mainTestAppId.toString()),
|
|
|
- getAttemptDirName(mainTestAppId));
|
|
|
+ getAttemptDirName(mainTestAppId));
|
|
|
//Delete the domain log from AppDirPath so first scan won't find it
|
|
|
fs.delete(new Path(attemptDirPath, TEST_DOMAIN_LOG_FILE_NAME), false);
|
|
|
appLogs.scanForLogs();
|
|
@@ -261,7 +258,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
|
|
|
//Generate the domain log
|
|
|
FSDataOutputStream out = fs.create(
|
|
|
- new Path(attemptDirPath, TEST_DOMAIN_LOG_FILE_NAME));
|
|
|
+ new Path(attemptDirPath, TEST_DOMAIN_LOG_FILE_NAME));
|
|
|
out.close();
|
|
|
|
|
|
appLogs.scanForLogs();
|
|
@@ -270,10 +267,10 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testMoveToDone() throws Exception {
|
|
|
+ void testMoveToDone() throws Exception {
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ AppState.COMPLETED);
|
|
|
Path pathBefore = appLogs.getAppDirPath();
|
|
|
appLogs.moveToDone();
|
|
|
Path pathAfter = appLogs.getAppDirPath();
|
|
@@ -284,13 +281,13 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testParseSummaryLogs() throws Exception {
|
|
|
+ void testParseSummaryLogs() throws Exception {
|
|
|
TimelineDataManager tdm = PluginStoreTestUtils.getTdmWithMemStore(config);
|
|
|
MutableCounterLong scanned = store.metrics.getEntitiesReadToSummary();
|
|
|
long beforeScan = scanned.value();
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ AppState.COMPLETED);
|
|
|
appLogs.scanForLogs();
|
|
|
appLogs.parseSummaryLogs(tdm);
|
|
|
PluginStoreTestUtils.verifyTestEntities(tdm);
|
|
@@ -298,31 +295,31 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testWithAnonymousUser() throws Exception {
|
|
|
+ void testWithAnonymousUser() throws Exception {
|
|
|
try {
|
|
|
TimelineDataManager tdm = PluginStoreTestUtils.getTdmWithMemStore(config);
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
- store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
+ AppState.COMPLETED);
|
|
|
FileStatus fileStatus = mock(FileStatus.class);
|
|
|
when(fileStatus.getOwner()).thenReturn(null);
|
|
|
appLogs.scanForLogs();
|
|
|
appLogs.parseSummaryLogs(tdm);
|
|
|
PluginStoreTestUtils.verifyTestEntities(tdm);
|
|
|
} catch (IllegalArgumentException ie) {
|
|
|
- Assert.fail("No exception needs to be thrown as anonymous user is configured");
|
|
|
+ fail("No exception needs to be thrown as anonymous user is configured");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testCleanLogs() throws Exception {
|
|
|
+ void testCleanLogs() throws Exception {
|
|
|
// Create test dirs and files
|
|
|
// Irrelevant file, should not be reclaimed
|
|
|
String appDirName = mainTestAppId.toString();
|
|
|
String attemptDirName = ApplicationAttemptId.appAttemptIdStrPrefix
|
|
|
+ appDirName + "_1";
|
|
|
Path irrelevantFilePath = new Path(
|
|
|
- testDoneDirPath, "irrelevant.log");
|
|
|
+ testDoneDirPath, "irrelevant.log");
|
|
|
FSDataOutputStream stream = fs.create(irrelevantFilePath);
|
|
|
stream.close();
|
|
|
// Irrelevant directory, should not be reclaimed
|
|
@@ -395,7 +392,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testCleanBuckets() throws Exception {
|
|
|
+ void testCleanBuckets() throws Exception {
|
|
|
// ClusterTimeStampDir with App Log Dirs
|
|
|
Path clusterTimeStampDir1 = new Path(testDoneDirPath,
|
|
|
Long.toString(sampleAppIds.get(0).getClusterTimestamp()));
|
|
@@ -441,17 +438,17 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testNullCheckGetEntityTimelines() throws Exception {
|
|
|
+ void testNullCheckGetEntityTimelines() throws Exception {
|
|
|
try {
|
|
|
store.getEntityTimelines("YARN_APPLICATION", null, null, null, null,
|
|
|
null);
|
|
|
} catch (NullPointerException e) {
|
|
|
- Assert.fail("NPE when getEntityTimelines called with Null EntityIds");
|
|
|
+ fail("NPE when getEntityTimelines called with Null EntityIds");
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testPluginRead() throws Exception {
|
|
|
+ void testPluginRead() throws Exception {
|
|
|
// Verify precondition
|
|
|
assertEquals(EntityGroupPlugInForTest.class.getName(),
|
|
|
store.getConfig().get(
|
|
@@ -459,8 +456,8 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
List<TimelineEntityGroupPlugin> currPlugins = store.getPlugins();
|
|
|
for (TimelineEntityGroupPlugin plugin : currPlugins) {
|
|
|
ClassLoader pluginClassLoader = plugin.getClass().getClassLoader();
|
|
|
- assertTrue("Should set up ApplicationClassLoader",
|
|
|
- pluginClassLoader instanceof ApplicationClassLoader);
|
|
|
+ assertTrue(pluginClassLoader instanceof ApplicationClassLoader,
|
|
|
+ "Should set up ApplicationClassLoader");
|
|
|
URL[] paths = ((URLClassLoader) pluginClassLoader).getURLs();
|
|
|
boolean foundJAR = false;
|
|
|
for (URL path : paths) {
|
|
@@ -468,13 +465,13 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
foundJAR = true;
|
|
|
}
|
|
|
}
|
|
|
- assertTrue("Not found path " + testJar.getAbsolutePath()
|
|
|
- + " for plugin " + plugin.getClass().getName(), foundJAR);
|
|
|
+ assertTrue(foundJAR, "Not found path " + testJar.getAbsolutePath()
|
|
|
+ + " for plugin " + plugin.getClass().getName());
|
|
|
}
|
|
|
// Load data and cache item, prepare timeline store by making a cache item
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ AppState.COMPLETED);
|
|
|
EntityCacheItem cacheItem = new EntityCacheItem(
|
|
|
EntityGroupPlugInForTest.getStandardTimelineGroupId(mainTestAppId),
|
|
|
config);
|
|
@@ -514,11 +511,11 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testSummaryRead() throws Exception {
|
|
|
+ void testSummaryRead() throws Exception {
|
|
|
// Load data
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath,
|
|
|
- AppState.COMPLETED);
|
|
|
+ AppState.COMPLETED);
|
|
|
MutableCounterLong summaryLogEntityRead
|
|
|
= store.metrics.getGetEntityToSummaryOps();
|
|
|
long numEntityReadBefore = summaryLogEntityRead.value();
|
|
@@ -543,7 +540,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testGetEntityPluginRead() throws Exception {
|
|
|
+ void testGetEntityPluginRead() throws Exception {
|
|
|
EntityGroupFSTimelineStore store = null;
|
|
|
ApplicationId appId =
|
|
|
ApplicationId.fromString("application_1501509265053_0001");
|
|
@@ -571,7 +568,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testScanActiveLogsWithInvalidFile() throws Exception {
|
|
|
+ void testScanActiveLogsWithInvalidFile() throws Exception {
|
|
|
Path invalidFile = new Path(testActiveDirPath, "invalidfile");
|
|
|
try {
|
|
|
if (!fs.exists(invalidFile)) {
|
|
@@ -579,7 +576,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
store.scanActiveLogs();
|
|
|
} catch (StackOverflowError error) {
|
|
|
- Assert.fail("EntityLogScanner crashed with StackOverflowError");
|
|
|
+ fail("EntityLogScanner crashed with StackOverflowError");
|
|
|
} finally {
|
|
|
if (fs.exists(invalidFile)) {
|
|
|
fs.delete(invalidFile, false);
|
|
@@ -588,7 +585,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testScanActiveLogsAndMoveToDonePluginRead() throws Exception {
|
|
|
+ void testScanActiveLogsAndMoveToDonePluginRead() throws Exception {
|
|
|
EntityGroupFSTimelineStore store = null;
|
|
|
ApplicationId appId =
|
|
|
ApplicationId.fromString("application_1501509265053_0002");
|
|
@@ -637,7 +634,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testIfAnyDuplicateEntities() throws Exception {
|
|
|
+ void testIfAnyDuplicateEntities() throws Exception {
|
|
|
// Create an application with some entities
|
|
|
ApplicationId appId =
|
|
|
ApplicationId.fromString("application_1501509265053_0002");
|
|
@@ -683,15 +680,14 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
return TestTimelineStore.getEntitiesCount() == 2;
|
|
|
}
|
|
|
}, 100, 10000);
|
|
|
- assertEquals("Wrong Initial Entities Count",
|
|
|
- 2, TestTimelineStore.getEntitiesCount());
|
|
|
+ assertEquals(2, TestTimelineStore.getEntitiesCount(), "Wrong Initial Entities Count");
|
|
|
|
|
|
// Append the Summary log file with few more entities
|
|
|
TimelineEntities entities = PluginStoreTestUtils.generateTestEntities();
|
|
|
FSDataOutputStream outStream = fs.append(
|
|
|
new Path(attemptDirPath, TEST_SUMMARY_LOG_FILE_NAME));
|
|
|
JsonGenerator jsonGenerator
|
|
|
- = new JsonFactory().createGenerator((OutputStream)outStream);
|
|
|
+ = new JsonFactory().createGenerator((OutputStream) outStream);
|
|
|
jsonGenerator.setPrettyPrinter(new MinimalPrettyPrinter("\n"));
|
|
|
ObjectMapper objMapper = new ObjectMapper();
|
|
|
objMapper.setAnnotationIntrospector(
|
|
@@ -710,8 +706,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
return TestTimelineStore.getEntitiesCount() == 4;
|
|
|
}
|
|
|
}, 100, 10000);
|
|
|
- assertEquals("Duplicate Entities present",
|
|
|
- 4, TestTimelineStore.getEntitiesCount());
|
|
|
+ assertEquals(4, TestTimelineStore.getEntitiesCount(), "Duplicate Entities present");
|
|
|
|
|
|
} finally {
|
|
|
if (newStore != null) {
|
|
@@ -722,7 +717,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testStateStoreAndRecovery() throws Exception {
|
|
|
+ void testStateStoreAndRecovery() throws Exception {
|
|
|
// Prepare the AppLogs Data
|
|
|
EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
store.new AppLogs(mainTestAppId, mainTestAppDirPath, AppState.COMPLETED);
|
|
@@ -736,7 +731,7 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
try (DataOutputStream dataOutputStream = fs.create(checkpointFile)) {
|
|
|
store.storeLogFiles(logsList, dataOutputStream);
|
|
|
} catch (IOException e) {
|
|
|
- Assert.fail("Failed to store the log files");
|
|
|
+ fail("Failed to store the log files");
|
|
|
}
|
|
|
|
|
|
// Recover the Log files and validate the contents
|
|
@@ -748,14 +743,14 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
String logFileName = logInfo.getAttemptDirName() +
|
|
|
Path.SEPARATOR + logInfo.getFilename();
|
|
|
Pair<Long, Long> pair = logFiles.get(logFileName);
|
|
|
- assertNotNull("Failed to recover " + logFileName, pair);
|
|
|
- assertTrue("LastProcessedTime is not same",
|
|
|
- logInfo.getLastProcessedTime() == pair.getLeft());
|
|
|
- assertTrue("Offset is not same",
|
|
|
- logInfo.getOffset() == pair.getRight());
|
|
|
+ assertNotNull(pair, "Failed to recover " + logFileName);
|
|
|
+ assertTrue(logInfo.getLastProcessedTime() == pair.getLeft(),
|
|
|
+ "LastProcessedTime is not same");
|
|
|
+ assertTrue(logInfo.getOffset() == pair.getRight(),
|
|
|
+ "Offset is not same");
|
|
|
}
|
|
|
} catch (IOException e) {
|
|
|
- Assert.fail("Failed to recover the log files");
|
|
|
+ fail("Failed to recover the log files");
|
|
|
}
|
|
|
}
|
|
|
|