|
@@ -23,13 +23,13 @@ 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.HashMap;
|
|
|
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 javax.security.auth.login.AppConfigurationEntry;
|
|
|
|
|
@@ -40,10 +40,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;
|
|
@@ -113,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 =
|
|
@@ -139,10 +138,8 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
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;
|
|
@@ -158,8 +155,6 @@ 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,
|
|
@@ -333,7 +328,6 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
|
|
throw new IOException("Could not create namespace", e);
|
|
|
}
|
|
|
}
|
|
|
- listenerThreadPool = Executors.newSingleThreadExecutor();
|
|
|
try {
|
|
|
delTokSeqCounter = new SharedCount(zkClient, ZK_DTSM_SEQNUM_ROOT, 0);
|
|
|
if (delTokSeqCounter != null) {
|
|
@@ -363,71 +357,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);
|
|
|
}
|
|
|
if (isTokenWatcherEnabled) {
|
|
|
LOG.info("TokenCache is enabled");
|
|
|
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().getData());
|
|
|
- break;
|
|
|
- case CHILD_UPDATED:
|
|
|
- processTokenAddOrUpdate(event.getData().getData());
|
|
|
- break;
|
|
|
- case CHILD_REMOVED:
|
|
|
- processTokenRemoved(event.getData());
|
|
|
- break;
|
|
|
- default:
|
|
|
- break;
|
|
|
+ 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);
|
|
|
}
|
|
|
- }
|
|
|
- }, listenerThreadPool);
|
|
|
- loadFromZKCache(true);
|
|
|
- }
|
|
|
+ })
|
|
|
+ .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 PathChildrenCache for tokens", e);
|
|
|
+ throw new IOException(
|
|
|
+ "Could not start Curator tokenCacheListener 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.
|
|
@@ -435,33 +423,34 @@ 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.getData());
|
|
|
+ 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 (isTokenCache) {
|
|
|
syncTokenOwnerStats();
|
|
|
}
|
|
|
- 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);
|
|
|
}
|
|
@@ -550,20 +539,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 {
|
|
@@ -992,11 +967,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);
|