HBASE-12641 Grant all permissions of hbase zookeeper node to hbase superuser in a secure cluster (Liu Shaohui)

This commit is contained in:
stack 2014-12-27 21:11:57 -08:00
parent 9abab54d8e
commit a8766fd623
2 changed files with 17 additions and 8 deletions

View File

@ -61,9 +61,11 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.Op;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooDefs.Perms;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.client.ZooKeeperSaslClient;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.data.Stat;
import org.apache.zookeeper.proto.CreateRequest;
import org.apache.zookeeper.proto.DeleteRequest;
@ -949,8 +951,17 @@ public class ZKUtil {
conf.get("hbase.zookeeper.client.keytab.file") != null);
}
private static List<ACL> createACL(ZooKeeperWatcher zkw, String node) {
private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
if (!node.startsWith(zkw.baseZNode)) {
return Ids.OPEN_ACL_UNSAFE;
}
if (isSecureZooKeeper(zkw.getConfiguration())) {
String superUser = zkw.getConfiguration().get("hbase.superuser");
ArrayList<ACL> acls = new ArrayList<ACL>();
// add permission to hbase supper user
if (superUser != null) {
acls.add(new ACL(Perms.ALL, new Id("auth", superUser)));
}
// Certain znodes are accessed directly by the client,
// so they must be readable by non-authenticated clients
if ((node.equals(zkw.baseZNode) == true) ||
@ -960,9 +971,12 @@ public class ZKUtil {
(node.equals(zkw.rsZNode) == true) ||
(node.equals(zkw.backupMasterAddressesZNode) == true) ||
(node.startsWith(zkw.tableZNode) == true)) {
return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
acls.addAll(Ids.CREATOR_ALL_ACL);
acls.addAll(Ids.READ_ACL_UNSAFE);
} else {
acls.addAll(Ids.CREATOR_ALL_ACL);
}
return Ids.CREATOR_ALL_ACL;
return acls;
} else {
return Ids.OPEN_ACL_UNSAFE;
}

View File

@ -111,11 +111,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public static String namespaceZNode = "namespace";
// Certain ZooKeeper nodes need to be world-readable
public static final List<ACL> CREATOR_ALL_AND_WORLD_READABLE =
Arrays.asList(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE),
new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
private final Configuration conf;
private final Exception constructorCaller;