HBASE-7771 Secure HBase Client in MR job causes tasks to wait forever (Francis and Matteo)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1442833 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2013-02-06 02:12:15 +00:00
parent 5dc539e778
commit 6fcae1027d
4 changed files with 10 additions and 51 deletions

View File

@ -199,7 +199,13 @@ public class ZKUtil {
if (System.getProperty("java.security.auth.login.config") != null)
return;
// No keytab specified, no auth
String keytabFilename = conf.get(keytabFileKey);
if (keytabFilename == null) {
LOG.warn("no keytab specified for: " + keytabFileKey);
return;
}
String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
@ -910,7 +916,8 @@ public class ZKUtil {
return true;
// Master & RSs uses hbase.zookeeper.client.*
return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
return("kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication")) &&
conf.get("hbase.zookeeper.client.keytab.file") != null);
}
private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
@ -933,15 +940,6 @@ public class ZKUtil {
}
}
public static void waitForZKConnectionIfAuthenticating(ZooKeeperWatcher zkw)
throws InterruptedException {
if (isSecureZooKeeper(zkw.getConfiguration())) {
LOG.debug("Waiting for ZooKeeperWatcher to authenticate");
zkw.saslLatch.await();
LOG.debug("Done waiting.");
}
}
//
// Node creation
//
@ -968,7 +966,6 @@ public class ZKUtil {
String znode, byte [] data)
throws KeeperException {
try {
waitForZKConnectionIfAuthenticating(zkw);
zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
CreateMode.EPHEMERAL);
} catch (KeeperException.NodeExistsException nee) {
@ -1008,7 +1005,6 @@ public class ZKUtil {
ZooKeeperWatcher zkw, String znode, byte [] data)
throws KeeperException {
try {
waitForZKConnectionIfAuthenticating(zkw);
zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
CreateMode.PERSISTENT);
} catch (KeeperException.NodeExistsException nee) {
@ -1046,7 +1042,6 @@ public class ZKUtil {
String znode, byte [] data)
throws KeeperException, KeeperException.NodeExistsException {
try {
waitForZKConnectionIfAuthenticating(zkw);
zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
CreateMode.PERSISTENT);
Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
@ -1080,13 +1075,8 @@ public class ZKUtil {
public static void asyncCreate(ZooKeeperWatcher zkw,
String znode, byte [] data, final AsyncCallback.StringCallback cb,
final Object ctx) {
try {
waitForZKConnectionIfAuthenticating(zkw);
zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
} catch (InterruptedException e) {
zkw.interruptedException(e);
}
}
/**
@ -1111,7 +1101,6 @@ public class ZKUtil {
String znode = create.getPath();
try {
RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
waitForZKConnectionIfAuthenticating(zkw);
if (zk.exists(znode, false) == null) {
zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
}
@ -1148,7 +1137,6 @@ public class ZKUtil {
if(znode == null) {
return;
}
waitForZKConnectionIfAuthenticating(zkw);
zkw.getRecoverableZooKeeper().create(znode, new byte[0], createACL(zkw, znode),
CreateMode.PERSISTENT);
} catch(KeeperException.NodeExistsException nee) {

View File

@ -74,12 +74,6 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
* or {@link #getData(boolean)} to get the data of the node if it is available.
*/
public synchronized void start() {
try {
ZKUtil.waitForZKConnectionIfAuthenticating(watcher);
} catch (InterruptedException e) {
throw new IllegalStateException("ZookeeperNodeTracker on " + this.node
+ " interuppted while waiting for SASL Authentication", e);
}
this.watcher.registerListener(this);
try {
if(ZKUtil.watchAndCheckExists(watcher, node)) {

View File

@ -342,34 +342,12 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
LOG.debug(this.identifier + " connected");
break;
case SaslAuthenticated:
if (ZKUtil.isSecureZooKeeper(this.conf)) {
// We are authenticated, clients can proceed.
saslLatch.countDown();
}
break;
case AuthFailed:
if (ZKUtil.isSecureZooKeeper(this.conf)) {
// We could not be authenticated, but clients should proceed anyway.
// Only access to znodes that require SASL authentication will be
// denied. The client may never need to access them.
saslLatch.countDown();
}
break;
// Abort the server if Disconnected or Expired
case Disconnected:
LOG.debug(prefix("Received Disconnected from ZooKeeper, ignoring"));
break;
case Expired:
if (ZKUtil.isSecureZooKeeper(this.conf)) {
// We consider Expired equivalent to AuthFailed for this
// connection. Authentication is never going to complete. The
// client should proceed to do cleanup.
saslLatch.countDown();
}
String msg = prefix(this.identifier + " received expired from " +
"ZooKeeper, aborting");
// TODO: One thought is to add call to ZooKeeperListener so say,

View File

@ -86,7 +86,6 @@ public class TestZooKeeperACL {
zkw = new ZooKeeperWatcher(
new Configuration(TEST_UTIL.getConfiguration()),
TestZooKeeper.class.getName(), null);
ZKUtil.waitForZKConnectionIfAuthenticating(zkw);
}
/**