Skip to content

Commit e283375

Browse files
authored
HADOOP-18851: Performance improvement for DelegationTokenSecretManager. (#6001). Contributed by Vikas Kumar.
Signed-off-by: Wei-Chiu Chuang <[email protected]> Signed-off-by: He Xiaoqiao <[email protected]>
1 parent 23360b3 commit e283375

File tree

2 files changed

+38
-32
lines changed

2 files changed

+38
-32
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

Lines changed: 15 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -115,12 +115,12 @@ private String formatTokenId(TokenIdent id) {
115115
/**
116116
* Access to currentKey is protected by this object lock
117117
*/
118-
private DelegationKey currentKey;
118+
private volatile DelegationKey currentKey;
119119

120-
private long keyUpdateInterval;
121-
private long tokenMaxLifetime;
122-
private long tokenRemoverScanInterval;
123-
private long tokenRenewInterval;
120+
private final long keyUpdateInterval;
121+
private final long tokenMaxLifetime;
122+
private final long tokenRemoverScanInterval;
123+
private final long tokenRenewInterval;
124124
/**
125125
* Whether to store a token's tracking ID in its TokenInformation.
126126
* Can be overridden by a subclass.
@@ -486,17 +486,18 @@ private synchronized void removeExpiredKeys() {
486486
}
487487

488488
@Override
489-
protected synchronized byte[] createPassword(TokenIdent identifier) {
489+
protected byte[] createPassword(TokenIdent identifier) {
490490
int sequenceNum;
491491
long now = Time.now();
492492
sequenceNum = incrementDelegationTokenSeqNum();
493493
identifier.setIssueDate(now);
494494
identifier.setMaxDate(now + tokenMaxLifetime);
495-
identifier.setMasterKeyId(currentKey.getKeyId());
495+
DelegationKey delegationCurrentKey = currentKey;
496+
identifier.setMasterKeyId(delegationCurrentKey.getKeyId());
496497
identifier.setSequenceNumber(sequenceNum);
497498
LOG.info("Creating password for identifier: " + formatTokenId(identifier)
498-
+ ", currentKey: " + currentKey.getKeyId());
499-
byte[] password = createPassword(identifier.getBytes(), currentKey.getKey());
499+
+ ", currentKey: " + delegationCurrentKey.getKeyId());
500+
byte[] password = createPassword(identifier.getBytes(), delegationCurrentKey.getKey());
500501
DelegationTokenInformation tokenInfo = new DelegationTokenInformation(now
501502
+ tokenRenewInterval, password, getTrackingIdIfEnabled(identifier));
502503
try {
@@ -521,7 +522,6 @@ protected synchronized byte[] createPassword(TokenIdent identifier) {
521522
*/
522523
protected DelegationTokenInformation checkToken(TokenIdent identifier)
523524
throws InvalidToken {
524-
assert Thread.holdsLock(this);
525525
DelegationTokenInformation info = getTokenInfo(identifier);
526526
String err;
527527
if (info == null) {
@@ -541,7 +541,7 @@ protected DelegationTokenInformation checkToken(TokenIdent identifier)
541541
}
542542

543543
@Override
544-
public synchronized byte[] retrievePassword(TokenIdent identifier)
544+
public byte[] retrievePassword(TokenIdent identifier)
545545
throws InvalidToken {
546546
return checkToken(identifier).getPassword();
547547
}
@@ -553,7 +553,7 @@ protected String getTrackingIdIfEnabled(TokenIdent ident) {
553553
return null;
554554
}
555555

556-
public synchronized String getTokenTrackingId(TokenIdent identifier) {
556+
public String getTokenTrackingId(TokenIdent identifier) {
557557
DelegationTokenInformation info = getTokenInfo(identifier);
558558
if (info == null) {
559559
return null;
@@ -567,7 +567,7 @@ public synchronized String getTokenTrackingId(TokenIdent identifier) {
567567
* @param password Password in the token.
568568
* @throws InvalidToken InvalidToken.
569569
*/
570-
public synchronized void verifyToken(TokenIdent identifier, byte[] password)
570+
public void verifyToken(TokenIdent identifier, byte[] password)
571571
throws InvalidToken {
572572
byte[] storedPassword = retrievePassword(identifier);
573573
if (!MessageDigest.isEqual(password, storedPassword)) {
@@ -584,7 +584,7 @@ public synchronized void verifyToken(TokenIdent identifier, byte[] password)
584584
* @throws InvalidToken if the token is invalid
585585
* @throws AccessControlException if the user can't renew token
586586
*/
587-
public synchronized long renewToken(Token<TokenIdent> token,
587+
public long renewToken(Token<TokenIdent> token,
588588
String renewer) throws InvalidToken, IOException {
589589
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
590590
DataInputStream in = new DataInputStream(buf);
@@ -646,7 +646,7 @@ public synchronized long renewToken(Token<TokenIdent> token,
646646
* @throws InvalidToken for invalid token
647647
* @throws AccessControlException if the user isn't allowed to cancel
648648
*/
649-
public synchronized TokenIdent cancelToken(Token<TokenIdent> token,
649+
public TokenIdent cancelToken(Token<TokenIdent> token,
650650
String canceller) throws IOException {
651651
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
652652
DataInputStream in = new DataInputStream(buf);

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java

Lines changed: 23 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.util.Collections;
2828
import java.util.List;
2929
import java.util.concurrent.atomic.AtomicInteger;
30+
import java.util.concurrent.locks.ReentrantLock;
3031
import java.util.stream.Stream;
3132

3233
import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
@@ -148,7 +149,7 @@ protected static CuratorFramework getCurator() {
148149
private final int seqNumBatchSize;
149150
private int currentSeqNum;
150151
private int currentMaxSeqNum;
151-
152+
private final ReentrantLock currentSeqNumLock;
152153
private final boolean isTokenWatcherEnabled;
153154

154155
public ZKDelegationTokenSecretManager(Configuration conf) {
@@ -164,6 +165,7 @@ public ZKDelegationTokenSecretManager(Configuration conf) {
164165
ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT);
165166
isTokenWatcherEnabled = conf.getBoolean(ZK_DTSM_TOKEN_WATCHER_ENABLED,
166167
ZK_DTSM_TOKEN_WATCHER_ENABLED_DEFAULT);
168+
this.currentSeqNumLock = new ReentrantLock(true);
167169
if (CURATOR_TL.get() != null) {
168170
zkClient =
169171
CURATOR_TL.get().usingNamespace(
@@ -520,24 +522,28 @@ protected int incrementDelegationTokenSeqNum() {
520522
// The secret manager will keep a local range of seq num which won't be
521523
// seen by peers, so only when the range is exhausted it will ask zk for
522524
// another range again
523-
if (currentSeqNum >= currentMaxSeqNum) {
524-
try {
525-
// after a successful batch request, we can get the range starting point
526-
currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize);
527-
currentMaxSeqNum = currentSeqNum + seqNumBatchSize;
528-
LOG.info("Fetched new range of seq num, from {} to {} ",
529-
currentSeqNum+1, currentMaxSeqNum);
530-
} catch (InterruptedException e) {
531-
// The ExpirationThread is just finishing.. so dont do anything..
532-
LOG.debug(
533-
"Thread interrupted while performing token counter increment", e);
534-
Thread.currentThread().interrupt();
535-
} catch (Exception e) {
536-
throw new RuntimeException("Could not increment shared counter !!", e);
525+
try {
526+
this.currentSeqNumLock.lock();
527+
if (currentSeqNum >= currentMaxSeqNum) {
528+
try {
529+
// after a successful batch request, we can get the range starting point
530+
currentSeqNum = incrSharedCount(delTokSeqCounter, seqNumBatchSize);
531+
currentMaxSeqNum = currentSeqNum + seqNumBatchSize;
532+
LOG.info("Fetched new range of seq num, from {} to {} ",
533+
currentSeqNum+1, currentMaxSeqNum);
534+
} catch (InterruptedException e) {
535+
// The ExpirationThread is just finishing.. so dont do anything..
536+
LOG.debug(
537+
"Thread interrupted while performing token counter increment", e);
538+
Thread.currentThread().interrupt();
539+
} catch (Exception e) {
540+
throw new RuntimeException("Could not increment shared counter !!", e);
541+
}
537542
}
543+
return ++currentSeqNum;
544+
} finally {
545+
this.currentSeqNumLock.unlock();
538546
}
539-
540-
return ++currentSeqNum;
541547
}
542548

543549
@Override

0 commit comments

Comments
 (0)