|
@@ -78,16 +78,51 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
protected static final String ROOT_ZNODE_NAME = "ZKRMStateRoot";
|
|
|
protected static final RMStateVersion CURRENT_VERSION_INFO = RMStateVersion
|
|
|
.newInstance(1, 0);
|
|
|
+ private static final String RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME =
|
|
|
+ "RMDelegationTokensRoot";
|
|
|
+ private static final String RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME =
|
|
|
+ "RMDTSequentialNumber";
|
|
|
+ private static final String RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME =
|
|
|
+ "RMDTMasterKeysRoot";
|
|
|
private int numRetries;
|
|
|
|
|
|
private String zkHostPort = null;
|
|
|
private int zkSessionTimeout;
|
|
|
private long zkRetryInterval;
|
|
|
private List<ACL> zkAcl;
|
|
|
+
|
|
|
+ /**
|
|
|
+ *
|
|
|
+ * ROOT_DIR_PATH
|
|
|
+ * |--- VERSION_INFO
|
|
|
+ * |--- RM_ZK_FENCING_LOCK
|
|
|
+ * |--- RM_APP_ROOT
|
|
|
+ * | |----- (#ApplicationId1)
|
|
|
+ * | | |----- (#ApplicationAttemptIds)
|
|
|
+ * | |
|
|
|
+ * | |----- (#ApplicationId2)
|
|
|
+ * | | |----- (#ApplicationAttemptIds)
|
|
|
+ * | ....
|
|
|
+ * |
|
|
|
+ * |--- RM_DT_SECRET_MANAGER_ROOT
|
|
|
+ * |----- RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME
|
|
|
+ * |----- RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME
|
|
|
+ * | |----- Token_1
|
|
|
+ * | |----- Token_2
|
|
|
+ * | ....
|
|
|
+ * |
|
|
|
+ * |----- RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME
|
|
|
+ * | |----- Key_1
|
|
|
+ * | |----- Key_2
|
|
|
+ * ....
|
|
|
+ *
|
|
|
+ */
|
|
|
private String zkRootNodePath;
|
|
|
- private String rmDTSecretManagerRoot;
|
|
|
private String rmAppRoot;
|
|
|
- private String dtSequenceNumberPath = null;
|
|
|
+ private String rmDTSecretManagerRoot;
|
|
|
+ private String dtMasterKeysRootPath;
|
|
|
+ private String delegationTokensRootPath;
|
|
|
+ private String dtSequenceNumberPath;
|
|
|
|
|
|
@VisibleForTesting
|
|
|
protected String znodeWorkingPath;
|
|
@@ -178,12 +213,11 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
throw bafe;
|
|
|
}
|
|
|
|
|
|
- zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
|
|
|
- rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
|
|
|
- rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
|
|
|
+ zkRootNodePath = getNodePath(znodeWorkingPath, ROOT_ZNODE_NAME);
|
|
|
+ rmAppRoot = getNodePath(zkRootNodePath, RM_APP_ROOT);
|
|
|
|
|
|
/* Initialize fencing related paths, acls, and ops */
|
|
|
- fencingNodePath = zkRootNodePath + "/" + FENCING_LOCK;
|
|
|
+ fencingNodePath = getNodePath(zkRootNodePath, FENCING_LOCK);
|
|
|
createFencingNodePathOp = Op.create(fencingNodePath, new byte[0], zkAcl,
|
|
|
CreateMode.PERSISTENT);
|
|
|
deleteFencingNodePathOp = Op.delete(fencingNodePath, -1);
|
|
@@ -204,6 +238,15 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
zkRootNodeAcl = constructZkRootNodeACL(conf, zkAcl);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ rmDTSecretManagerRoot =
|
|
|
+ getNodePath(zkRootNodePath, RM_DT_SECRET_MANAGER_ROOT);
|
|
|
+ dtMasterKeysRootPath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ RM_DT_MASTER_KEYS_ROOT_ZNODE_NAME);
|
|
|
+ delegationTokensRootPath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ RM_DELEGATION_TOKENS_ROOT_ZNODE_NAME);
|
|
|
+ dtSequenceNumberPath = getNodePath(rmDTSecretManagerRoot,
|
|
|
+ RM_DT_SEQUENTIAL_NUMBER_ZNODE_NAME);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -217,8 +260,11 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
if (HAUtil.isHAEnabled(getConfig())){
|
|
|
fence();
|
|
|
}
|
|
|
- createRootDir(rmDTSecretManagerRoot);
|
|
|
createRootDir(rmAppRoot);
|
|
|
+ createRootDir(rmDTSecretManagerRoot);
|
|
|
+ createRootDir(dtMasterKeysRootPath);
|
|
|
+ createRootDir(delegationTokensRootPath);
|
|
|
+ createRootDir(dtSequenceNumberPath);
|
|
|
}
|
|
|
|
|
|
private void createRootDir(final String rootPath) throws Exception {
|
|
@@ -350,26 +396,69 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
|
|
|
private synchronized void loadRMDTSecretManagerState(RMState rmState)
|
|
|
throws Exception {
|
|
|
- List<String> childNodes =
|
|
|
- getChildrenWithRetries(rmDTSecretManagerRoot, true);
|
|
|
+ loadRMDelegationKeyState(rmState);
|
|
|
+ loadRMSequentialNumberState(rmState);
|
|
|
+ loadRMDelegationTokenState(rmState);
|
|
|
+ }
|
|
|
|
|
|
+ private void loadRMDelegationKeyState(RMState rmState) throws Exception {
|
|
|
+ List<String> childNodes =
|
|
|
+ getChildrenWithRetries(dtMasterKeysRootPath, true);
|
|
|
for (String childNodeName : childNodes) {
|
|
|
- if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
|
|
- rmState.rmSecretManagerState.dtSequenceNumber =
|
|
|
- Integer.parseInt(childNodeName.split("_")[1]);
|
|
|
+ String childNodePath = getNodePath(dtMasterKeysRootPath, childNodeName);
|
|
|
+ byte[] childData = getDataWithRetries(childNodePath, true);
|
|
|
+
|
|
|
+ if (childData == null) {
|
|
|
+ LOG.warn("Content of " + childNodePath + " is broken.");
|
|
|
continue;
|
|
|
}
|
|
|
- String childNodePath = getNodePath(rmDTSecretManagerRoot, childNodeName);
|
|
|
- byte[] childData = getDataWithRetries(childNodePath, true);
|
|
|
|
|
|
ByteArrayInputStream is = new ByteArrayInputStream(childData);
|
|
|
DataInputStream fsIn = new DataInputStream(is);
|
|
|
+
|
|
|
try {
|
|
|
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)) {
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ is.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void loadRMSequentialNumberState(RMState rmState) throws Exception {
|
|
|
+ byte[] seqData = getDataWithRetries(dtSequenceNumberPath, false);
|
|
|
+ if (seqData != null) {
|
|
|
+ ByteArrayInputStream seqIs = new ByteArrayInputStream(seqData);
|
|
|
+ DataInputStream seqIn = new DataInputStream(seqIs);
|
|
|
+
|
|
|
+ try {
|
|
|
+ rmState.rmSecretManagerState.dtSequenceNumber = seqIn.readInt();
|
|
|
+ } finally {
|
|
|
+ seqIn.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void loadRMDelegationTokenState(RMState rmState) throws Exception {
|
|
|
+ List<String> childNodes = zkClient.getChildren(delegationTokensRootPath, true);
|
|
|
+ for (String childNodeName : childNodes) {
|
|
|
+ String childNodePath =
|
|
|
+ getNodePath(delegationTokensRootPath, childNodeName);
|
|
|
+ byte[] childData = getDataWithRetries(childNodePath, true);
|
|
|
+
|
|
|
+ if (childData == null) {
|
|
|
+ LOG.warn("Content of " + childNodePath + " is broken.");
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ ByteArrayInputStream is = new ByteArrayInputStream(childData);
|
|
|
+ DataInputStream fsIn = new DataInputStream(is);
|
|
|
+
|
|
|
+ try {
|
|
|
+ if (childNodeName.startsWith(DELEGATION_TOKEN_PREFIX)) {
|
|
|
RMDelegationTokenIdentifier identifier =
|
|
|
new RMDelegationTokenIdentifier();
|
|
|
identifier.readFields(fsIn);
|
|
@@ -385,8 +474,6 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
|
|
|
private synchronized void loadRMAppState(RMState rmState) throws Exception {
|
|
|
List<String> childNodes = getChildrenWithRetries(rmAppRoot, true);
|
|
|
- List<ApplicationAttemptState> attempts =
|
|
|
- new ArrayList<ApplicationAttemptState>();
|
|
|
for (String childNodeName : childNodes) {
|
|
|
String childNodePath = getNodePath(rmAppRoot, childNodeName);
|
|
|
byte[] childData = getDataWithRetries(childNodePath, true);
|
|
@@ -411,17 +498,28 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
"from the application id");
|
|
|
}
|
|
|
rmState.appState.put(appId, appState);
|
|
|
- } else if (childNodeName
|
|
|
- .startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
|
|
|
- // attempt
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Loading application attempt from znode: " + childNodeName);
|
|
|
- }
|
|
|
+ loadApplicationAttemptState(appState, appId);
|
|
|
+ } else {
|
|
|
+ LOG.info("Unknown child node with name: " + childNodeName);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void loadApplicationAttemptState(ApplicationState appState,
|
|
|
+ ApplicationId appId)
|
|
|
+ throws Exception {
|
|
|
+ String appPath = getNodePath(rmAppRoot, appId.toString());
|
|
|
+ List<String> attempts = getChildrenWithRetries(appPath, false);
|
|
|
+ for (String attemptIDStr : attempts) {
|
|
|
+ if (attemptIDStr.startsWith(ApplicationAttemptId.appAttemptIdStrPrefix)) {
|
|
|
+ String attemptPath = getNodePath(appPath, attemptIDStr);
|
|
|
+ byte[] attemptData = getDataWithRetries(attemptPath, true);
|
|
|
+
|
|
|
ApplicationAttemptId attemptId =
|
|
|
- ConverterUtils.toApplicationAttemptId(childNodeName);
|
|
|
+ ConverterUtils.toApplicationAttemptId(attemptIDStr);
|
|
|
ApplicationAttemptStateDataPBImpl attemptStateData =
|
|
|
new ApplicationAttemptStateDataPBImpl(
|
|
|
- ApplicationAttemptStateDataProto.parseFrom(childData));
|
|
|
+ ApplicationAttemptStateDataProto.parseFrom(attemptData));
|
|
|
Credentials credentials = null;
|
|
|
if (attemptStateData.getAppAttemptTokens() != null) {
|
|
|
credentials = new Credentials();
|
|
@@ -429,47 +527,26 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
dibb.reset(attemptStateData.getAppAttemptTokens());
|
|
|
credentials.readTokenStorageStream(dibb);
|
|
|
}
|
|
|
+
|
|
|
ApplicationAttemptState attemptState =
|
|
|
new ApplicationAttemptState(attemptId,
|
|
|
- attemptStateData.getMasterContainer(), credentials,
|
|
|
- attemptStateData.getStartTime(),
|
|
|
- attemptStateData.getState(),
|
|
|
- attemptStateData.getFinalTrackingUrl(),
|
|
|
- attemptStateData.getDiagnostics(),
|
|
|
- attemptStateData.getFinalApplicationStatus());
|
|
|
- if (!attemptId.equals(attemptState.getAttemptId())) {
|
|
|
- throw new YarnRuntimeException("The child node name is different " +
|
|
|
- "from the application attempt id");
|
|
|
- }
|
|
|
- attempts.add(attemptState);
|
|
|
- } else {
|
|
|
- LOG.info("Unknown child node with name: " + childNodeName);
|
|
|
- }
|
|
|
- }
|
|
|
+ attemptStateData.getMasterContainer(), credentials,
|
|
|
+ attemptStateData.getStartTime(),
|
|
|
+ attemptStateData.getState(),
|
|
|
+ attemptStateData.getFinalTrackingUrl(),
|
|
|
+ attemptStateData.getDiagnostics(),
|
|
|
+ attemptStateData.getFinalApplicationStatus());
|
|
|
|
|
|
- // go through all attempts and add them to their apps
|
|
|
- for (ApplicationAttemptState attemptState : attempts) {
|
|
|
- ApplicationId appId = attemptState.getAttemptId().getApplicationId();
|
|
|
- ApplicationState appState = rmState.appState.get(appId);
|
|
|
- if (appState != null) {
|
|
|
appState.attempts.put(attemptState.getAttemptId(), attemptState);
|
|
|
- } else {
|
|
|
- // the application znode may have been removed when the application
|
|
|
- // completed but the RM might have stopped before it could remove the
|
|
|
- // application attempt znodes
|
|
|
- LOG.info("Application node not found for attempt: "
|
|
|
- + attemptState.getAttemptId());
|
|
|
- deleteWithRetries(
|
|
|
- getNodePath(rmAppRoot, attemptState.getAttemptId().toString()), -1);
|
|
|
}
|
|
|
}
|
|
|
LOG.info("Done Loading applications from ZK state store");
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void storeApplicationStateInternal(String appId,
|
|
|
+ public synchronized void storeApplicationStateInternal(ApplicationId appId,
|
|
|
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
|
|
|
- String nodeCreatePath = getNodePath(rmAppRoot, appId);
|
|
|
+ String nodeCreatePath = getNodePath(rmAppRoot, appId.toString());
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Storing info for app: " + appId + " at: " + nodeCreatePath);
|
|
@@ -481,25 +558,29 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized void updateApplicationStateInternal(String appId,
|
|
|
+ public synchronized void updateApplicationStateInternal(ApplicationId appId,
|
|
|
ApplicationStateDataPBImpl appStateDataPB) throws Exception {
|
|
|
- String nodeCreatePath = getNodePath(rmAppRoot, appId);
|
|
|
+ String nodeUpdatePath = getNodePath(rmAppRoot, appId.toString());
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Storing final state info for app: " + appId + " at: "
|
|
|
- + nodeCreatePath);
|
|
|
+ + nodeUpdatePath);
|
|
|
}
|
|
|
byte[] appStateData = appStateDataPB.getProto().toByteArray();
|
|
|
- setDataWithRetries(nodeCreatePath, appStateData, 0);
|
|
|
+ setDataWithRetries(nodeUpdatePath, appStateData, 0);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void storeApplicationAttemptStateInternal(
|
|
|
- String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
|
|
|
+ ApplicationAttemptId appAttemptId,
|
|
|
+ ApplicationAttemptStateDataPBImpl attemptStateDataPB)
|
|
|
throws Exception {
|
|
|
- String nodeCreatePath = getNodePath(rmAppRoot, attemptId);
|
|
|
+ String appDirPath = getNodePath(rmAppRoot,
|
|
|
+ appAttemptId.getApplicationId().toString());
|
|
|
+ String nodeCreatePath = getNodePath(appDirPath, appAttemptId.toString());
|
|
|
+
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Storing info for attempt: " + attemptId + " at: "
|
|
|
+ LOG.debug("Storing info for attempt: " + appAttemptId + " at: "
|
|
|
+ nodeCreatePath);
|
|
|
}
|
|
|
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
|
|
@@ -509,31 +590,36 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
|
|
|
@Override
|
|
|
public synchronized void updateApplicationAttemptStateInternal(
|
|
|
- String attemptId, ApplicationAttemptStateDataPBImpl attemptStateDataPB)
|
|
|
+ ApplicationAttemptId appAttemptId,
|
|
|
+ ApplicationAttemptStateDataPBImpl attemptStateDataPB)
|
|
|
throws Exception {
|
|
|
- String nodeCreatePath = getNodePath(rmAppRoot, attemptId);
|
|
|
+ String appIdStr = appAttemptId.getApplicationId().toString();
|
|
|
+ String appAttemptIdStr = appAttemptId.toString();
|
|
|
+ String appDirPath = getNodePath(rmAppRoot, appIdStr);
|
|
|
+ String nodeUpdatePath = getNodePath(appDirPath, appAttemptIdStr);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Storing final state info for attempt: " + attemptId + " at: "
|
|
|
- + nodeCreatePath);
|
|
|
+ LOG.debug("Storing final state info for attempt: " + appAttemptIdStr
|
|
|
+ + " at: " + nodeUpdatePath);
|
|
|
}
|
|
|
byte[] attemptStateData = attemptStateDataPB.getProto().toByteArray();
|
|
|
- setDataWithRetries(nodeCreatePath, attemptStateData, 0);
|
|
|
+ setDataWithRetries(nodeUpdatePath, attemptStateData, 0);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void removeApplicationStateInternal(ApplicationState appState)
|
|
|
throws Exception {
|
|
|
String appId = appState.getAppId().toString();
|
|
|
- String nodeRemovePath = getNodePath(rmAppRoot, appId);
|
|
|
+ String appIdRemovePath = getNodePath(rmAppRoot, appId);
|
|
|
ArrayList<Op> opList = new ArrayList<Op>();
|
|
|
- opList.add(Op.delete(nodeRemovePath, -1));
|
|
|
|
|
|
for (ApplicationAttemptId attemptId : appState.attempts.keySet()) {
|
|
|
- String attemptRemovePath = getNodePath(rmAppRoot, attemptId.toString());
|
|
|
+ String attemptRemovePath = getNodePath(appIdRemovePath, attemptId.toString());
|
|
|
opList.add(Op.delete(attemptRemovePath, -1));
|
|
|
}
|
|
|
+ opList.add(Op.delete(appIdRemovePath, -1));
|
|
|
+
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Removing info for app: " + appId + " at: " + nodeRemovePath
|
|
|
+ LOG.debug("Removing info for app: " + appId + " at: " + appIdRemovePath
|
|
|
+ " and its attempts.");
|
|
|
}
|
|
|
doMultiWithRetries(opList);
|
|
@@ -546,38 +632,37 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
ArrayList<Op> opList = new ArrayList<Op>();
|
|
|
// store RM delegation token
|
|
|
String nodeCreatePath =
|
|
|
- getNodePath(rmDTSecretManagerRoot, DELEGATION_TOKEN_PREFIX
|
|
|
+ getNodePath(delegationTokensRootPath, DELEGATION_TOKEN_PREFIX
|
|
|
+ rmDTIdentifier.getSequenceNumber());
|
|
|
- ByteArrayOutputStream os = new ByteArrayOutputStream();
|
|
|
- DataOutputStream fsOut = new DataOutputStream(os);
|
|
|
+ ByteArrayOutputStream tokenOs = new ByteArrayOutputStream();
|
|
|
+ DataOutputStream tokenOut = new DataOutputStream(tokenOs);
|
|
|
+ ByteArrayOutputStream seqOs = new ByteArrayOutputStream();
|
|
|
+ DataOutputStream seqOut = new DataOutputStream(seqOs);
|
|
|
+
|
|
|
try {
|
|
|
- rmDTIdentifier.write(fsOut);
|
|
|
- fsOut.writeLong(renewDate);
|
|
|
+ rmDTIdentifier.write(tokenOut);
|
|
|
+ tokenOut.writeLong(renewDate);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Storing RMDelegationToken_" +
|
|
|
rmDTIdentifier.getSequenceNumber());
|
|
|
}
|
|
|
- opList.add(Op.create(nodeCreatePath, os.toByteArray(), zkAcl,
|
|
|
+
|
|
|
+ opList.add(Op.create(nodeCreatePath, tokenOs.toByteArray(), zkAcl,
|
|
|
CreateMode.PERSISTENT));
|
|
|
- } finally {
|
|
|
- os.close();
|
|
|
- }
|
|
|
|
|
|
- // store sequence number
|
|
|
- String latestSequenceNumberPath =
|
|
|
- getNodePath(rmDTSecretManagerRoot,
|
|
|
- DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX + latestSequenceNumber);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Storing " + DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX +
|
|
|
- latestSequenceNumber);
|
|
|
- }
|
|
|
|
|
|
- if (dtSequenceNumberPath != null) {
|
|
|
- opList.add(Op.delete(dtSequenceNumberPath, -1));
|
|
|
+ seqOut.writeInt(latestSequenceNumber);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Storing " + dtSequenceNumberPath +
|
|
|
+ ". SequenceNumber: " + latestSequenceNumber);
|
|
|
+ }
|
|
|
+
|
|
|
+ opList.add(Op.setData(dtSequenceNumberPath, seqOs.toByteArray(), -1));
|
|
|
+ } finally {
|
|
|
+ tokenOs.close();
|
|
|
+ seqOs.close();
|
|
|
}
|
|
|
- opList.add(Op.create(latestSequenceNumberPath, null, zkAcl,
|
|
|
- CreateMode.PERSISTENT));
|
|
|
- dtSequenceNumberPath = latestSequenceNumberPath;
|
|
|
+
|
|
|
doMultiWithRetries(opList);
|
|
|
}
|
|
|
|
|
@@ -585,7 +670,7 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
protected synchronized void removeRMDelegationTokenState(
|
|
|
RMDelegationTokenIdentifier rmDTIdentifier) throws Exception {
|
|
|
String nodeRemovePath =
|
|
|
- getNodePath(rmDTSecretManagerRoot, DELEGATION_TOKEN_PREFIX
|
|
|
+ getNodePath(delegationTokensRootPath, DELEGATION_TOKEN_PREFIX
|
|
|
+ rmDTIdentifier.getSequenceNumber());
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Removing RMDelegationToken_"
|
|
@@ -598,7 +683,7 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
protected synchronized void storeRMDTMasterKeyState(
|
|
|
DelegationKey delegationKey) throws Exception {
|
|
|
String nodeCreatePath =
|
|
|
- getNodePath(rmDTSecretManagerRoot, DELEGATION_KEY_PREFIX
|
|
|
+ getNodePath(dtMasterKeysRootPath, DELEGATION_KEY_PREFIX
|
|
|
+ delegationKey.getKeyId());
|
|
|
ByteArrayOutputStream os = new ByteArrayOutputStream();
|
|
|
DataOutputStream fsOut = new DataOutputStream(os);
|
|
@@ -618,7 +703,7 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
protected synchronized void removeRMDTMasterKeyState(
|
|
|
DelegationKey delegationKey) throws Exception {
|
|
|
String nodeRemovePath =
|
|
|
- getNodePath(rmDTSecretManagerRoot, DELEGATION_KEY_PREFIX
|
|
|
+ getNodePath(dtMasterKeysRootPath, DELEGATION_KEY_PREFIX
|
|
|
+ delegationKey.getKeyId());
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Removing RMDelegationKey_" + delegationKey.getKeyId());
|
|
@@ -757,8 +842,7 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
return new ZKAction<byte[]>() {
|
|
|
@Override
|
|
|
public byte[] run() throws KeeperException, InterruptedException {
|
|
|
- Stat stat = new Stat();
|
|
|
- return zkClient.getData(path, watch, stat);
|
|
|
+ return zkClient.getData(path, watch, null);
|
|
|
}
|
|
|
}.runWithRetries();
|
|
|
}
|
|
@@ -865,4 +949,5 @@ public class ZKRMStateStore extends RMStateStore {
|
|
|
zk.register(new ForwardingWatcher());
|
|
|
return zk;
|
|
|
}
|
|
|
+
|
|
|
}
|