|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.timeline;
|
|
|
|
|
|
+import org.apache.commons.lang3.tuple.Pair;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileContext;
|
|
@@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
|
+import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
|
|
|
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;
|
|
@@ -50,8 +52,19 @@ import org.junit.Rule;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.rules.TestName;
|
|
|
|
|
|
+import com.fasterxml.jackson.annotation.JsonInclude;
|
|
|
+import com.fasterxml.jackson.core.JsonFactory;
|
|
|
+import com.fasterxml.jackson.core.JsonGenerator;
|
|
|
+import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
|
|
|
+import com.fasterxml.jackson.databind.ObjectMapper;
|
|
|
+import com.fasterxml.jackson.databind.type.TypeFactory;
|
|
|
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
|
|
|
+
|
|
|
+import java.io.DataInputStream;
|
|
|
+import java.io.DataOutputStream;
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.OutputStream;
|
|
|
import java.net.URL;
|
|
|
import java.net.URLClassLoader;
|
|
|
import java.util.ArrayList;
|
|
@@ -61,6 +74,8 @@ import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
+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;
|
|
@@ -602,6 +617,149 @@ public class TestEntityGroupFSTimelineStore extends TimelineStoreTestUtils {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ // TestTimelineStore to validate the put entities call
|
|
|
+ static class TestTimelineStore extends LeveldbTimelineStore {
|
|
|
+ static final AtomicInteger ENTITIES_COUNT = new AtomicInteger(0);
|
|
|
+
|
|
|
+ TestTimelineStore() {
|
|
|
+ super();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public TimelinePutResponse put(TimelineEntities entities) {
|
|
|
+ ENTITIES_COUNT.getAndAdd(entities.getEntities().size());
|
|
|
+ return new TimelinePutResponse();
|
|
|
+ }
|
|
|
+
|
|
|
+ public static int getEntitiesCount() {
|
|
|
+ return ENTITIES_COUNT.get();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testIfAnyDuplicateEntities() throws Exception {
|
|
|
+ // Create an application with some entities
|
|
|
+ ApplicationId appId =
|
|
|
+ ApplicationId.fromString("application_1501509265053_0002");
|
|
|
+ String user = UserGroupInformation.getCurrentUser().getShortUserName();
|
|
|
+ Path activeDirPath = getTestRootPath("active1");
|
|
|
+ Path doneDirPath = getTestRootPath("done1");
|
|
|
+ Path userBase = new Path(activeDirPath, user);
|
|
|
+ Path userAppRoot = new Path(userBase, appId.toString());
|
|
|
+ Path attemptDirPath = new Path(userAppRoot, getAttemptDirName(appId));
|
|
|
+
|
|
|
+ String logFileName = EntityGroupFSTimelineStore.ENTITY_LOG_PREFIX
|
|
|
+ + EntityGroupPlugInForTest.getStandardTimelineGroupId(appId);
|
|
|
+ createTestFiles(appId, attemptDirPath, logFileName);
|
|
|
+
|
|
|
+ // stop the default store before creating new store to get the lock
|
|
|
+ store.stop();
|
|
|
+ EntityGroupFSTimelineStore newStore = new EntityGroupFSTimelineStore() {
|
|
|
+ @Override
|
|
|
+ protected AppState getAppState(ApplicationId appId) throws IOException {
|
|
|
+ return AppState.ACTIVE;
|
|
|
+ }
|
|
|
+ };
|
|
|
+
|
|
|
+ try {
|
|
|
+ // Start ATS with TestTimelineStore
|
|
|
+ Configuration newConfig = new YarnConfiguration(config);
|
|
|
+ newConfig.set(YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_SUMMARY_STORE,
|
|
|
+ TestTimelineStore.class.getName());
|
|
|
+ newConfig.set(YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR,
|
|
|
+ doneDirPath.toString());
|
|
|
+ newConfig.set(
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_ACTIVE_DIR,
|
|
|
+ activeDirPath.toString());
|
|
|
+ newStore.init(newConfig);
|
|
|
+ newStore.setFs(fs);
|
|
|
+ newStore.start();
|
|
|
+
|
|
|
+ // Validate if the initial entities count are correct
|
|
|
+ newStore.scanActiveLogs();
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ return TestTimelineStore.getEntitiesCount() == 2;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ assertEquals("Wrong Initial Entities Count",
|
|
|
+ 2, TestTimelineStore.getEntitiesCount());
|
|
|
+
|
|
|
+ // 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);
|
|
|
+ jsonGenerator.setPrettyPrinter(new MinimalPrettyPrinter("\n"));
|
|
|
+ ObjectMapper objMapper = new ObjectMapper();
|
|
|
+ objMapper.setAnnotationIntrospector(
|
|
|
+ new JaxbAnnotationIntrospector(TypeFactory.defaultInstance()));
|
|
|
+ objMapper.setSerializationInclusion(JsonInclude.Include.NON_NULL);
|
|
|
+ for (TimelineEntity entity : entities.getEntities()) {
|
|
|
+ objMapper.writeValue(jsonGenerator, entity);
|
|
|
+ }
|
|
|
+ outStream.close();
|
|
|
+
|
|
|
+ // Validate if there are any duplicates
|
|
|
+ newStore.scanActiveLogs();
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ return TestTimelineStore.getEntitiesCount() == 4;
|
|
|
+ }
|
|
|
+ }, 100, 10000);
|
|
|
+ assertEquals("Duplicate Entities present",
|
|
|
+ 4, TestTimelineStore.getEntitiesCount());
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (newStore != null) {
|
|
|
+ newStore.stop();
|
|
|
+ }
|
|
|
+ fs.delete(userAppRoot, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testStateStoreAndRecovery() throws Exception {
|
|
|
+ // Prepare the AppLogs Data
|
|
|
+ EntityGroupFSTimelineStore.AppLogs appLogs =
|
|
|
+ store.new AppLogs(mainTestAppId, mainTestAppDirPath, AppState.COMPLETED);
|
|
|
+ appLogs.scanForLogs();
|
|
|
+ List<LogInfo> summaryLogs = appLogs.getSummaryLogs();
|
|
|
+ List<EntityGroupFSTimelineStore.AppLogs> logsList = new ArrayList<>();
|
|
|
+ logsList.add(appLogs);
|
|
|
+
|
|
|
+ // Store the Log files
|
|
|
+ Path checkpointFile = new Path(fs.getHomeDirectory(), "atscheckpoint");
|
|
|
+ try (DataOutputStream dataOutputStream = fs.create(checkpointFile)) {
|
|
|
+ store.storeLogFiles(logsList, dataOutputStream);
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.fail("Failed to store the log files");
|
|
|
+ }
|
|
|
+
|
|
|
+ // Recover the Log files and validate the contents
|
|
|
+ try (DataInputStream dataInputStream = fs.open(checkpointFile)) {
|
|
|
+ HashMap<String, Pair<Long, Long>> logFiles =
|
|
|
+ store.recoverLogFiles(dataInputStream);
|
|
|
+ assertEquals(summaryLogs.size(), logFiles.size());
|
|
|
+ for (LogInfo logInfo : summaryLogs) {
|
|
|
+ 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());
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ Assert.fail("Failed to recover the log files");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
private EntityGroupFSTimelineStore createAndStartTimelineStore(
|
|
|
AppState appstate) {
|
|
|
// stop before creating new store to get the lock
|