Browse Source

Hadoop-18520. Backport HADOOP-18427 and HADOOP-18452 to branch-3.3 (#5118)

* HADOOP-18427. Improve ZKDelegationTokenSecretManager#startThead With recommended methods. (#4812)

* HADOOP-18452. Fix TestKMS#testKMSHAZooKeeperDelegationToken Failed By Hadoop-18427. (#4885)

Co-authored-by: slfan1989 <55643692+slfan1989@users.noreply.github.com>
Melissa You 2 years ago
parent
commit
d33ee67151

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

@@ -42,7 +42,6 @@ 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;
-import org.apache.curator.utils.EnsurePath;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
@@ -60,6 +59,7 @@ import org.apache.zookeeper.ZooDefs.Perms;
 import org.apache.zookeeper.client.ZKClientConfig;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -134,6 +134,11 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
     CURATOR_TL.set(curator);
   }
 
+  @VisibleForTesting
+  protected static CuratorFramework getCurator() {
+    return CURATOR_TL.get();
+  }
+
   private final boolean isExternalClient;
   protected final CuratorFramework zkClient;
   private SharedCount delTokSeqCounter;
@@ -260,10 +265,12 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
       // If namespace parents are implicitly created, they won't have ACLs.
       // So, let's explicitly create them.
       CuratorFramework nullNsFw = zkClient.usingNamespace(null);
-      EnsurePath ensureNs =
-          nullNsFw.newNamespaceAwareEnsurePath("/" + zkClient.getNamespace());
       try {
-        ensureNs.ensure(nullNsFw.getZookeeperClient());
+        String nameSpace = "/" + zkClient.getNamespace();
+        Stat stat = nullNsFw.checkExists().forPath(nameSpace);
+        if (stat == null) {
+          nullNsFw.create().creatingParentContainersIfNeeded().forPath(nameSpace);
+        }
       } catch (Exception e) {
         throw new IOException("Could not create namespace", e);
       }

+ 66 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestZKDelegationTokenSecretManager.java

@@ -27,6 +27,8 @@ import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
 import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.api.CreateBuilder;
+import org.apache.curator.framework.api.ProtectACLCreateModeStatPathAndBytesable;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.test.TestingServer;
 import org.apache.hadoop.conf.Configuration;
@@ -37,9 +39,12 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 import org.junit.After;
 import org.junit.Assert;
@@ -506,4 +511,65 @@ public class TestZKDelegationTokenSecretManager {
       }
     }, 1000, 5000);
   }
+
+  @Test
+  public void testCreatingParentContainersIfNeeded() throws Exception {
+
+    String connectString = zkServer.getConnectString();
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    Configuration conf = getSecretConf(connectString);
+    CuratorFramework curatorFramework =
+        CuratorFrameworkFactory.builder()
+        .connectString(connectString)
+        .retryPolicy(retryPolicy)
+        .build();
+    curatorFramework.start();
+    ZKDelegationTokenSecretManager.setCurator(curatorFramework);
+    DelegationTokenManager tm1 = new DelegationTokenManager(conf, new Text("foo"));
+
+    // When the init method is called,
+    // the ZKDelegationTokenSecretManager#startThread method will be called,
+    // and the creatingParentContainersIfNeeded will be called to create the nameSpace.
+    tm1.init();
+
+    String workingPath = "/" + conf.get(ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH,
+        ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH_DEAFULT) + "/ZKDTSMRoot";
+
+    // Check if the created NameSpace exists.
+    Stat stat = curatorFramework.checkExists().forPath(workingPath);
+    Assert.assertNotNull(stat);
+
+    tm1.destroy();
+    curatorFramework.close();
+  }
+
+  @Test
+  public void testCreateNameSpaceRepeatedly() throws Exception {
+
+    String connectString = zkServer.getConnectString();
+    RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
+    Configuration conf = getSecretConf(connectString);
+    CuratorFramework curatorFramework =
+        CuratorFrameworkFactory.builder().
+        connectString(connectString).
+        retryPolicy(retryPolicy).
+        build();
+    curatorFramework.start();
+
+    String workingPath = "/" + conf.get(ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH,
+        ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH_DEAFULT) + "/ZKDTSMRoot-Test";
+    CreateBuilder createBuilder = curatorFramework.create();
+    ProtectACLCreateModeStatPathAndBytesable<String> createModeStat =
+        createBuilder.creatingParentContainersIfNeeded();
+    createModeStat.forPath(workingPath);
+
+    // Check if the created NameSpace exists.
+    Stat stat = curatorFramework.checkExists().forPath(workingPath);
+    Assert.assertNotNull(stat);
+
+    // Repeated creation will throw NodeExists exception
+    LambdaTestUtils.intercept(KeeperException.class,
+        "KeeperErrorCode = NodeExists for "+workingPath,
+        () -> createModeStat.forPath(workingPath));
+  }
 }