|
@@ -25,14 +25,19 @@ import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.classification.VisibleForTesting;
|
|
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
import org.apache.commons.lang3.mutable.MutableBoolean;
|
|
import org.apache.commons.lang3.mutable.MutableBoolean;
|
|
|
|
+import org.apache.commons.lang3.tuple.Pair;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
import org.apache.commons.lang3.StringUtils;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.RemoteIterator;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
|
+import org.apache.hadoop.io.LongWritable;
|
|
|
|
+import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.service.CompositeService;
|
|
import org.apache.hadoop.service.CompositeService;
|
|
import org.apache.hadoop.service.ServiceOperations;
|
|
import org.apache.hadoop.service.ServiceOperations;
|
|
import org.apache.hadoop.ipc.CallerContext;
|
|
import org.apache.hadoop.ipc.CallerContext;
|
|
@@ -59,6 +64,8 @@ import org.apache.hadoop.yarn.util.Apps;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
+import java.io.DataInputStream;
|
|
|
|
+import java.io.DataOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.lang.reflect.UndeclaredThrowableException;
|
|
import java.lang.reflect.UndeclaredThrowableException;
|
|
import java.net.MalformedURLException;
|
|
import java.net.MalformedURLException;
|
|
@@ -70,6 +77,7 @@ import java.util.Arrays;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.Collections;
|
|
import java.util.Collections;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
|
|
+import java.util.HashMap;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.LinkedHashMap;
|
|
import java.util.LinkedHashMap;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
@@ -132,6 +140,11 @@ public class EntityGroupFSTimelineStore extends CompositeService
|
|
private long logRetainMillis;
|
|
private long logRetainMillis;
|
|
private long unknownActiveMillis;
|
|
private long unknownActiveMillis;
|
|
private int appCacheMaxSize = 0;
|
|
private int appCacheMaxSize = 0;
|
|
|
|
+ private boolean recoveryEnabled;
|
|
|
|
+ private Path checkpointFile;
|
|
|
|
+ private ConcurrentMap<String, Pair<Long, Long>> recoveredLogs =
|
|
|
|
+ new ConcurrentHashMap<String, Pair<Long, Long>>();
|
|
|
|
+
|
|
private List<TimelineEntityGroupPlugin> cacheIdPlugins;
|
|
private List<TimelineEntityGroupPlugin> cacheIdPlugins;
|
|
private Map<TimelineEntityGroupId, EntityCacheItem> cachedLogs;
|
|
private Map<TimelineEntityGroupId, EntityCacheItem> cachedLogs;
|
|
private boolean aclsEnabled;
|
|
private boolean aclsEnabled;
|
|
@@ -205,6 +218,11 @@ public class EntityGroupFSTimelineStore extends CompositeService
|
|
YarnConfiguration
|
|
YarnConfiguration
|
|
.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT));
|
|
.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_DONE_DIR_DEFAULT));
|
|
fs = activeRootPath.getFileSystem(conf);
|
|
fs = activeRootPath.getFileSystem(conf);
|
|
|
|
+ checkpointFile = new Path(fs.getHomeDirectory(), "atscheckpoint");
|
|
|
|
+ recoveryEnabled = conf.getBoolean(
|
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RECOVERY_ENABLED,
|
|
|
|
+ YarnConfiguration.TIMELINE_SERVICE_ENTITYGROUP_FS_STORE_RECOVERY_ENABLED_DEFAULT);
|
|
|
|
+
|
|
aclsEnabled = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
|
|
aclsEnabled = conf.getBoolean(YarnConfiguration.YARN_ACL_ENABLE,
|
|
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
|
|
YarnConfiguration.DEFAULT_YARN_ACL_ENABLE);
|
|
CallerContext.setCurrent(
|
|
CallerContext.setCurrent(
|
|
@@ -293,6 +311,15 @@ public class EntityGroupFSTimelineStore extends CompositeService
|
|
fs.setPermission(doneRootPath, DONE_DIR_PERMISSION);
|
|
fs.setPermission(doneRootPath, DONE_DIR_PERMISSION);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ // Recover the lastProcessedTime and offset for logfiles
|
|
|
|
+ if (recoveryEnabled && fs.exists(checkpointFile)) {
|
|
|
|
+ try (FSDataInputStream in = fs.open(checkpointFile)) {
|
|
|
|
+ recoveredLogs.putAll(recoverLogFiles(in));
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.warn("Failed to recover summarylog files from the checkpointfile", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
objMapper = new ObjectMapper();
|
|
objMapper = new ObjectMapper();
|
|
objMapper.setAnnotationIntrospector(
|
|
objMapper.setAnnotationIntrospector(
|
|
new JaxbAnnotationIntrospector(TypeFactory.defaultInstance()));
|
|
new JaxbAnnotationIntrospector(TypeFactory.defaultInstance()));
|
|
@@ -352,10 +379,62 @@ public class EntityGroupFSTimelineStore extends CompositeService
|
|
super.serviceStop();
|
|
super.serviceStop();
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /* Returns Map of SummaryLog files. The Value Pair has
|
|
|
|
+ lastProcessedTime and offset */
|
|
|
|
+ HashMap<String, Pair<Long, Long>> recoverLogFiles(
|
|
|
|
+ DataInputStream in) throws IOException {
|
|
|
|
+ HashMap<String, Pair<Long, Long>> logFiles = new HashMap<>();
|
|
|
|
+ long totalEntries = in.readLong();
|
|
|
|
+ for (long i = 0; i < totalEntries; i++) {
|
|
|
|
+ Text attemptDirName = new Text();
|
|
|
|
+ attemptDirName.readFields(in);
|
|
|
|
+ Text fileName = new Text();
|
|
|
|
+ fileName.readFields(in);
|
|
|
|
+ LongWritable lastProcessedTime = new LongWritable();
|
|
|
|
+ lastProcessedTime.readFields(in);
|
|
|
|
+ LongWritable offset = new LongWritable();
|
|
|
|
+ offset.readFields(in);
|
|
|
|
+ Pair<Long, Long> pair = Pair.of(lastProcessedTime.get(), offset.get());
|
|
|
|
+ logFiles.put(attemptDirName + Path.SEPARATOR + fileName, pair);
|
|
|
|
+ }
|
|
|
|
+ LOG.info("Recovered {} summarylog files", totalEntries);
|
|
|
|
+ return logFiles;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Stores set of SummaryLog files
|
|
|
|
+ void storeLogFiles(Collection<AppLogs> appLogs,
|
|
|
|
+ DataOutputStream checkPointStream) throws IOException {
|
|
|
|
+ long totalEntries = 0L;
|
|
|
|
+ for (AppLogs appLog : appLogs) {
|
|
|
|
+ totalEntries += appLog.summaryLogs.size();
|
|
|
|
+ }
|
|
|
|
+ checkPointStream.writeLong(totalEntries);
|
|
|
|
+ for (AppLogs appLog : appLogs) {
|
|
|
|
+ for (LogInfo summaryLog : appLog.summaryLogs) {
|
|
|
|
+ new Text(summaryLog.getAttemptDirName()).write(checkPointStream);
|
|
|
|
+ new Text(summaryLog.getFilename()).write(checkPointStream);
|
|
|
|
+ new LongWritable(summaryLog.getLastProcessedTime()).write(checkPointStream);
|
|
|
|
+ new LongWritable(summaryLog.getOffset()).write(checkPointStream);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ LOG.info("Stored {} summarylog files into checkPointFile", totalEntries);
|
|
|
|
+ }
|
|
|
|
+
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
int scanActiveLogs() throws IOException {
|
|
int scanActiveLogs() throws IOException {
|
|
long startTime = Time.monotonicNow();
|
|
long startTime = Time.monotonicNow();
|
|
|
|
+ // Store the Last Processed Time and Offset
|
|
|
|
+ if (recoveryEnabled && appIdLogMap.size() > 0) {
|
|
|
|
+
|
|
|
|
+ try (FSDataOutputStream checkPointStream = fs.create(checkpointFile, true)) {
|
|
|
|
+
|
|
|
|
+ storeLogFiles(appIdLogMap.values(), checkPointStream);
|
|
|
|
+
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.warn("Failed to checkpoint the summarylog files", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
int logsToScanCount = scanActiveLogs(activeRootPath);
|
|
int logsToScanCount = scanActiveLogs(activeRootPath);
|
|
metrics.addActiveLogDirScanTime(Time.monotonicNow() - startTime);
|
|
metrics.addActiveLogDirScanTime(Time.monotonicNow() - startTime);
|
|
return logsToScanCount;
|
|
return logsToScanCount;
|
|
@@ -824,6 +903,15 @@ public class EntityGroupFSTimelineStore extends CompositeService
|
|
log = new EntityLogInfo(attemptDirName, filename, owner);
|
|
log = new EntityLogInfo(attemptDirName, filename, owner);
|
|
summaryLogs.add(log);
|
|
summaryLogs.add(log);
|
|
}
|
|
}
|
|
|
|
+ // This is to avoid processing summary files again during Restart of ATS
|
|
|
|
+ if (recoveryEnabled) {
|
|
|
|
+ Pair<Long, Long> pair = recoveredLogs.remove(log.getAttemptDirName()
|
|
|
|
+ + Path.SEPARATOR + log.getFilename());
|
|
|
|
+ if (pair != null) {
|
|
|
|
+ log.setLastProcessedTime(pair.getKey());
|
|
|
|
+ log.setOffset(pair.getValue());
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
private synchronized void addDetailLog(String attemptDirName,
|
|
private synchronized void addDetailLog(String attemptDirName,
|