HADOOP-11973. Ensure ZkDelegationTokenSecretManager namespace znodes get created with ACLs. (Gregory Chanan via asuresh)

(cherry picked from commit fd3cb533d2)
This commit is contained in:
Arun Suresh 2015-05-19 11:35:57 -07:00
parent dafe33efbf
commit 00e2a0a494
3 changed files with 75 additions and 0 deletions

View File

@ -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

View File

@ -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<TokenIdent extends Abstract
} catch (Exception e) {
throw new IOException("Could not start Curator Framework", e);
}
} else {
// 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());
} catch (Exception e) {
throw new IOException("Could not create namespace", e);
}
}
listenerThreadPool = Executors.newSingleThreadExecutor();
try {

View File

@ -19,9 +19,16 @@
package org.apache.hadoop.security.token.delegation;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
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.retry.ExponentialBackoffRetry;
import org.apache.curator.test.TestingServer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
@ -30,6 +37,10 @@ import org.apache.hadoop.security.token.SecretManager;
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.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -296,6 +307,55 @@ public class TestZKDelegationTokenSecretManager {
tm1.destroy();
}
@Test
public void testACLs() throws Exception {
DelegationTokenManager tm1;
String connectString = zkServer.getConnectString();
Configuration conf = getSecretConf(connectString);
RetryPolicy retryPolicy = new ExponentialBackoffRetry(1000, 3);
String userPass = "myuser:mypass";
final ACL digestACL = new ACL(ZooDefs.Perms.ALL, new Id("digest",
DigestAuthenticationProvider.generateDigest(userPass)));
ACLProvider digestAclProvider = new ACLProvider() {
@Override
public List<ACL> getAclForPath(String path) { return getDefaultAcl(); }
@Override
public List<ACL> getDefaultAcl() {
List<ACL> ret = new ArrayList<ACL>();
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<ACL> 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