|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.security;
|
|
package org.apache.hadoop.yarn.server.nodemanager.security;
|
|
|
|
|
|
|
|
+import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
@@ -31,6 +32,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
import org.apache.hadoop.yarn.server.api.records.MasterKey;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
|
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService.RecoveredNMTokenState;
|
|
import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
|
|
import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
|
|
import org.apache.hadoop.yarn.server.security.MasterKeyData;
|
|
import org.apache.hadoop.yarn.server.security.MasterKeyData;
|
|
|
|
|
|
@@ -45,16 +49,78 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
|
|
|
|
|
|
private final Map<ApplicationAttemptId, MasterKeyData> oldMasterKeys;
|
|
private final Map<ApplicationAttemptId, MasterKeyData> oldMasterKeys;
|
|
private final Map<ApplicationId, List<ApplicationAttemptId>> appToAppAttemptMap;
|
|
private final Map<ApplicationId, List<ApplicationAttemptId>> appToAppAttemptMap;
|
|
|
|
+ private final NMStateStoreService stateStore;
|
|
private NodeId nodeId;
|
|
private NodeId nodeId;
|
|
|
|
|
|
-
|
|
|
|
public NMTokenSecretManagerInNM() {
|
|
public NMTokenSecretManagerInNM() {
|
|
|
|
+ this(new NMNullStateStoreService());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public NMTokenSecretManagerInNM(NMStateStoreService stateStore) {
|
|
this.oldMasterKeys =
|
|
this.oldMasterKeys =
|
|
new HashMap<ApplicationAttemptId, MasterKeyData>();
|
|
new HashMap<ApplicationAttemptId, MasterKeyData>();
|
|
appToAppAttemptMap =
|
|
appToAppAttemptMap =
|
|
new HashMap<ApplicationId, List<ApplicationAttemptId>>();
|
|
new HashMap<ApplicationId, List<ApplicationAttemptId>>();
|
|
|
|
+ this.stateStore = stateStore;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public synchronized void recover(RecoveredNMTokenState state)
|
|
|
|
+ throws IOException {
|
|
|
|
+ MasterKey key = state.getCurrentMasterKey();
|
|
|
|
+ if (key != null) {
|
|
|
|
+ super.currentMasterKey =
|
|
|
|
+ new MasterKeyData(key, createSecretKey(key.getBytes().array()));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ key = state.getPreviousMasterKey();
|
|
|
|
+ if (key != null) {
|
|
|
|
+ previousMasterKey =
|
|
|
|
+ new MasterKeyData(key, createSecretKey(key.getBytes().array()));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // restore the serial number from the current master key
|
|
|
|
+ if (super.currentMasterKey != null) {
|
|
|
|
+ super.serialNo = super.currentMasterKey.getMasterKey().getKeyId() + 1;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ for (Map.Entry<ApplicationAttemptId, MasterKey> entry :
|
|
|
|
+ state.getApplicationMasterKeys().entrySet()) {
|
|
|
|
+ key = entry.getValue();
|
|
|
|
+ oldMasterKeys.put(entry.getKey(),
|
|
|
|
+ new MasterKeyData(key, createSecretKey(key.getBytes().array())));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // reconstruct app to app attempts map
|
|
|
|
+ appToAppAttemptMap.clear();
|
|
|
|
+ for (ApplicationAttemptId attempt : oldMasterKeys.keySet()) {
|
|
|
|
+ ApplicationId app = attempt.getApplicationId();
|
|
|
|
+ List<ApplicationAttemptId> attempts = appToAppAttemptMap.get(app);
|
|
|
|
+ if (attempts == null) {
|
|
|
|
+ attempts = new ArrayList<ApplicationAttemptId>();
|
|
|
|
+ appToAppAttemptMap.put(app, attempts);
|
|
|
|
+ }
|
|
|
|
+ attempts.add(attempt);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void updateCurrentMasterKey(MasterKeyData key) {
|
|
|
|
+ super.currentMasterKey = key;
|
|
|
|
+ try {
|
|
|
|
+ stateStore.storeNMTokenCurrentMasterKey(key.getMasterKey());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Unable to update current master key in state store", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void updatePreviousMasterKey(MasterKeyData key) {
|
|
|
|
+ previousMasterKey = key;
|
|
|
|
+ try {
|
|
|
|
+ stateStore.storeNMTokenPreviousMasterKey(key.getMasterKey());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Unable to update previous master key in state store", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Used by NodeManagers to create a token-secret-manager with the key
|
|
* Used by NodeManagers to create a token-secret-manager with the key
|
|
* obtained from the RM. This can happen during registration or when the RM
|
|
* obtained from the RM. This can happen during registration or when the RM
|
|
@@ -62,20 +128,16 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
|
|
*/
|
|
*/
|
|
@Private
|
|
@Private
|
|
public synchronized void setMasterKey(MasterKey masterKey) {
|
|
public synchronized void setMasterKey(MasterKey masterKey) {
|
|
- LOG.info("Rolling master-key for nm-tokens, got key with id :"
|
|
|
|
- + masterKey.getKeyId());
|
|
|
|
- if (super.currentMasterKey == null) {
|
|
|
|
- super.currentMasterKey =
|
|
|
|
- new MasterKeyData(masterKey, createSecretKey(masterKey.getBytes()
|
|
|
|
- .array()));
|
|
|
|
- } else {
|
|
|
|
- if (super.currentMasterKey.getMasterKey().getKeyId() != masterKey
|
|
|
|
- .getKeyId()) {
|
|
|
|
- this.previousMasterKey = super.currentMasterKey;
|
|
|
|
- super.currentMasterKey =
|
|
|
|
- new MasterKeyData(masterKey, createSecretKey(masterKey.getBytes()
|
|
|
|
- .array()));
|
|
|
|
|
|
+ // Update keys only if the key has changed.
|
|
|
|
+ if (super.currentMasterKey == null || super.currentMasterKey.getMasterKey()
|
|
|
|
+ .getKeyId() != masterKey.getKeyId()) {
|
|
|
|
+ LOG.info("Rolling master-key for container-tokens, got key with id "
|
|
|
|
+ + masterKey.getKeyId());
|
|
|
|
+ if (super.currentMasterKey != null) {
|
|
|
|
+ updatePreviousMasterKey(super.currentMasterKey);
|
|
}
|
|
}
|
|
|
|
+ updateCurrentMasterKey(new MasterKeyData(masterKey,
|
|
|
|
+ createSecretKey(masterKey.getBytes().array())));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -128,7 +190,7 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
|
|
LOG.debug("Removing application attempts NMToken keys for application "
|
|
LOG.debug("Removing application attempts NMToken keys for application "
|
|
+ appId);
|
|
+ appId);
|
|
for (ApplicationAttemptId appAttemptId : appAttemptList) {
|
|
for (ApplicationAttemptId appAttemptId : appAttemptList) {
|
|
- this.oldMasterKeys.remove(appAttemptId);
|
|
|
|
|
|
+ removeAppAttemptKey(appAttemptId);
|
|
}
|
|
}
|
|
appToAppAttemptMap.remove(appId);
|
|
appToAppAttemptMap.remove(appId);
|
|
} else {
|
|
} else {
|
|
@@ -164,11 +226,11 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
|
|
+ identifier.getApplicationAttemptId().toString());
|
|
+ identifier.getApplicationAttemptId().toString());
|
|
if (identifier.getKeyId() == currentMasterKey.getMasterKey()
|
|
if (identifier.getKeyId() == currentMasterKey.getMasterKey()
|
|
.getKeyId()) {
|
|
.getKeyId()) {
|
|
- oldMasterKeys.put(appAttemptId, currentMasterKey);
|
|
|
|
|
|
+ updateAppAttemptKey(appAttemptId, currentMasterKey);
|
|
} else if (previousMasterKey != null
|
|
} else if (previousMasterKey != null
|
|
&& identifier.getKeyId() == previousMasterKey.getMasterKey()
|
|
&& identifier.getKeyId() == previousMasterKey.getMasterKey()
|
|
.getKeyId()) {
|
|
.getKeyId()) {
|
|
- oldMasterKeys.put(appAttemptId, previousMasterKey);
|
|
|
|
|
|
+ updateAppAttemptKey(appAttemptId, previousMasterKey);
|
|
} else {
|
|
} else {
|
|
throw new InvalidToken(
|
|
throw new InvalidToken(
|
|
"Older NMToken should not be used while starting the container.");
|
|
"Older NMToken should not be used while starting the container.");
|
|
@@ -193,4 +255,24 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
|
|
public synchronized NodeId getNodeId() {
|
|
public synchronized NodeId getNodeId() {
|
|
return this.nodeId;
|
|
return this.nodeId;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private void updateAppAttemptKey(ApplicationAttemptId attempt,
|
|
|
|
+ MasterKeyData key) {
|
|
|
|
+ this.oldMasterKeys.put(attempt, key);
|
|
|
|
+ try {
|
|
|
|
+ stateStore.storeNMTokenApplicationMasterKey(attempt,
|
|
|
|
+ key.getMasterKey());
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Unable to store master key for application " + attempt, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void removeAppAttemptKey(ApplicationAttemptId attempt) {
|
|
|
|
+ this.oldMasterKeys.remove(attempt);
|
|
|
|
+ try {
|
|
|
|
+ stateStore.removeNMTokenApplicationMasterKey(attempt);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ LOG.error("Unable to remove master key for application " + attempt, e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|