|
@@ -18,6 +18,10 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
|
|
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
+import java.io.ByteArrayOutputStream;
|
|
|
+import java.io.DataInputStream;
|
|
|
+import java.io.DataOutputStream;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
|
|
@@ -33,11 +37,13 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
|
|
|
+import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
@@ -57,11 +63,19 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
public static final Log LOG = LogFactory.getLog(FileSystemRMStateStore.class);
|
|
|
|
|
|
private static final String ROOT_DIR_NAME = "FSRMStateRoot";
|
|
|
-
|
|
|
+ private static final String RM_DT_SECRET_MANAGER_ROOT = "RMDTSecretManagerRoot";
|
|
|
+ private static final String RM_APP_ROOT = "RMAppRoot";
|
|
|
+ private static final String DELEGATION_KEY_PREFIX = "DelegationKey_";
|
|
|
+ private static final String DELEGATION_TOKEN_PREFIX = "RMDelegationToken_";
|
|
|
+ private static final String DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX =
|
|
|
+ "RMDTSequenceNumber_";
|
|
|
|
|
|
private FileSystem fs;
|
|
|
|
|
|
- private Path fsRootDirPath;
|
|
|
+ private Path rootDirPath;
|
|
|
+ private Path rmDTSecretManagerRoot;
|
|
|
+ private Path rmAppRoot;
|
|
|
+ private Path dtSequenceNumberPath = null;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
Path fsWorkingPath;
|
|
@@ -70,11 +84,14 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
throws Exception{
|
|
|
|
|
|
fsWorkingPath = new Path(conf.get(YarnConfiguration.FS_RM_STATE_STORE_URI));
|
|
|
- fsRootDirPath = new Path(fsWorkingPath, ROOT_DIR_NAME);
|
|
|
+ rootDirPath = new Path(fsWorkingPath, ROOT_DIR_NAME);
|
|
|
+ rmDTSecretManagerRoot = new Path(rootDirPath, RM_DT_SECRET_MANAGER_ROOT);
|
|
|
+ rmAppRoot = new Path(rootDirPath, RM_APP_ROOT);
|
|
|
|
|
|
// create filesystem
|
|
|
fs = fsWorkingPath.getFileSystem(conf);
|
|
|
- fs.mkdirs(fsRootDirPath);
|
|
|
+ fs.mkdirs(rmDTSecretManagerRoot);
|
|
|
+ fs.mkdirs(rmAppRoot);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -84,15 +101,23 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
|
|
|
@Override
|
|
|
public synchronized RMState loadState() throws Exception {
|
|
|
+ RMState rmState = new RMState();
|
|
|
+ // recover DelegationTokenSecretManager
|
|
|
+ loadRMDTSecretManagerState(rmState);
|
|
|
+ // recover RM applications
|
|
|
+ loadRMAppState(rmState);
|
|
|
+ return rmState;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void loadRMAppState(RMState rmState) throws Exception {
|
|
|
try {
|
|
|
- RMState state = new RMState();
|
|
|
- FileStatus[] childNodes = fs.listStatus(fsRootDirPath);
|
|
|
+ FileStatus[] childNodes = fs.listStatus(rmAppRoot);
|
|
|
List<ApplicationAttemptState> attempts =
|
|
|
new ArrayList<ApplicationAttemptState>();
|
|
|
for(FileStatus childNodeStatus : childNodes) {
|
|
|
assert childNodeStatus.isFile();
|
|
|
String childNodeName = childNodeStatus.getPath().getName();
|
|
|
- Path childNodePath = getNodePath(childNodeName);
|
|
|
+ Path childNodePath = getNodePath(rmAppRoot, childNodeName);
|
|
|
byte[] childData = readFile(childNodePath, childNodeStatus.getLen());
|
|
|
if(childNodeName.startsWith(ApplicationId.appIdStrPrefix)){
|
|
|
// application
|
|
@@ -107,7 +132,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
appStateData.getUser());
|
|
|
// assert child node name is same as actual applicationId
|
|
|
assert appId.equals(appState.context.getApplicationId());
|
|
|
- state.appState.put(appId, appState);
|
|
|
+ rmState.appState.put(appId, appState);
|
|
|
} else if(childNodeName.startsWith(
|
|
|
ApplicationAttemptId.appAttemptIdStrPrefix)) {
|
|
|
// attempt
|
|
@@ -139,7 +164,7 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
// go through all attempts and add them to their apps
|
|
|
for(ApplicationAttemptState attemptState : attempts) {
|
|
|
ApplicationId appId = attemptState.getAttemptId().getApplicationId();
|
|
|
- ApplicationState appState = state.appState.get(appId);
|
|
|
+ ApplicationState appState = rmState.appState.get(appId);
|
|
|
if(appState != null) {
|
|
|
appState.attempts.put(attemptState.getAttemptId(), attemptState);
|
|
|
} else {
|
|
@@ -148,22 +173,49 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
// application attempt nodes
|
|
|
LOG.info("Application node not found for attempt: "
|
|
|
+ attemptState.getAttemptId());
|
|
|
- deleteFile(getNodePath(attemptState.getAttemptId().toString()));
|
|
|
+ deleteFile(getNodePath(rmAppRoot, attemptState.getAttemptId().toString()));
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return state;
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Failed to load state.", e);
|
|
|
throw e;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void loadRMDTSecretManagerState(RMState rmState) throws Exception {
|
|
|
+ FileStatus[] childNodes = fs.listStatus(rmDTSecretManagerRoot);
|
|
|
+
|
|
|
+ for(FileStatus childNodeStatus : childNodes) {
|
|
|
+ assert childNodeStatus.isFile();
|
|
|
+ String childNodeName = childNodeStatus.getPath().getName();
|
|
|
+ Path childNodePath = getNodePath(rmDTSecretManagerRoot, childNodeName);
|
|
|
+ byte[] childData = readFile(childNodePath, childNodeStatus.getLen());
|
|
|
+ ByteArrayInputStream is = new ByteArrayInputStream(childData);
|
|
|
+ DataInputStream fsIn = new DataInputStream(is);
|
|
|
+ if(childNodeName.startsWith(DELEGATION_KEY_PREFIX)){
|
|
|
+ DelegationKey key = new DelegationKey();
|
|
|
+ key.readFields(fsIn);
|
|
|
+ rmState.rmSecretManagerState.masterKeyState.add(key);
|
|
|
+ } else if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
|
|
|
+ RMDelegationTokenIdentifier identifier = new RMDelegationTokenIdentifier();
|
|
|
+ identifier.readFields(fsIn);
|
|
|
+ long renewDate = fsIn.readLong();
|
|
|
+ rmState.rmSecretManagerState.delegationTokenState.put(identifier,
|
|
|
+ renewDate);
|
|
|
+ } else if(childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
|
|
+ rmState.rmSecretManagerState.dtSequenceNumber =
|
|
|
+ Integer.parseInt(childNodeName.split("_")[1]);
|
|
|
+ }else {
|
|
|
+ LOG.warn("Unknown file for recovering RMDelegationTokenSecretManager");
|
|
|
+ }
|
|
|
+ fsIn.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public synchronized void storeApplicationState(String appId,
|
|
|
- ApplicationStateDataPBImpl appStateDataPB)
|
|
|
- throws Exception {
|
|
|
- Path nodeCreatePath = getNodePath(appId);
|
|
|
+ ApplicationStateDataPBImpl appStateDataPB) throws Exception {
|
|
|
+ Path nodeCreatePath = getNodePath(rmAppRoot, appId);
|
|
|
|
|
|
LOG.info("Storing info for app: " + appId + " at: " + nodeCreatePath);
|
|
|
byte[] appStateData = appStateDataPB.getProto().toByteArray();
|
|
@@ -179,9 +231,8 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
|
|
|
@Override
|
|
|
public synchronized void storeApplicationAttemptState(String attemptId,
|
|
|
- ApplicationAttemptStateDataPBImpl attemptStateDataPB)
|
|
|
- throws Exception {
|
|
|
- Path nodeCreatePath = getNodePath(attemptId);
|
|
|
+ ApplicationAttemptStateDataPBImpl attemptStateDataPB) throws Exception {
|
|
|
+ Path nodeCreatePath = getNodePath(rmAppRoot, attemptId);
|
|
|
LOG.info("Storing info for attempt: " + attemptId
|
|
|
+ " at: " + nodeCreatePath);
|
|
|
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
|
|
@@ -197,9 +248,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
|
|
|
@Override
|
|
|
public synchronized void removeApplicationState(ApplicationState appState)
|
|
|
- throws Exception {
|
|
|
+ throws Exception {
|
|
|
String appId = appState.getAppId().toString();
|
|
|
- Path nodeRemovePath = getNodePath(appId);
|
|
|
+ Path nodeRemovePath = getNodePath(rmAppRoot, appId);
|
|
|
LOG.info("Removing info for app: " + appId + " at: " + nodeRemovePath);
|
|
|
deleteFile(nodeRemovePath);
|
|
|
for(ApplicationAttemptId attemptId : appState.attempts.keySet()) {
|
|
@@ -208,13 +259,76 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
public synchronized void removeApplicationAttemptState(String attemptId)
|
|
|
- throws Exception {
|
|
|
- Path nodeRemovePath = getNodePath(attemptId);
|
|
|
+ throws Exception {
|
|
|
+ Path nodeRemovePath = getNodePath(rmAppRoot, attemptId);
|
|
|
LOG.info("Removing info for attempt: " + attemptId
|
|
|
+ " at: " + nodeRemovePath);
|
|
|
deleteFile(nodeRemovePath);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public synchronized void storeRMDelegationTokenAndSequenceNumberState(
|
|
|
+ RMDelegationTokenIdentifier identifier, Long renewDate,
|
|
|
+ int latestSequenceNumber) throws Exception {
|
|
|
+ Path nodeCreatePath =
|
|
|
+ getNodePath(rmDTSecretManagerRoot,
|
|
|
+ DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
|
|
|
+ ByteArrayOutputStream os = new ByteArrayOutputStream();
|
|
|
+ DataOutputStream fsOut = new DataOutputStream(os);
|
|
|
+ LOG.info("Storing RMDelegationToken_" + identifier.getSequenceNumber());
|
|
|
+ identifier.write(fsOut);
|
|
|
+ fsOut.writeLong(renewDate);
|
|
|
+ writeFile(nodeCreatePath, os.toByteArray());
|
|
|
+ fsOut.close();
|
|
|
+
|
|
|
+ // store sequence number
|
|
|
+ Path latestSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX + latestSequenceNumber);
|
|
|
+ LOG.info("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX
|
|
|
+ + latestSequenceNumber);
|
|
|
+ if (dtSequenceNumberPath == null) {
|
|
|
+ if (!createFile(latestSequenceNumberPath)) {
|
|
|
+ throw new Exception("Failed to create " + latestSequenceNumberPath);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ if (!renameFile(dtSequenceNumberPath, latestSequenceNumberPath)) {
|
|
|
+ throw new Exception("Failed to rename " + dtSequenceNumberPath);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ dtSequenceNumberPath = latestSequenceNumberPath;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void removeRMDelegationTokenState(
|
|
|
+ RMDelegationTokenIdentifier identifier) throws Exception {
|
|
|
+ Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ DELEGATION_TOKEN_PREFIX + identifier.getSequenceNumber());
|
|
|
+ LOG.info("Removing RMDelegationToken_" + identifier.getSequenceNumber());
|
|
|
+ deleteFile(nodeCreatePath);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void storeRMDTMasterKeyState(DelegationKey masterKey)
|
|
|
+ throws Exception {
|
|
|
+ Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ DELEGATION_KEY_PREFIX + masterKey.getKeyId());
|
|
|
+ ByteArrayOutputStream os = new ByteArrayOutputStream();
|
|
|
+ DataOutputStream fsOut = new DataOutputStream(os);
|
|
|
+ LOG.info("Storing RMDelegationKey_" + masterKey.getKeyId());
|
|
|
+ masterKey.write(fsOut);
|
|
|
+ writeFile(nodeCreatePath, os.toByteArray());
|
|
|
+ fsOut.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public synchronized void
|
|
|
+ removeRMDTMasterKeyState(DelegationKey masterKey) throws Exception {
|
|
|
+ Path nodeCreatePath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ DELEGATION_KEY_PREFIX + masterKey.getKeyId());
|
|
|
+ LOG.info("Removing RMDelegationKey_"+ masterKey.getKeyId());
|
|
|
+ deleteFile(nodeCreatePath);
|
|
|
+ }
|
|
|
+
|
|
|
// FileSystem related code
|
|
|
|
|
|
private void deleteFile(Path deletePath) throws Exception {
|
|
@@ -228,18 +342,25 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|
|
// state data will not be that "long"
|
|
|
byte[] data = new byte[(int)len];
|
|
|
fsIn.readFully(data);
|
|
|
+ fsIn.close();
|
|
|
return data;
|
|
|
}
|
|
|
|
|
|
private void writeFile(Path outputPath, byte[] data) throws Exception {
|
|
|
FSDataOutputStream fsOut = fs.create(outputPath, false);
|
|
|
fsOut.write(data);
|
|
|
- fsOut.flush();
|
|
|
fsOut.close();
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- Path getNodePath(String nodeName) {
|
|
|
- return new Path(fsRootDirPath, nodeName);
|
|
|
+ private boolean renameFile(Path src, Path dst) throws Exception {
|
|
|
+ return fs.rename(src, dst);
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean createFile(Path newFile) throws Exception {
|
|
|
+ return fs.createNewFile(newFile);
|
|
|
+ }
|
|
|
+
|
|
|
+ private Path getNodePath(Path root, String nodeName) {
|
|
|
+ return new Path(root, nodeName);
|
|
|
}
|
|
|
}
|