HADOOP-18139: Allow configuration of zookeeper server principal.
Fixes #4024 Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
This commit is contained in:
parent
6b07c851f3
commit
12fa38d546
@ -399,6 +399,8 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
|
||||
public static final String ZK_ACL_DEFAULT = "world:anyone:rwcda";
|
||||
/** Authentication for the ZooKeeper ensemble. */
|
||||
public static final String ZK_AUTH = ZK_PREFIX + "auth";
|
||||
/** Principal name for zookeeper servers. */
|
||||
public static final String ZK_SERVER_PRINCIPAL = ZK_PREFIX + "server.principal";
|
||||
|
||||
/** Address of the ZooKeeper ensemble. */
|
||||
public static final String ZK_ADDRESS = ZK_PREFIX + "address";
|
||||
|
@ -55,6 +55,7 @@
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
|
||||
import static org.apache.hadoop.util.Time.now;
|
||||
import org.apache.hadoop.util.curator.ZKCuratorManager;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
@ -98,6 +99,8 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
|
||||
+ "kerberos.keytab";
|
||||
public static final String ZK_DTSM_ZK_KERBEROS_PRINCIPAL = ZK_CONF_PREFIX
|
||||
+ "kerberos.principal";
|
||||
public static final String ZK_DTSM_ZK_KERBEROS_SERVER_PRINCIPAL = ZK_CONF_PREFIX
|
||||
+ "kerberos.server.principal";
|
||||
public static final String ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE = ZK_CONF_PREFIX
|
||||
+ "token.seqnum.batch.size";
|
||||
public static final String ZK_DTSM_TOKEN_WATCHER_ENABLED = ZK_CONF_PREFIX
|
||||
@ -202,6 +205,8 @@ public ZKDelegationTokenSecretManager(Configuration conf) {
|
||||
builder =
|
||||
CuratorFrameworkFactory
|
||||
.builder()
|
||||
.zookeeperFactory(new ZKCuratorManager.HadoopZookeeperFactory(
|
||||
conf.get(ZK_DTSM_ZK_KERBEROS_SERVER_PRINCIPAL)))
|
||||
.aclProvider(aclProvider)
|
||||
.namespace(
|
||||
conf.get(ZK_DTSM_ZNODE_WORKING_PATH,
|
||||
|
@ -28,12 +28,16 @@
|
||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||
import org.apache.curator.framework.api.transaction.CuratorOp;
|
||||
import org.apache.curator.retry.RetryNTimes;
|
||||
import org.apache.curator.utils.ZookeeperFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.util.ZKUtil;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.client.ZKClientConfig;
|
||||
import org.apache.zookeeper.data.ACL;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.slf4j.Logger;
|
||||
@ -148,6 +152,8 @@ public void start(List<AuthInfo> authInfos) throws IOException {
|
||||
|
||||
CuratorFramework client = CuratorFrameworkFactory.builder()
|
||||
.connectString(zkHostPort)
|
||||
.zookeeperFactory(new HadoopZookeeperFactory(
|
||||
conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL)))
|
||||
.sessionTimeoutMs(zkSessionTimeout)
|
||||
.retryPolicy(retryPolicy)
|
||||
.authorization(authInfos)
|
||||
@ -428,4 +434,27 @@ public void setData(String path, byte[] data, int version)
|
||||
.forPath(path, data));
|
||||
}
|
||||
}
|
||||
|
||||
public static class HadoopZookeeperFactory implements ZookeeperFactory {
|
||||
private final String zkPrincipal;
|
||||
|
||||
public HadoopZookeeperFactory(String zkPrincipal) {
|
||||
this.zkPrincipal = zkPrincipal;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeper newZooKeeper(String connectString, int sessionTimeout,
|
||||
Watcher watcher, boolean canBeReadOnly
|
||||
) throws Exception {
|
||||
ZKClientConfig zkClientConfig = new ZKClientConfig();
|
||||
if (zkPrincipal != null) {
|
||||
LOG.info("Configuring zookeeper to use {} as the server principal",
|
||||
zkPrincipal);
|
||||
zkClientConfig.setProperty(ZKClientConfig.ZK_SASL_CLIENT_USERNAME,
|
||||
zkPrincipal);
|
||||
}
|
||||
return new ZooKeeper(connectString, sessionTimeout, watcher,
|
||||
canBeReadOnly, zkClientConfig);
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue
Block a user