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