|
@@ -33,6 +33,7 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
|
|
|
|
import javax.crypto.SecretKey;
|
|
|
|
|
@@ -122,10 +123,10 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*/
|
|
|
private DelegationKey currentKey;
|
|
|
|
|
|
- private long keyUpdateInterval;
|
|
|
- private long tokenMaxLifetime;
|
|
|
- private long tokenRemoverScanInterval;
|
|
|
- private long tokenRenewInterval;
|
|
|
+ private final long keyUpdateInterval;
|
|
|
+ private final long tokenMaxLifetime;
|
|
|
+ private final long tokenRemoverScanInterval;
|
|
|
+ private final long tokenRenewInterval;
|
|
|
/**
|
|
|
* Whether to store a token's tracking ID in its TokenInformation.
|
|
|
* Can be overridden by a subclass.
|
|
@@ -140,6 +141,8 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*/
|
|
|
protected Object noInterruptsLock = new Object();
|
|
|
|
|
|
+ private final ReentrantReadWriteLock apiLock = new ReentrantReadWriteLock(true);
|
|
|
+
|
|
|
/**
|
|
|
* Create a secret manager
|
|
|
* @param delegationKeyUpdateInterval the number of milliseconds for rolling
|
|
@@ -169,21 +172,29 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
public void startThreads() throws IOException {
|
|
|
Preconditions.checkState(!running);
|
|
|
updateCurrentKey();
|
|
|
- synchronized (this) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
running = true;
|
|
|
tokenRemoverThread = new Daemon(new ExpiredTokenRemover());
|
|
|
tokenRemoverThread.start();
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Reset all data structures and mutable state.
|
|
|
*/
|
|
|
- public synchronized void reset() {
|
|
|
- setCurrentKeyId(0);
|
|
|
- allKeys.clear();
|
|
|
- setDelegationTokenSeqNum(0);
|
|
|
- currentTokens.clear();
|
|
|
+ public void reset() {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ setCurrentKeyId(0);
|
|
|
+ allKeys.clear();
|
|
|
+ setDelegationTokenSeqNum(0);
|
|
|
+ currentTokens.clear();
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -210,17 +221,27 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @param key delegation key.
|
|
|
* @throws IOException raised on errors performing I/O.
|
|
|
*/
|
|
|
- public synchronized void addKey(DelegationKey key) throws IOException {
|
|
|
+ public void addKey(DelegationKey key) throws IOException {
|
|
|
if (running) // a safety check
|
|
|
throw new IOException("Can't add delegation key to a running SecretManager.");
|
|
|
- if (key.getKeyId() > getCurrentKeyId()) {
|
|
|
- setCurrentKeyId(key.getKeyId());
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ if (key.getKeyId() > getCurrentKeyId()) {
|
|
|
+ setCurrentKeyId(key.getKeyId());
|
|
|
+ }
|
|
|
+ allKeys.put(key.getKeyId(), key);
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
- allKeys.put(key.getKeyId(), key);
|
|
|
}
|
|
|
|
|
|
- public synchronized DelegationKey[] getAllKeys() {
|
|
|
- return allKeys.values().toArray(new DelegationKey[0]);
|
|
|
+ public DelegationKey[] getAllKeys() {
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ return allKeys.values().toArray(new DelegationKey[0]);
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// HDFS
|
|
@@ -263,8 +284,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @return currentId.
|
|
|
*/
|
|
|
- protected synchronized int getCurrentKeyId() {
|
|
|
- return currentId;
|
|
|
+ protected int getCurrentKeyId() {
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ return currentId;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -273,8 +299,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @return currentId.
|
|
|
*/
|
|
|
- protected synchronized int incrementCurrentKeyId() {
|
|
|
- return ++currentId;
|
|
|
+ protected int incrementCurrentKeyId() {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ return ++currentId;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -283,8 +314,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @param keyId keyId.
|
|
|
*/
|
|
|
- protected synchronized void setCurrentKeyId(int keyId) {
|
|
|
- currentId = keyId;
|
|
|
+ protected void setCurrentKeyId(int keyId) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ currentId = keyId;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -293,8 +329,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @return delegationTokenSequenceNumber.
|
|
|
*/
|
|
|
- protected synchronized int getDelegationTokenSeqNum() {
|
|
|
- return delegationTokenSequenceNumber;
|
|
|
+ protected int getDelegationTokenSeqNum() {
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ return delegationTokenSequenceNumber;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -303,8 +344,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @return delegationTokenSequenceNumber.
|
|
|
*/
|
|
|
- protected synchronized int incrementDelegationTokenSeqNum() {
|
|
|
- return ++delegationTokenSequenceNumber;
|
|
|
+ protected int incrementDelegationTokenSeqNum() {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ return ++delegationTokenSequenceNumber;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -313,8 +359,13 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*
|
|
|
* @param seqNum seqNum.
|
|
|
*/
|
|
|
- protected synchronized void setDelegationTokenSeqNum(int seqNum) {
|
|
|
- delegationTokenSequenceNumber = seqNum;
|
|
|
+ protected void setDelegationTokenSeqNum(int seqNum) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ delegationTokenSequenceNumber = seqNum;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -401,34 +452,39 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @param renewDate token renew time
|
|
|
* @throws IOException raised on errors performing I/O.
|
|
|
*/
|
|
|
- public synchronized void addPersistedDelegationToken(
|
|
|
+ public void addPersistedDelegationToken(
|
|
|
TokenIdent identifier, long renewDate) throws IOException {
|
|
|
if (running) {
|
|
|
// a safety check
|
|
|
throw new IOException(
|
|
|
"Can't add persisted delegation token to a running SecretManager.");
|
|
|
}
|
|
|
- int keyId = identifier.getMasterKeyId();
|
|
|
- DelegationKey dKey = allKeys.get(keyId);
|
|
|
- byte[] password = null;
|
|
|
- if (dKey == null) {
|
|
|
- LOG.warn("No KEY found for persisted identifier, expiring stored token "
|
|
|
- + formatTokenId(identifier));
|
|
|
- // make sure the token is expired
|
|
|
- renewDate = 0L;
|
|
|
- } else {
|
|
|
- password = createPassword(identifier.getBytes(), dKey.getKey());
|
|
|
- }
|
|
|
- if (identifier.getSequenceNumber() > getDelegationTokenSeqNum()) {
|
|
|
- setDelegationTokenSeqNum(identifier.getSequenceNumber());
|
|
|
- }
|
|
|
- if (getTokenInfo(identifier) == null) {
|
|
|
- currentTokens.put(identifier, new DelegationTokenInformation(renewDate,
|
|
|
- password, getTrackingIdIfEnabled(identifier)));
|
|
|
- addTokenForOwnerStats(identifier);
|
|
|
- } else {
|
|
|
- throw new IOException("Same delegation token being added twice: "
|
|
|
- + formatTokenId(identifier));
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ int keyId = identifier.getMasterKeyId();
|
|
|
+ DelegationKey dKey = allKeys.get(keyId);
|
|
|
+ byte[] password = null;
|
|
|
+ if (dKey == null) {
|
|
|
+ LOG.warn("No KEY found for persisted identifier, expiring stored token " + formatTokenId(
|
|
|
+ identifier));
|
|
|
+ // make sure the token is expired
|
|
|
+ renewDate = 0L;
|
|
|
+ } else {
|
|
|
+ password = createPassword(identifier.getBytes(), dKey.getKey());
|
|
|
+ }
|
|
|
+ if (identifier.getSequenceNumber() > getDelegationTokenSeqNum()) {
|
|
|
+ setDelegationTokenSeqNum(identifier.getSequenceNumber());
|
|
|
+ }
|
|
|
+ if (getTokenInfo(identifier) == null) {
|
|
|
+ currentTokens.put(identifier, new DelegationTokenInformation(renewDate, password,
|
|
|
+ getTrackingIdIfEnabled(identifier)));
|
|
|
+ addTokenForOwnerStats(identifier);
|
|
|
+ } else {
|
|
|
+ throw new IOException("Same delegation token being added twice: " +
|
|
|
+ formatTokenId(identifier));
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -441,17 +497,18 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
LOG.info("Updating the current master key for generating delegation tokens");
|
|
|
/* Create a new currentKey with an estimated expiry date. */
|
|
|
int newCurrentId;
|
|
|
- synchronized (this) {
|
|
|
- newCurrentId = incrementCurrentKeyId();
|
|
|
- }
|
|
|
+ newCurrentId = incrementCurrentKeyId();
|
|
|
DelegationKey newKey = new DelegationKey(newCurrentId, System
|
|
|
.currentTimeMillis()
|
|
|
+ keyUpdateInterval + tokenMaxLifetime, generateSecret());
|
|
|
//Log must be invoked outside the lock on 'this'
|
|
|
logUpdateMasterKey(newKey);
|
|
|
- synchronized (this) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
currentKey = newKey;
|
|
|
storeDelegationKey(currentKey);
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -461,7 +518,8 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @throws IOException raised on errors performing I/O.
|
|
|
*/
|
|
|
protected void rollMasterKey() throws IOException {
|
|
|
- synchronized (this) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
removeExpiredKeys();
|
|
|
/* set final expiry date for retiring currentKey */
|
|
|
currentKey.setExpiryDate(Time.now() + tokenMaxLifetime);
|
|
@@ -471,46 +529,59 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* allKeys just in case.
|
|
|
*/
|
|
|
updateDelegationKey(currentKey);
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
updateCurrentKey();
|
|
|
}
|
|
|
|
|
|
- private synchronized void removeExpiredKeys() {
|
|
|
- long now = Time.now();
|
|
|
- for (Iterator<Map.Entry<Integer, DelegationKey>> it = allKeys.entrySet()
|
|
|
- .iterator(); it.hasNext();) {
|
|
|
- Map.Entry<Integer, DelegationKey> e = it.next();
|
|
|
- if (e.getValue().getExpiryDate() < now) {
|
|
|
- it.remove();
|
|
|
- // ensure the tokens generated by this current key can be recovered
|
|
|
- // with this current key after this current key is rolled
|
|
|
- if(!e.getValue().equals(currentKey))
|
|
|
- removeStoredMasterKey(e.getValue());
|
|
|
+ private void removeExpiredKeys() {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ long now = Time.now();
|
|
|
+ for (Iterator<Map.Entry<Integer, DelegationKey>> it =
|
|
|
+ allKeys.entrySet().iterator(); it.hasNext();) {
|
|
|
+ Map.Entry<Integer, DelegationKey> e = it.next();
|
|
|
+ if (e.getValue().getExpiryDate() < now) {
|
|
|
+ it.remove();
|
|
|
+ // ensure the tokens generated by this current key can be recovered
|
|
|
+ // with this current key after this current key is rolled
|
|
|
+ if (!e.getValue().equals(currentKey)) {
|
|
|
+ removeStoredMasterKey(e.getValue());
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected synchronized byte[] createPassword(TokenIdent identifier) {
|
|
|
- int sequenceNum;
|
|
|
- long now = Time.now();
|
|
|
- sequenceNum = incrementDelegationTokenSeqNum();
|
|
|
- identifier.setIssueDate(now);
|
|
|
- identifier.setMaxDate(now + tokenMaxLifetime);
|
|
|
- identifier.setMasterKeyId(currentKey.getKeyId());
|
|
|
- identifier.setSequenceNumber(sequenceNum);
|
|
|
- LOG.info("Creating password for identifier: " + formatTokenId(identifier)
|
|
|
- + ", currentKey: " + currentKey.getKeyId());
|
|
|
- byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
|
|
|
- DelegationTokenInformation tokenInfo = new DelegationTokenInformation(now
|
|
|
- + tokenRenewInterval, password, getTrackingIdIfEnabled(identifier));
|
|
|
+ protected byte[] createPassword(TokenIdent identifier) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
try {
|
|
|
- METRICS.trackStoreToken(() -> storeToken(identifier, tokenInfo));
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.error("Could not store token " + formatTokenId(identifier) + "!!",
|
|
|
- ioe);
|
|
|
+ int sequenceNum;
|
|
|
+ long now = Time.now();
|
|
|
+ sequenceNum = incrementDelegationTokenSeqNum();
|
|
|
+ identifier.setIssueDate(now);
|
|
|
+ identifier.setMaxDate(now + tokenMaxLifetime);
|
|
|
+ identifier.setMasterKeyId(currentKey.getKeyId());
|
|
|
+ identifier.setSequenceNumber(sequenceNum);
|
|
|
+ LOG.info("Creating password for identifier: " + formatTokenId(identifier) +
|
|
|
+ ", currentKey: " + currentKey.getKeyId());
|
|
|
+ byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
|
|
|
+ DelegationTokenInformation tokenInfo =
|
|
|
+ new DelegationTokenInformation(now + tokenRenewInterval, password,
|
|
|
+ getTrackingIdIfEnabled(identifier));
|
|
|
+ try {
|
|
|
+ METRICS.trackStoreToken(() -> storeToken(identifier, tokenInfo));
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.error("Could not store token " + formatTokenId(identifier) + "!!", ioe);
|
|
|
+ }
|
|
|
+ return password;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
- return password;
|
|
|
}
|
|
|
|
|
|
|
|
@@ -526,7 +597,6 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
*/
|
|
|
protected DelegationTokenInformation checkToken(TokenIdent identifier)
|
|
|
throws InvalidToken {
|
|
|
- assert Thread.holdsLock(this);
|
|
|
DelegationTokenInformation info = getTokenInfo(identifier);
|
|
|
String err;
|
|
|
if (info == null) {
|
|
@@ -546,9 +616,14 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public synchronized byte[] retrievePassword(TokenIdent identifier)
|
|
|
+ public byte[] retrievePassword(TokenIdent identifier)
|
|
|
throws InvalidToken {
|
|
|
- return checkToken(identifier).getPassword();
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ return checkToken(identifier).getPassword();
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
protected String getTrackingIdIfEnabled(TokenIdent ident) {
|
|
@@ -558,12 +633,17 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- public synchronized String getTokenTrackingId(TokenIdent identifier) {
|
|
|
- DelegationTokenInformation info = getTokenInfo(identifier);
|
|
|
- if (info == null) {
|
|
|
- return null;
|
|
|
+ public String getTokenTrackingId(TokenIdent identifier) {
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ DelegationTokenInformation info = getTokenInfo(identifier);
|
|
|
+ if (info == null) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ return info.getTrackingId();
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
}
|
|
|
- return info.getTrackingId();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -572,12 +652,17 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @param password Password in the token.
|
|
|
* @throws InvalidToken InvalidToken.
|
|
|
*/
|
|
|
- public synchronized void verifyToken(TokenIdent identifier, byte[] password)
|
|
|
+ public void verifyToken(TokenIdent identifier, byte[] password)
|
|
|
throws InvalidToken {
|
|
|
- byte[] storedPassword = retrievePassword(identifier);
|
|
|
- if (!MessageDigest.isEqual(password, storedPassword)) {
|
|
|
- throw new InvalidToken("token " + formatTokenId(identifier)
|
|
|
- + " is invalid, password doesn't match");
|
|
|
+ this.apiLock.readLock().lock();
|
|
|
+ try {
|
|
|
+ byte[] storedPassword = retrievePassword(identifier);
|
|
|
+ if (!MessageDigest.isEqual(password, storedPassword)) {
|
|
|
+ throw new InvalidToken("token " + formatTokenId(identifier)
|
|
|
+ + " is invalid, password doesn't match");
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ this.apiLock.readLock().unlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -589,57 +674,55 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @throws InvalidToken if the token is invalid
|
|
|
* @throws AccessControlException if the user can't renew token
|
|
|
*/
|
|
|
- public synchronized long renewToken(Token<TokenIdent> token,
|
|
|
+ public long renewToken(Token<TokenIdent> token,
|
|
|
String renewer) throws InvalidToken, IOException {
|
|
|
- ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
|
|
|
- DataInputStream in = new DataInputStream(buf);
|
|
|
- TokenIdent id = createIdentifier();
|
|
|
- id.readFields(in);
|
|
|
- LOG.info("Token renewal for identifier: " + formatTokenId(id)
|
|
|
- + "; total currentTokens " + currentTokens.size());
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
|
|
|
+ DataInputStream in = new DataInputStream(buf);
|
|
|
+ TokenIdent id = createIdentifier();
|
|
|
+ id.readFields(in);
|
|
|
+ LOG.info("Token renewal for identifier: " + formatTokenId(id) + "; total currentTokens "
|
|
|
+ + currentTokens.size());
|
|
|
+
|
|
|
+ long now = Time.now();
|
|
|
+ if (id.getMaxDate() < now) {
|
|
|
+ throw new InvalidToken(renewer + " tried to renew an expired token " + formatTokenId(id) +
|
|
|
+ " max expiration date: " + Time.formatTime(id.getMaxDate()) + " currentTime: " +
|
|
|
+ Time.formatTime(now));
|
|
|
+ }
|
|
|
+ if ((id.getRenewer() == null) || (id.getRenewer().toString().isEmpty())) {
|
|
|
+ throw new AccessControlException(renewer + " tried to renew a token " + formatTokenId(id) +
|
|
|
+ " without a renewer");
|
|
|
+ }
|
|
|
+ if (!id.getRenewer().toString().equals(renewer)) {
|
|
|
+ throw new AccessControlException(renewer + " tries to renew a token " + formatTokenId(id) +
|
|
|
+ " with non-matching renewer " + id.getRenewer());
|
|
|
+ }
|
|
|
+ DelegationKey key = getDelegationKey(id.getMasterKeyId());
|
|
|
+ if (key == null) {
|
|
|
+ throw new InvalidToken("Unable to find master key for keyId=" + id.getMasterKeyId() +
|
|
|
+ " from cache. Failed to renew an unexpired token " + formatTokenId(id) +
|
|
|
+ " with sequenceNumber=" + id.getSequenceNumber());
|
|
|
+ }
|
|
|
+ byte[] password = createPassword(token.getIdentifier(), key.getKey());
|
|
|
+ if (!MessageDigest.isEqual(password, token.getPassword())) {
|
|
|
+ throw new AccessControlException(
|
|
|
+ renewer + " is trying to renew a token " + formatTokenId(id) + " with wrong password");
|
|
|
+ }
|
|
|
+ long renewTime = Math.min(id.getMaxDate(), now + tokenRenewInterval);
|
|
|
+ String trackingId = getTrackingIdIfEnabled(id);
|
|
|
+ DelegationTokenInformation info =
|
|
|
+ new DelegationTokenInformation(renewTime, password, trackingId);
|
|
|
|
|
|
- long now = Time.now();
|
|
|
- if (id.getMaxDate() < now) {
|
|
|
- throw new InvalidToken(renewer + " tried to renew an expired token "
|
|
|
- + formatTokenId(id) + " max expiration date: "
|
|
|
- + Time.formatTime(id.getMaxDate())
|
|
|
- + " currentTime: " + Time.formatTime(now));
|
|
|
- }
|
|
|
- if ((id.getRenewer() == null) || (id.getRenewer().toString().isEmpty())) {
|
|
|
- throw new AccessControlException(renewer +
|
|
|
- " tried to renew a token " + formatTokenId(id)
|
|
|
- + " without a renewer");
|
|
|
- }
|
|
|
- if (!id.getRenewer().toString().equals(renewer)) {
|
|
|
- throw new AccessControlException(renewer
|
|
|
- + " tries to renew a token " + formatTokenId(id)
|
|
|
- + " with non-matching renewer " + id.getRenewer());
|
|
|
- }
|
|
|
- DelegationKey key = getDelegationKey(id.getMasterKeyId());
|
|
|
- if (key == null) {
|
|
|
- throw new InvalidToken("Unable to find master key for keyId="
|
|
|
- + id.getMasterKeyId()
|
|
|
- + " from cache. Failed to renew an unexpired token "
|
|
|
- + formatTokenId(id) + " with sequenceNumber="
|
|
|
- + id.getSequenceNumber());
|
|
|
- }
|
|
|
- byte[] password = createPassword(token.getIdentifier(), key.getKey());
|
|
|
- if (!MessageDigest.isEqual(password, token.getPassword())) {
|
|
|
- throw new AccessControlException(renewer
|
|
|
- + " is trying to renew a token "
|
|
|
- + formatTokenId(id) + " with wrong password");
|
|
|
- }
|
|
|
- long renewTime = Math.min(id.getMaxDate(), now + tokenRenewInterval);
|
|
|
- String trackingId = getTrackingIdIfEnabled(id);
|
|
|
- DelegationTokenInformation info = new DelegationTokenInformation(renewTime,
|
|
|
- password, trackingId);
|
|
|
-
|
|
|
- if (getTokenInfo(id) == null) {
|
|
|
- throw new InvalidToken("Renewal request for unknown token "
|
|
|
- + formatTokenId(id));
|
|
|
+ if (getTokenInfo(id) == null) {
|
|
|
+ throw new InvalidToken("Renewal request for unknown token " + formatTokenId(id));
|
|
|
+ }
|
|
|
+ METRICS.trackUpdateToken(() -> updateToken(id, info));
|
|
|
+ return renewTime;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
- METRICS.trackUpdateToken(() -> updateToken(id, info));
|
|
|
- return renewTime;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -651,37 +734,41 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* @throws InvalidToken for invalid token
|
|
|
* @throws AccessControlException if the user isn't allowed to cancel
|
|
|
*/
|
|
|
- public synchronized TokenIdent cancelToken(Token<TokenIdent> token,
|
|
|
+ public TokenIdent cancelToken(Token<TokenIdent> token,
|
|
|
String canceller) throws IOException {
|
|
|
- ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
|
|
|
- DataInputStream in = new DataInputStream(buf);
|
|
|
- TokenIdent id = createIdentifier();
|
|
|
- id.readFields(in);
|
|
|
- LOG.info("Token cancellation requested for identifier: "
|
|
|
- + formatTokenId(id));
|
|
|
-
|
|
|
- if (id.getUser() == null) {
|
|
|
- throw new InvalidToken("Token with no owner " + formatTokenId(id));
|
|
|
- }
|
|
|
- String owner = id.getUser().getUserName();
|
|
|
- Text renewer = id.getRenewer();
|
|
|
- HadoopKerberosName cancelerKrbName = new HadoopKerberosName(canceller);
|
|
|
- String cancelerShortName = cancelerKrbName.getShortName();
|
|
|
- if (!canceller.equals(owner)
|
|
|
- && (renewer == null || renewer.toString().isEmpty() || !cancelerShortName
|
|
|
- .equals(renewer.toString()))) {
|
|
|
- throw new AccessControlException(canceller
|
|
|
- + " is not authorized to cancel the token " + formatTokenId(id));
|
|
|
- }
|
|
|
- DelegationTokenInformation info = currentTokens.remove(id);
|
|
|
- if (info == null) {
|
|
|
- throw new InvalidToken("Token not found " + formatTokenId(id));
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
+ ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
|
|
|
+ DataInputStream in = new DataInputStream(buf);
|
|
|
+ TokenIdent id = createIdentifier();
|
|
|
+ id.readFields(in);
|
|
|
+ LOG.info("Token cancellation requested for identifier: " + formatTokenId(id));
|
|
|
+
|
|
|
+ if (id.getUser() == null) {
|
|
|
+ throw new InvalidToken("Token with no owner " + formatTokenId(id));
|
|
|
+ }
|
|
|
+ String owner = id.getUser().getUserName();
|
|
|
+ Text renewer = id.getRenewer();
|
|
|
+ HadoopKerberosName cancelerKrbName = new HadoopKerberosName(canceller);
|
|
|
+ String cancelerShortName = cancelerKrbName.getShortName();
|
|
|
+ if (!canceller.equals(owner) &&
|
|
|
+ (renewer == null || renewer.toString().isEmpty() ||
|
|
|
+ !cancelerShortName.equals(renewer.toString()))) {
|
|
|
+ throw new AccessControlException(canceller + " is not authorized to cancel the token " +
|
|
|
+ formatTokenId(id));
|
|
|
+ }
|
|
|
+ DelegationTokenInformation info = currentTokens.remove(id);
|
|
|
+ if (info == null) {
|
|
|
+ throw new InvalidToken("Token not found " + formatTokenId(id));
|
|
|
+ }
|
|
|
+ METRICS.trackRemoveToken(() -> {
|
|
|
+ removeTokenForOwnerStats(id);
|
|
|
+ removeStoredToken(id);
|
|
|
+ });
|
|
|
+ return id;
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
- METRICS.trackRemoveToken(() -> {
|
|
|
- removeTokenForOwnerStats(id);
|
|
|
- removeStoredToken(id);
|
|
|
- });
|
|
|
- return id;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -762,7 +849,8 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
private void removeExpiredToken() throws IOException {
|
|
|
long now = Time.now();
|
|
|
Set<TokenIdent> expiredTokens = new HashSet<>();
|
|
|
- synchronized (this) {
|
|
|
+ this.apiLock.writeLock().lock();
|
|
|
+ try {
|
|
|
Iterator<Map.Entry<TokenIdent, DelegationTokenInformation>> i =
|
|
|
getCandidateTokensForCleanup().entrySet().iterator();
|
|
|
while (i.hasNext()) {
|
|
@@ -774,6 +862,8 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
i.remove();
|
|
|
}
|
|
|
}
|
|
|
+ } finally {
|
|
|
+ this.apiLock.writeLock().unlock();
|
|
|
}
|
|
|
// don't hold lock on 'this' to avoid edit log updates blocking token ops
|
|
|
logExpireTokens(expiredTokens);
|
|
@@ -818,10 +908,10 @@ extends AbstractDelegationTokenIdentifier>
|
|
|
* is secretMgr running
|
|
|
* @return true if secret mgr is running
|
|
|
*/
|
|
|
- public synchronized boolean isRunning() {
|
|
|
+ public boolean isRunning() {
|
|
|
return running;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private class ExpiredTokenRemover extends Thread {
|
|
|
private long lastMasterKeyUpdate;
|
|
|
private long lastTokenCacheCleanup;
|