Skip to content

Commit 450528f

Browse files
committed
HADOOP-18139: RBF: Allow configuration of zookeeper server principals.
Cherry-picked from 8b3377a9 by Owen O'Malley Cherry-picked from ccbdc3cf by Owen O'Malley
1 parent a1319e2 commit 450528f

File tree

3 files changed

+36
-0
lines changed

3 files changed

+36
-0
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -396,6 +396,8 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
396396
public static final String ZK_ACL_DEFAULT = "world:anyone:rwcda";
397397
/** Authentication for the ZooKeeper ensemble. */
398398
public static final String ZK_AUTH = ZK_PREFIX + "auth";
399+
/** Principal name for zookeeper servers. */
400+
public static final String ZK_SERVER_PRINCIPAL = ZK_PREFIX + "server.principal";
399401

400402
/** Address of the ZooKeeper ensemble. */
401403
public static final String ZK_ADDRESS = ZK_PREFIX + "address";

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,9 +52,11 @@
5252
import org.apache.hadoop.classification.InterfaceAudience.Private;
5353
import org.apache.hadoop.classification.InterfaceStability.Unstable;
5454
import org.apache.hadoop.conf.Configuration;
55+
import org.apache.hadoop.fs.CommonConfigurationKeys;
5556
import org.apache.hadoop.security.SecurityUtil;
5657
import org.apache.hadoop.security.token.Token;
5758
import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
59+
import org.apache.hadoop.util.curator.ZKCuratorManager;
5860
import org.apache.zookeeper.CreateMode;
5961
import org.apache.zookeeper.KeeperException;
6062
import org.apache.zookeeper.KeeperException.NoNodeException;
@@ -98,6 +100,8 @@ public abstract class ZKDelegationTokenSecretManager<TokenIdent extends Abstract
98100
+ "kerberos.keytab";
99101
public static final String ZK_DTSM_ZK_KERBEROS_PRINCIPAL = ZK_CONF_PREFIX
100102
+ "kerberos.principal";
103+
public static final String ZK_DTSM_ZK_KERBEROS_SERVER_PRINCIPAL = ZK_CONF_PREFIX
104+
+ "kerberos.server.principal";
101105
public static final String ZK_DTSM_TOKEN_SEQNUM_BATCH_SIZE = ZK_CONF_PREFIX
102106
+ "token.seqnum.batch.size";
103107

@@ -199,6 +203,8 @@ public ZKDelegationTokenSecretManager(Configuration conf) {
199203
builder =
200204
CuratorFrameworkFactory
201205
.builder()
206+
.zookeeperFactory(new ZKCuratorManager.HadoopZookeeperFactory(
207+
conf.get(ZK_DTSM_ZK_KERBEROS_SERVER_PRINCIPAL)))
202208
.aclProvider(aclProvider)
203209
.namespace(
204210
conf.get(ZK_DTSM_ZNODE_WORKING_PATH,

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java

Lines changed: 28 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,12 +28,16 @@
2828
import org.apache.curator.framework.CuratorFrameworkFactory;
2929
import org.apache.curator.framework.api.transaction.CuratorOp;
3030
import org.apache.curator.retry.RetryNTimes;
31+
import org.apache.curator.utils.ZookeeperFactory;
3132
import org.apache.hadoop.classification.InterfaceAudience;
3233
import org.apache.hadoop.conf.Configuration;
3334
import org.apache.hadoop.fs.CommonConfigurationKeys;
3435
import org.apache.hadoop.security.SecurityUtil;
3536
import org.apache.hadoop.util.ZKUtil;
3637
import org.apache.zookeeper.CreateMode;
38+
import org.apache.zookeeper.Watcher;
39+
import org.apache.zookeeper.ZooKeeper;
40+
import org.apache.zookeeper.client.ZKClientConfig;
3741
import org.apache.zookeeper.data.ACL;
3842
import org.apache.zookeeper.data.Stat;
3943
import org.slf4j.Logger;
@@ -148,6 +152,8 @@ public void start(List<AuthInfo> authInfos) throws IOException {
148152

149153
CuratorFramework client = CuratorFrameworkFactory.builder()
150154
.connectString(zkHostPort)
155+
.zookeeperFactory(new HadoopZookeeperFactory(
156+
conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL)))
151157
.sessionTimeoutMs(zkSessionTimeout)
152158
.retryPolicy(retryPolicy)
153159
.authorization(authInfos)
@@ -428,4 +434,26 @@ public void setData(String path, byte[] data, int version)
428434
.forPath(path, data));
429435
}
430436
}
437+
438+
public static class HadoopZookeeperFactory implements ZookeeperFactory {
439+
private final String zkPrincipal;
440+
441+
public HadoopZookeeperFactory(String zkPrincipal) {
442+
this.zkPrincipal = zkPrincipal;
443+
}
444+
445+
@Override
446+
public ZooKeeper newZooKeeper(String connectString, int sessionTimeout,
447+
Watcher watcher, boolean canBeReadOnly
448+
) throws Exception {
449+
ZKClientConfig zkClientConfig = new ZKClientConfig();
450+
if (zkPrincipal != null) {
451+
LOG.info("Configuring zookeeper client to use {}", zkPrincipal);
452+
zkClientConfig.setProperty(ZKClientConfig.ZK_SASL_CLIENT_USERNAME,
453+
zkPrincipal);
454+
}
455+
return new ZooKeeper(connectString, sessionTimeout, watcher,
456+
canBeReadOnly, zkClientConfig);
457+
}
458+
}
431459
}

0 commit comments

Comments
 (0)