Explorar o código

HDFS-17270. RBF: Fix ZKDelegationTokenSecretManagerImpl use closed zookeeper client to get token in some case (#6315). Contributed by lei w.

Signed-off-by: Shuyan Zhang <zhangshuyan@apache.org>
Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
Thinker313 hai 1 ano
pai
achega
6a22bead54

+ 17 - 13
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/ZKDelegationTokenSecretManagerImpl.java

@@ -60,8 +60,7 @@ public class ZKDelegationTokenSecretManagerImpl extends
   // currentTokenMap
   private final Set<AbstractDelegationTokenIdentifier> localTokenCache =
       new HashSet<>();
-  // Native zk client for getting all tokens
-  private ZooKeeper zookeeper;
+
   private final String TOKEN_PATH = "/" + zkClient.getNamespace()
       + ZK_DTSM_TOKENS_ROOT;
   // The flag used to issue an extra check before deletion
@@ -103,16 +102,6 @@ public class ZKDelegationTokenSecretManagerImpl extends
               .withMode(CreateMode.PERSISTENT)
               .forPath(ZK_DTSM_TOKENS_ROOT);
         }
-        // Set up zookeeper client
-        try {
-          zookeeper = zkClient.getZookeeperClient().getZooKeeper();
-        } catch (Exception e) {
-          LOG.info("Cannot get zookeeper client ", e);
-        } finally {
-          if (zookeeper == null) {
-            throw new IOException("Zookeeper client is null");
-          }
-        }
 
         LOG.info("Start loading token cache");
         long start = Time.now();
@@ -163,7 +152,7 @@ public class ZKDelegationTokenSecretManagerImpl extends
     // millions of tokens
     List<String> zkTokens;
     try {
-      zkTokens = zookeeper.getChildren(TOKEN_PATH, false);
+      zkTokens = getZooKeeperClient().getChildren(TOKEN_PATH, false);
     } catch (KeeperException | InterruptedException e) {
       throw new IOException("Tokens cannot be fetched from path "
           + TOKEN_PATH, e);
@@ -222,4 +211,19 @@ public class ZKDelegationTokenSecretManagerImpl extends
     currentTokens.put(ident, info);
     super.addOrUpdateToken(ident, info, isUpdate);
   }
+
+  private ZooKeeper getZooKeeperClient() throws IOException {
+    // get zookeeper client
+    ZooKeeper zookeeper = null;
+    try {
+      zookeeper = zkClient.getZookeeperClient().getZooKeeper();
+    } catch (Exception e) {
+      LOG.info("Cannot get zookeeper client ", e);
+    } finally {
+      if (zookeeper == null) {
+        throw new IOException("Zookeeper client is null");
+      }
+    }
+    return zookeeper;
+  }
 }