Skip to content

Commit d1551e9

Browse files
slfan1989Melissa You
authored andcommitted
HADOOP-18452. Fix TestKMS#testKMSHAZooKeeperDelegationToken Failed By Hadoop-18427. (#4885)
1 parent 06af0ff commit d1551e9

File tree

2 files changed

+43
-1
lines changed

2 files changed

+43
-1
lines changed

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

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@
5959
import org.apache.zookeeper.client.ZKClientConfig;
6060
import org.apache.zookeeper.data.ACL;
6161
import org.apache.zookeeper.data.Id;
62+
import org.apache.zookeeper.data.Stat;
6263
import org.slf4j.Logger;
6364
import org.slf4j.LoggerFactory;
6465

@@ -265,7 +266,11 @@ public void startThreads() throws IOException {
265266
// So, let's explicitly create them.
266267
CuratorFramework nullNsFw = zkClient.usingNamespace(null);
267268
try {
268-
nullNsFw.create().creatingParentContainersIfNeeded().forPath("/" + zkClient.getNamespace());
269+
String nameSpace = "/" + zkClient.getNamespace();
270+
Stat stat = nullNsFw.checkExists().forPath(nameSpace);
271+
if (stat == null) {
272+
nullNsFw.create().creatingParentContainersIfNeeded().forPath(nameSpace);
273+
}
269274
} catch (Exception e) {
270275
throw new IOException("Could not create namespace", e);
271276
}

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

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,8 @@
2727
import org.apache.curator.framework.CuratorFramework;
2828
import org.apache.curator.framework.CuratorFrameworkFactory;
2929
import org.apache.curator.framework.api.ACLProvider;
30+
import org.apache.curator.framework.api.CreateBuilder;
31+
import org.apache.curator.framework.api.ProtectACLCreateModeStatPathAndBytesable;
3032
import org.apache.curator.retry.ExponentialBackoffRetry;
3133
import org.apache.curator.test.TestingServer;
3234
import org.apache.hadoop.conf.Configuration;
@@ -37,6 +39,8 @@
3739
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
3840
import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
3941
import org.apache.hadoop.test.GenericTestUtils;
42+
import org.apache.hadoop.test.LambdaTestUtils;
43+
import org.apache.zookeeper.KeeperException;
4044
import org.apache.zookeeper.ZooDefs;
4145
import org.apache.zookeeper.data.ACL;
4246
import org.apache.zookeeper.data.Id;
@@ -534,5 +538,38 @@ public void testCreatingParentContainersIfNeeded() throws Exception {
534538
// Check if the created NameSpace exists.
535539
Stat stat = curatorFramework.checkExists().forPath(workingPath);
536540
Assert.assertNotNull(stat);
541+
542+
tm1.destroy();
543+
curatorFramework.close();
544+
}
545+
546+
@Test
547+
public void testCreateNameSpaceRepeatedly() throws Exception {
548+
549+
String connectString = zkServer.getConnectString();
550+
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
551+
Configuration conf = getSecretConf(connectString);
552+
CuratorFramework curatorFramework =
553+
CuratorFrameworkFactory.builder().
554+
connectString(connectString).
555+
retryPolicy(retryPolicy).
556+
build();
557+
curatorFramework.start();
558+
559+
String workingPath = "/" + conf.get(ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH,
560+
ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH_DEAFULT) + "/ZKDTSMRoot-Test";
561+
CreateBuilder createBuilder = curatorFramework.create();
562+
ProtectACLCreateModeStatPathAndBytesable<String> createModeStat =
563+
createBuilder.creatingParentContainersIfNeeded();
564+
createModeStat.forPath(workingPath);
565+
566+
// Check if the created NameSpace exists.
567+
Stat stat = curatorFramework.checkExists().forPath(workingPath);
568+
Assert.assertNotNull(stat);
569+
570+
// Repeated creation will throw NodeExists exception
571+
LambdaTestUtils.intercept(KeeperException.class,
572+
"KeeperErrorCode = NodeExists for "+workingPath,
573+
() -> createModeStat.forPath(workingPath));
537574
}
538575
}

0 commit comments

Comments
 (0)