diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt index cd744f99ba4..b201d9acb72 100644 --- a/hadoop-common-project/hadoop-common/CHANGES.txt +++ b/hadoop-common-project/hadoop-common/CHANGES.txt @@ -292,6 +292,9 @@ Release 2.7.1 - UNRELEASED HADOOP-11966. Variable cygwin is undefined in hadoop-config.sh when executed through hadoop-daemon.sh. (cnauroth) + HADOOP-11973. Ensure ZkDelegationTokenSecretManager namespace znodes get + created with ACLs. (Gregory Chanan via asuresh) + Release 2.7.0 - 2015-04-20 INCOMPATIBLE CHANGES diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 73c3ab82390..da0e6adc626 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -47,6 +47,7 @@ import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; 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; @@ -298,6 +299,17 @@ public abstract class ZKDelegationTokenSecretManager getAclForPath(String path) { return getDefaultAcl(); } + + @Override + public List getDefaultAcl() { + List ret = new ArrayList(); + ret.add(digestACL); + return ret; + } + }; + + CuratorFramework curatorFramework = + CuratorFrameworkFactory.builder() + .connectString(connectString) + .retryPolicy(retryPolicy) + .aclProvider(digestAclProvider) + .authorization("digest", userPass.getBytes("UTF-8")) + .build(); + curatorFramework.start(); + ZKDelegationTokenSecretManager.setCurator(curatorFramework); + tm1 = new DelegationTokenManager(conf, new Text("bla")); + tm1.init(); + + // check ACL + String workingPath = conf.get(ZKDelegationTokenSecretManager.ZK_DTSM_ZNODE_WORKING_PATH); + verifyACL(curatorFramework, "/" + workingPath, digestACL); + + tm1.destroy(); + ZKDelegationTokenSecretManager.setCurator(null); + curatorFramework.close(); + } + + private void verifyACL(CuratorFramework curatorFramework, + String path, ACL expectedACL) throws Exception { + List acls = curatorFramework.getACL().forPath(path); + Assert.assertEquals(1, acls.size()); + Assert.assertEquals(expectedACL, acls.get(0)); + } + // Since it is possible that there can be a delay for the cancel token message // initiated by one node to reach another node.. The second node can ofcourse // verify with ZK directly if the token that needs verification has been