|
@@ -23,12 +23,11 @@ import java.io.ByteArrayOutputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.UncheckedIOException;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.concurrent.ExecutorService;
|
|
|
-import java.util.concurrent.Executors;
|
|
|
-import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
+import java.util.stream.Stream;
|
|
|
|
|
|
import org.apache.curator.ensemble.fixed.FixedEnsembleProvider;
|
|
|
import org.apache.curator.framework.CuratorFramework;
|
|
@@ -37,10 +36,9 @@ import org.apache.curator.framework.CuratorFrameworkFactory.Builder;
|
|
|
import org.apache.curator.framework.api.ACLProvider;
|
|
|
import org.apache.curator.framework.imps.DefaultACLProvider;
|
|
|
import org.apache.curator.framework.recipes.cache.ChildData;
|
|
|
-import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
|
|
-import org.apache.curator.framework.recipes.cache.PathChildrenCache.StartMode;
|
|
|
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
|
|
-import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
|
|
+import org.apache.curator.framework.recipes.cache.CuratorCache;
|
|
|
+import org.apache.curator.framework.recipes.cache.CuratorCacheBridge;
|
|
|
+import org.apache.curator.framework.recipes.cache.CuratorCacheListener;
|
|
|
import org.apache.curator.framework.recipes.shared.SharedCount;
|
|
|
import org.apache.curator.framework.recipes.shared.VersionedValue;
|
|
|
import org.apache.curator.retry.RetryNTimes;
|
|
@@ -54,6 +52,7 @@ import org.apache.hadoop.security.authentication.util.JaasConfiguration;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
|
|
|
import org.apache.hadoop.util.curator.ZKCuratorManager;
|
|
|
+import static org.apache.hadoop.util.Time.now;
|
|
|
import org.apache.zookeeper.CreateMode;
|
|
|
import org.apache.zookeeper.KeeperException;
|
|
|
import org.apache.zookeeper.KeeperException.NoNodeException;
|
|
@@ -78,7 +77,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
|
|
|
public abstract class ZKDelegationTokenSecretManager<TokenIdent extends AbstractDelegationTokenIdentifier>
|
|
|
extends AbstractDelegationTokenSecretManager<TokenIdent> {
|
|
|
|
|
|
- private static final String ZK_CONF_PREFIX = "zk-dt-secret-manager.";
|
|
|
+ public static final String ZK_CONF_PREFIX = "zk-dt-secret-manager.";
|
|
|
public static final String ZK_DTSM_ZK_NUM_RETRIES = ZK_CONF_PREFIX
|
|
|
+ "zkNumRetries";
|
|
|
public static final String ZK_DTSM_ZK_SESSION_TIMEOUT = ZK_CONF_PREFIX
|
|
@@ -101,6 +100,9 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
+ "kerberos.server.principal";
|
|
|
public static final String ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE = ZK_CONF_PREFIX
|
|
|
+ "token.seqnum.batch.size";
|
|
|
+ public static final String ZK_DTSM_TOKEN_WATCHER_ENABLED = ZK_CONF_PREFIX
|
|
|
+ + "token.watcher.enabled";
|
|
|
+ public static final boolean ZK_DTSM_TOKEN_WATCHER_ENABLED_DEFAULT = true;
|
|
|
|
|
|
public static final int ZK_DTSM_ZK_NUM_RETRIES_DEFAULT = 3;
|
|
|
public static final int ZK_DTSM_ZK_SESSION_TIMEOUT_DEFAULT = 10000;
|
|
@@ -110,7 +112,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
// by default it is still incrementing seq number by 1 each time
|
|
|
public static final int ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT = 1;
|
|
|
|
|
|
- private static Logger LOG = LoggerFactory
|
|
|
+ private static final Logger LOG = LoggerFactory
|
|
|
.getLogger(ZKDelegationTokenSecretManager.class);
|
|
|
|
|
|
private static final String JAAS_LOGIN_ENTRY_NAME =
|
|
@@ -119,7 +121,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
private static final String ZK_DTSM_NAMESPACE = "ZKDTSMRoot";
|
|
|
private static final String ZK_DTSM_SEQNUM_ROOT = "/ZKDTSMSeqNumRoot";
|
|
|
private static final String ZK_DTSM_KEYID_ROOT = "/ZKDTSMKeyIdRoot";
|
|
|
- private static final String ZK_DTSM_TOKENS_ROOT = "/ZKDTSMTokensRoot";
|
|
|
+ protected static final String ZK_DTSM_TOKENS_ROOT = "/ZKDTSMTokensRoot";
|
|
|
private static final String ZK_DTSM_MASTER_KEY_ROOT = "/ZKDTSMMasterKeyRoot";
|
|
|
|
|
|
private static final String DELEGATION_KEY_PREFIX = "DK_";
|
|
@@ -133,17 +135,17 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
}
|
|
|
|
|
|
private final boolean isExternalClient;
|
|
|
- private final CuratorFramework zkClient;
|
|
|
+ protected final CuratorFramework zkClient;
|
|
|
private SharedCount delTokSeqCounter;
|
|
|
private SharedCount keyIdSeqCounter;
|
|
|
- private PathChildrenCache keyCache;
|
|
|
- private PathChildrenCache tokenCache;
|
|
|
- private ExecutorService listenerThreadPool;
|
|
|
- private final long shutdownTimeout;
|
|
|
+ private CuratorCacheBridge keyCache;
|
|
|
+ private CuratorCacheBridge tokenCache;
|
|
|
private final int seqNumBatchSize;
|
|
|
private int currentSeqNum;
|
|
|
private int currentMaxSeqNum;
|
|
|
|
|
|
+ private final boolean isTokenWatcherEnabled;
|
|
|
+
|
|
|
public ZKDelegationTokenSecretManager(Configuration conf) {
|
|
|
super(conf.getLong(DelegationTokenManager.UPDATE_INTERVAL,
|
|
|
DelegationTokenManager.UPDATE_INTERVAL_DEFAULT) * 1000,
|
|
@@ -153,10 +155,10 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
DelegationTokenManager.RENEW_INTERVAL_DEFAULT) * 1000,
|
|
|
conf.getLong(DelegationTokenManager.REMOVAL_SCAN_INTERVAL,
|
|
|
DelegationTokenManager.REMOVAL_SCAN_INTERVAL_DEFAULT) * 1000);
|
|
|
- shutdownTimeout = conf.getLong(ZK_DTSM_ZK_SHUTDOWN_TIMEOUT,
|
|
|
- ZK_DTSM_ZK_SHUTDOWN_TIMEOUT_DEFAULT);
|
|
|
seqNumBatchSize = conf.getInt(ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE,
|
|
|
ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE_DEFAULT);
|
|
|
+ isTokenWatcherEnabled = conf.getBoolean(ZK_DTSM_TOKEN_WATCHER_ENABLED,
|
|
|
+ ZK_DTSM_TOKEN_WATCHER_ENABLED_DEFAULT);
|
|
|
if (CURATOR_TL.get() != null) {
|
|
|
zkClient =
|
|
|
CURATOR_TL.get().usingNamespace(
|
|
@@ -259,14 +261,13 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
// So, let's explicitly create them.
|
|
|
CuratorFramework nullNsFw = zkClient.usingNamespace(null);
|
|
|
EnsurePath ensureNs =
|
|
|
- nullNsFw.newNamespaceAwareEnsurePath("/" + zkClient.getNamespace());
|
|
|
+ nullNsFw.newNamespaceAwareEnsurePath("/" + zkClient.getNamespace());
|
|
|
try {
|
|
|
ensureNs.ensure(nullNsFw.getZookeeperClient());
|
|
|
} catch (Exception e) {
|
|
|
throw new IOException("Could not create namespace", e);
|
|
|
}
|
|
|
}
|
|
|
- listenerThreadPool = Executors.newSingleThreadExecutor();
|
|
|
try {
|
|
|
delTokSeqCounter = new SharedCount(zkClient, ZK_DTSM_SEQNUM_ROOT, 0);
|
|
|
if (delTokSeqCounter != null) {
|
|
@@ -296,68 +297,65 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
throw new RuntimeException("Could not create ZK paths");
|
|
|
}
|
|
|
try {
|
|
|
- keyCache = new PathChildrenCache(zkClient, ZK_DTSM_MASTER_KEY_ROOT, true);
|
|
|
- if (keyCache != null) {
|
|
|
- keyCache.start(StartMode.BUILD_INITIAL_CACHE);
|
|
|
- keyCache.getListenable().addListener(new PathChildrenCacheListener() {
|
|
|
- @Override
|
|
|
- public void childEvent(CuratorFramework client,
|
|
|
- PathChildrenCacheEvent event)
|
|
|
- throws Exception {
|
|
|
- switch (event.getType()) {
|
|
|
- case CHILD_ADDED:
|
|
|
- processKeyAddOrUpdate(event.getData().getData());
|
|
|
- break;
|
|
|
- case CHILD_UPDATED:
|
|
|
- processKeyAddOrUpdate(event.getData().getData());
|
|
|
- break;
|
|
|
- case CHILD_REMOVED:
|
|
|
- processKeyRemoved(event.getData().getPath());
|
|
|
- break;
|
|
|
- default:
|
|
|
- break;
|
|
|
+ keyCache = CuratorCache.bridgeBuilder(zkClient, ZK_DTSM_MASTER_KEY_ROOT)
|
|
|
+ .build();
|
|
|
+ CuratorCacheListener keyCacheListener = CuratorCacheListener.builder()
|
|
|
+ .forCreatesAndChanges((oldNode, node) -> {
|
|
|
+ try {
|
|
|
+ processKeyAddOrUpdate(node.getData());
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Error while processing Curator keyCacheListener "
|
|
|
+ + "NODE_CREATED / NODE_CHANGED event");
|
|
|
+ throw new UncheckedIOException(e);
|
|
|
}
|
|
|
- }
|
|
|
- }, listenerThreadPool);
|
|
|
- loadFromZKCache(false);
|
|
|
- }
|
|
|
+ })
|
|
|
+ .forDeletes(childData -> processKeyRemoved(childData.getPath()))
|
|
|
+ .build();
|
|
|
+ keyCache.listenable().addListener(keyCacheListener);
|
|
|
+ keyCache.start();
|
|
|
+ loadFromZKCache(false);
|
|
|
} catch (Exception e) {
|
|
|
- throw new IOException("Could not start PathChildrenCache for keys", e);
|
|
|
+ throw new IOException("Could not start Curator keyCacheListener for keys",
|
|
|
+ e);
|
|
|
}
|
|
|
- try {
|
|
|
- tokenCache = new PathChildrenCache(zkClient, ZK_DTSM_TOKENS_ROOT, true);
|
|
|
- if (tokenCache != null) {
|
|
|
- tokenCache.start(StartMode.BUILD_INITIAL_CACHE);
|
|
|
- tokenCache.getListenable().addListener(new PathChildrenCacheListener() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public void childEvent(CuratorFramework client,
|
|
|
- PathChildrenCacheEvent event) throws Exception {
|
|
|
- switch (event.getType()) {
|
|
|
- case CHILD_ADDED:
|
|
|
- processTokenAddOrUpdate(event.getData());
|
|
|
- break;
|
|
|
- case CHILD_UPDATED:
|
|
|
- processTokenAddOrUpdate(event.getData());
|
|
|
- break;
|
|
|
- case CHILD_REMOVED:
|
|
|
- processTokenRemoved(event.getData());
|
|
|
- break;
|
|
|
- default:
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }, listenerThreadPool);
|
|
|
+ if (isTokenWatcherEnabled) {
|
|
|
+ LOG.info("TokenCache is enabled");
|
|
|
+ try {
|
|
|
+ tokenCache = CuratorCache.bridgeBuilder(zkClient, ZK_DTSM_TOKENS_ROOT)
|
|
|
+ .build();
|
|
|
+ CuratorCacheListener tokenCacheListener = CuratorCacheListener.builder()
|
|
|
+ .forCreatesAndChanges((oldNode, node) -> {
|
|
|
+ try {
|
|
|
+ processTokenAddOrUpdate(node.getData());
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Error while processing Curator tokenCacheListener "
|
|
|
+ + "NODE_CREATED / NODE_CHANGED event");
|
|
|
+ throw new UncheckedIOException(e);
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .forDeletes(childData -> {
|
|
|
+ try {
|
|
|
+ processTokenRemoved(childData);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.error("Error while processing Curator tokenCacheListener "
|
|
|
+ + "NODE_DELETED event");
|
|
|
+ throw new UncheckedIOException(e);
|
|
|
+ }
|
|
|
+ })
|
|
|
+ .build();
|
|
|
+ tokenCache.listenable().addListener(tokenCacheListener);
|
|
|
+ tokenCache.start();
|
|
|
loadFromZKCache(true);
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw new IOException(
|
|
|
+ "Could not start Curator tokenCacheListener for tokens", e);
|
|
|
}
|
|
|
- } catch (Exception e) {
|
|
|
- throw new IOException("Could not start PathChildrenCache for tokens", e);
|
|
|
}
|
|
|
super.startThreads();
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Load the PathChildrenCache into the in-memory map. Possible caches to be
|
|
|
+ * Load the CuratorCache into the in-memory map. Possible caches to be
|
|
|
* loaded are keyCache and tokenCache.
|
|
|
*
|
|
|
* @param isTokenCache true if loading tokenCache, false if loading keyCache.
|
|
@@ -365,30 +363,31 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
private void loadFromZKCache(final boolean isTokenCache) {
|
|
|
final String cacheName = isTokenCache ? "token" : "key";
|
|
|
LOG.info("Starting to load {} cache.", cacheName);
|
|
|
- final List<ChildData> children;
|
|
|
+ final Stream<ChildData> children;
|
|
|
if (isTokenCache) {
|
|
|
- children = tokenCache.getCurrentData();
|
|
|
+ children = tokenCache.stream();
|
|
|
} else {
|
|
|
- children = keyCache.getCurrentData();
|
|
|
+ children = keyCache.stream();
|
|
|
}
|
|
|
|
|
|
- int count = 0;
|
|
|
- for (ChildData child : children) {
|
|
|
+ final AtomicInteger count = new AtomicInteger(0);
|
|
|
+ children.forEach(childData -> {
|
|
|
try {
|
|
|
if (isTokenCache) {
|
|
|
- processTokenAddOrUpdate(child);
|
|
|
+ processTokenAddOrUpdate(childData.getData());
|
|
|
} else {
|
|
|
- processKeyAddOrUpdate(child.getData());
|
|
|
+ processKeyAddOrUpdate(childData.getData());
|
|
|
}
|
|
|
} catch (Exception e) {
|
|
|
LOG.info("Ignoring node {} because it failed to load.",
|
|
|
- child.getPath());
|
|
|
+ childData.getPath());
|
|
|
LOG.debug("Failure exception:", e);
|
|
|
- ++count;
|
|
|
+ count.getAndIncrement();
|
|
|
}
|
|
|
- }
|
|
|
- if (count > 0) {
|
|
|
- LOG.warn("Ignored {} nodes while loading {} cache.", count, cacheName);
|
|
|
+ });
|
|
|
+ if (count.get() > 0) {
|
|
|
+ LOG.warn("Ignored {} nodes while loading {} cache.", count.get(),
|
|
|
+ cacheName);
|
|
|
}
|
|
|
LOG.info("Loaded {} cache.", cacheName);
|
|
|
}
|
|
@@ -398,9 +397,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
DataInputStream din = new DataInputStream(bin);
|
|
|
DelegationKey key = new DelegationKey();
|
|
|
key.readFields(din);
|
|
|
- synchronized (this) {
|
|
|
- allKeys.put(key.getKeyId(), key);
|
|
|
- }
|
|
|
+ allKeys.put(key.getKeyId(), key);
|
|
|
}
|
|
|
|
|
|
private void processKeyRemoved(String path) {
|
|
@@ -410,15 +407,13 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
int j = tokSeg.indexOf('_');
|
|
|
if (j > 0) {
|
|
|
int keyId = Integer.parseInt(tokSeg.substring(j + 1));
|
|
|
- synchronized (this) {
|
|
|
- allKeys.remove(keyId);
|
|
|
- }
|
|
|
+ allKeys.remove(keyId);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void processTokenAddOrUpdate(ChildData data) throws IOException {
|
|
|
- ByteArrayInputStream bin = new ByteArrayInputStream(data.getData());
|
|
|
+ protected TokenIdent processTokenAddOrUpdate(byte[] data) throws IOException {
|
|
|
+ ByteArrayInputStream bin = new ByteArrayInputStream(data);
|
|
|
DataInputStream din = new DataInputStream(bin);
|
|
|
TokenIdent ident = createIdentifier();
|
|
|
ident.readFields(din);
|
|
@@ -429,12 +424,10 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
if (numRead > -1) {
|
|
|
DelegationTokenInformation tokenInfo =
|
|
|
new DelegationTokenInformation(renewDate, password);
|
|
|
- synchronized (this) {
|
|
|
- currentTokens.put(ident, tokenInfo);
|
|
|
- // The cancel task might be waiting
|
|
|
- notifyAll();
|
|
|
- }
|
|
|
+ currentTokens.put(ident, tokenInfo);
|
|
|
+ return ident;
|
|
|
}
|
|
|
+ return null;
|
|
|
}
|
|
|
|
|
|
private void processTokenRemoved(ChildData data) throws IOException {
|
|
@@ -442,11 +435,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
DataInputStream din = new DataInputStream(bin);
|
|
|
TokenIdent ident = createIdentifier();
|
|
|
ident.readFields(din);
|
|
|
- synchronized (this) {
|
|
|
- currentTokens.remove(ident);
|
|
|
- // The cancel task might be waiting
|
|
|
- notifyAll();
|
|
|
- }
|
|
|
+ currentTokens.remove(ident);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -487,20 +476,6 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Could not stop Curator Framework", e);
|
|
|
}
|
|
|
- if (listenerThreadPool != null) {
|
|
|
- listenerThreadPool.shutdown();
|
|
|
- try {
|
|
|
- // wait for existing tasks to terminate
|
|
|
- if (!listenerThreadPool.awaitTermination(shutdownTimeout,
|
|
|
- TimeUnit.MILLISECONDS)) {
|
|
|
- LOG.error("Forcing Listener threadPool to shutdown !!");
|
|
|
- listenerThreadPool.shutdownNow();
|
|
|
- }
|
|
|
- } catch (InterruptedException ie) {
|
|
|
- listenerThreadPool.shutdownNow();
|
|
|
- Thread.currentThread().interrupt();
|
|
|
- }
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
private void createPersistentNode(String nodePath) throws Exception {
|
|
@@ -647,7 +622,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
*
|
|
|
* @param ident Identifier of the token
|
|
|
*/
|
|
|
- private synchronized void syncLocalCacheWithZk(TokenIdent ident) {
|
|
|
+ protected void syncLocalCacheWithZk(TokenIdent ident) {
|
|
|
try {
|
|
|
DelegationTokenInformation tokenInfo = getTokenInfoFromZK(ident);
|
|
|
if (tokenInfo != null && !currentTokens.containsKey(ident)) {
|
|
@@ -661,16 +636,21 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private DelegationTokenInformation getTokenInfoFromZK(TokenIdent ident)
|
|
|
+ protected DelegationTokenInformation getTokenInfoFromZK(TokenIdent ident)
|
|
|
throws IOException {
|
|
|
return getTokenInfoFromZK(ident, false);
|
|
|
}
|
|
|
|
|
|
- private DelegationTokenInformation getTokenInfoFromZK(TokenIdent ident,
|
|
|
+ protected DelegationTokenInformation getTokenInfoFromZK(TokenIdent ident,
|
|
|
boolean quiet) throws IOException {
|
|
|
String nodePath =
|
|
|
getNodePath(ZK_DTSM_TOKENS_ROOT,
|
|
|
DELEGATION_TOKEN_PREFIX + ident.getSequenceNumber());
|
|
|
+ return getTokenInfoFromZK(nodePath, quiet);
|
|
|
+ }
|
|
|
+
|
|
|
+ protected DelegationTokenInformation getTokenInfoFromZK(String nodePath,
|
|
|
+ boolean quiet) throws IOException {
|
|
|
try {
|
|
|
byte[] data = zkClient.getData().forPath(nodePath);
|
|
|
if ((data == null) || (data.length == 0)) {
|
|
@@ -805,15 +785,30 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
@Override
|
|
|
protected void removeStoredToken(TokenIdent ident)
|
|
|
throws IOException {
|
|
|
+ removeStoredToken(ident, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ protected void removeStoredToken(TokenIdent ident,
|
|
|
+ boolean checkAgainstZkBeforeDeletion) throws IOException {
|
|
|
String nodeRemovePath =
|
|
|
getNodePath(ZK_DTSM_TOKENS_ROOT, DELEGATION_TOKEN_PREFIX
|
|
|
+ ident.getSequenceNumber());
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Removing ZKDTSMDelegationToken_"
|
|
|
- + ident.getSequenceNumber());
|
|
|
- }
|
|
|
try {
|
|
|
- if (zkClient.checkExists().forPath(nodeRemovePath) != null) {
|
|
|
+ DelegationTokenInformation dtInfo = getTokenInfoFromZK(ident, true);
|
|
|
+ if (dtInfo != null) {
|
|
|
+ // For the case there is no sync or watch miss, it is possible that the
|
|
|
+ // local storage has expired tokens which have been renewed by peer
|
|
|
+ // so double check again to avoid accidental delete
|
|
|
+ if (checkAgainstZkBeforeDeletion
|
|
|
+ && dtInfo.getRenewDate() > now()) {
|
|
|
+ LOG.info("Node already renewed by peer " + nodeRemovePath +
|
|
|
+ " so this token should not be deleted");
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Removing ZKDTSMDelegationToken_"
|
|
|
+ + ident.getSequenceNumber());
|
|
|
+ }
|
|
|
while(zkClient.checkExists().forPath(nodeRemovePath) != null){
|
|
|
try {
|
|
|
zkClient.delete().guaranteed().forPath(nodeRemovePath);
|
|
@@ -836,7 +831,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- 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);
|
|
@@ -847,7 +842,7 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
return super.cancelToken(token, canceller);
|
|
|
}
|
|
|
|
|
|
- private void addOrUpdateToken(TokenIdent ident,
|
|
|
+ protected void addOrUpdateToken(TokenIdent ident,
|
|
|
DelegationTokenInformation info, boolean isUpdate) throws Exception {
|
|
|
String nodeCreatePath =
|
|
|
getNodePath(ZK_DTSM_TOKENS_ROOT, DELEGATION_TOKEN_PREFIX
|
|
@@ -874,6 +869,10 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ public boolean isTokenWatcherEnabled() {
|
|
|
+ return isTokenWatcherEnabled;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Simple implementation of an {@link ACLProvider} that simply returns an ACL
|
|
|
* that gives all permissions only to a single principal.
|
|
@@ -905,11 +904,6 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
return (root + "/" + nodeName);
|
|
|
}
|
|
|
|
|
|
- @VisibleForTesting
|
|
|
- public ExecutorService getListenerThreadPool() {
|
|
|
- return listenerThreadPool;
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
DelegationTokenInformation getTokenInfoFromMemory(TokenIdent ident) {
|
|
|
return currentTokens.get(ident);
|