From cf33164857530a38aa3edc99a9051a47aae9dede Mon Sep 17 00:00:00 2001 From: hchaverr Date: Thu, 16 Jun 2022 10:35:26 -0700 Subject: [PATCH] HDFS-16591. Setup JaasConfiguration in ZKCuratorManager when SASL is enabled Fixes #4447 Signed-off-by: Owen O'Malley --- .../util/JaasConfiguration.java | 77 +++++++++++++++++++ .../util/ZKSignerSecretProvider.java | 61 --------------- .../util/TestJaasConfiguration.java | 4 +- .../hadoop/fs/CommonConfigurationKeys.java | 4 + .../ZKDelegationTokenSecretManager.java | 67 +--------------- .../hadoop/util/curator/ZKCuratorManager.java | 39 +++++++++- .../util/curator/TestZKCuratorManager.java | 51 ++++++++++++ .../client/impl/zk/RegistrySecurity.java | 63 +-------------- 8 files changed, 174 insertions(+), 192 deletions(-) create mode 100644 hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java new file mode 100644 index 0000000000..d03e630ced --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java @@ -0,0 +1,77 @@ +/** + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. See accompanying LICENSE file. + */ +package org.apache.hadoop.security.authentication.util; + +import java.util.HashMap; +import java.util.Map; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; + + +/** + * Creates a programmatic version of a jaas.conf file. This can be used + * instead of writing a jaas.conf file and setting the system property, + * "java.security.auth.login.config", to point to that file. It is meant to be + * used for connecting to ZooKeeper. + */ +public class JaasConfiguration extends Configuration { + + private final javax.security.auth.login.Configuration baseConfig = + javax.security.auth.login.Configuration.getConfiguration(); + private final AppConfigurationEntry[] entry; + private final String entryName; + + /** + * Add an entry to the jaas configuration with the passed in name, + * principal, and keytab. The other necessary options will be set for you. + * + * @param entryName The name of the entry (e.g. "Client") + * @param principal The principal of the user + * @param keytab The location of the keytab + */ + public JaasConfiguration(String entryName, String principal, String keytab) { + this.entryName = entryName; + Map options = new HashMap<>(); + options.put("keyTab", keytab); + options.put("principal", principal); + options.put("useKeyTab", "true"); + options.put("storeKey", "true"); + options.put("useTicketCache", "false"); + options.put("refreshKrb5Config", "true"); + String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); + if ("true".equalsIgnoreCase(jaasEnvVar)) { + options.put("debug", "true"); + } + entry = new AppConfigurationEntry[]{ + new AppConfigurationEntry(getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options)}; + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) { + return (entryName.equals(name)) ? entry : ((baseConfig != null) + ? baseConfig.getAppConfigurationEntry(name) : null); + } + + private String getKrb5LoginModuleName() { + String krb5LoginModuleName; + if (System.getProperty("java.vendor").contains("IBM")) { + krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; + } else { + krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; + } + return krb5LoginModuleName; + } +} diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java index 374f4a5665..53b6f4d239 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java @@ -17,12 +17,9 @@ import java.nio.ByteBuffer; import java.security.SecureRandom; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.Random; -import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; import javax.servlet.ServletContext; import org.apache.curator.RetryPolicy; @@ -429,62 +426,4 @@ public List getAclForPath(String path) { return saslACL; } } - - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName The name of the entry (e.g. "Client") - * @param principal The principal of the user - * @param keytab The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[]{ - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options)}; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } } diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java index 2b70135800..5de4122471 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java @@ -32,8 +32,8 @@ public void test() throws Exception { krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; } - ZKSignerSecretProvider.JaasConfiguration jConf = - new ZKSignerSecretProvider.JaasConfiguration("foo", "foo/localhost", + JaasConfiguration jConf = + new JaasConfiguration("foo", "foo/localhost", "/some/location/foo.keytab"); AppConfigurationEntry[] entries = jConf.getAppConfigurationEntry("bar"); Assert.assertNull(entries); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index db82498b3c..16144206ee 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -401,6 +401,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { 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"; + /** Kerberos principal name for zookeeper connection. */ + public static final String ZK_KERBEROS_PRINCIPAL = ZK_PREFIX + "kerberos.principal"; + /** Kerberos keytab for zookeeper connection. */ + public static final String ZK_KERBEROS_KEYTAB = ZK_PREFIX + "kerberos.keytab"; /** Address of the ZooKeeper ensemble. */ public static final String ZK_ADDRESS = ZK_PREFIX + "address"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 452565676a..d0630e38b4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -25,14 +25,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; -import javax.security.auth.login.AppConfigurationEntry; - import org.apache.curator.ensemble.fixed.FixedEnsembleProvider; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -52,6 +48,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; import static org.apache.hadoop.util.Time.now; @@ -251,68 +248,6 @@ private String setJaasConfiguration(Configuration config) throws Exception { return principal.split("[/@]")[0]; } - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends - javax.security.auth.login.Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName - * The name of the entry (e.g. "Client") - * @param principal - * The principal of the user - * @param keytab - * The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[] { - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options) }; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } - @Override public void startThreads() throws IOException { if (!isExternalClient) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java index 7d35977e5b..c11b868386 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.ZKUtil; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.Watcher; @@ -159,7 +160,9 @@ public void start(List authInfos) throws IOException { CuratorFramework client = CuratorFrameworkFactory.builder() .connectString(zkHostPort) .zookeeperFactory(new HadoopZookeeperFactory( - conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL))) + conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL), + conf.get(CommonConfigurationKeys.ZK_KERBEROS_PRINCIPAL), + conf.get(CommonConfigurationKeys.ZK_KERBEROS_KEYTAB))) .sessionTimeoutMs(zkSessionTimeout) .retryPolicy(retryPolicy) .authorization(authInfos) @@ -445,10 +448,20 @@ public void setData(String path, byte[] data, int version) } public static class HadoopZookeeperFactory implements ZookeeperFactory { + public final static String JAAS_CLIENT_ENTRY = "HadoopZookeeperClient"; private final String zkPrincipal; + private final String kerberosPrincipal; + private final String kerberosKeytab; public HadoopZookeeperFactory(String zkPrincipal) { + this(zkPrincipal, null, null); + } + + public HadoopZookeeperFactory(String zkPrincipal, String kerberosPrincipal, + String kerberosKeytab) { this.zkPrincipal = zkPrincipal; + this.kerberosPrincipal = kerberosPrincipal; + this.kerberosKeytab = kerberosKeytab; } @Override @@ -462,8 +475,32 @@ public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, zkClientConfig.setProperty(ZKClientConfig.ZK_SASL_CLIENT_USERNAME, zkPrincipal); } + if (zkClientConfig.isSaslClientEnabled() && !isJaasConfigurationSet(zkClientConfig)) { + setJaasConfiguration(zkClientConfig); + } return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly, zkClientConfig); } + + private boolean isJaasConfigurationSet(ZKClientConfig zkClientConfig) { + String clientConfig = zkClientConfig.getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, + ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT); + return javax.security.auth.login.Configuration.getConfiguration() + .getAppConfigurationEntry(clientConfig) != null; + } + + private void setJaasConfiguration(ZKClientConfig zkClientConfig) throws IOException { + if (kerberosPrincipal == null || kerberosKeytab == null) { + LOG.warn("JaasConfiguration has not been set since kerberos principal " + + "or keytab is not specified"); + return; + } + + String principal = SecurityUtil.getServerPrincipal(kerberosPrincipal, ""); + JaasConfiguration jconf = new JaasConfiguration(JAAS_CLIENT_ENTRY, principal, + kerberosKeytab); + javax.security.auth.login.Configuration.setConfiguration(jconf); + zkClientConfig.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, JAAS_CLIENT_ENTRY); + } } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java index a2156ee6d9..fd15a0c2b1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java @@ -25,11 +25,15 @@ import java.util.Arrays; import java.util.List; +import javax.security.auth.login.AppConfigurationEntry; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.ZKUtil; import org.apache.zookeeper.CreateMode; +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.junit.After; @@ -154,4 +158,51 @@ public void testTransaction() throws Exception { assertFalse(curator.exists(node2)); assertTrue(Arrays.equals(setData, curator.getData(node1))); } + + @Test + public void testJaasConfiguration() throws Exception { + // Validate that HadoopZooKeeperFactory will set ZKConfig with given principals + ZKCuratorManager.HadoopZookeeperFactory factory1 = + new ZKCuratorManager.HadoopZookeeperFactory("foo1", "bar1", "bar1.keytab"); + ZooKeeper zk1 = factory1.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(ZKCuratorManager.HadoopZookeeperFactory.JAAS_CLIENT_ENTRY, + "bar1", "bar1.keytab", zk1); + + // Validate that a new HadoopZooKeeperFactory will use the new principals + ZKCuratorManager.HadoopZookeeperFactory factory2 = + new ZKCuratorManager.HadoopZookeeperFactory("foo2", "bar2", "bar2.keytab"); + ZooKeeper zk2 = factory2.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(ZKCuratorManager.HadoopZookeeperFactory.JAAS_CLIENT_ENTRY, + "bar2", "bar2.keytab", zk2); + + try { + // Setting global configuration + String testClientConfig = "TestClientConfig"; + JaasConfiguration jconf = new JaasConfiguration(testClientConfig, "test", "test.keytab"); + javax.security.auth.login.Configuration.setConfiguration(jconf); + System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, testClientConfig); + + // Validate that a new HadoopZooKeeperFactory will use the global principals + ZKCuratorManager.HadoopZookeeperFactory factory3 = + new ZKCuratorManager.HadoopZookeeperFactory("foo3", "bar3", "bar3.keytab"); + ZooKeeper zk3 = factory3.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(testClientConfig, "test", "test.keytab", zk3); + } finally { + // Remove global configuration + System.clearProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY); + } + } + + private void validateJaasConfiguration(String clientConfig, String principal, String keytab, + ZooKeeper zk) { + assertEquals("Validate that expected clientConfig is set in ZK config", clientConfig, + zk.getClientConfig().getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY)); + + AppConfigurationEntry[] entries = javax.security.auth.login.Configuration.getConfiguration() + .getAppConfigurationEntry(clientConfig); + assertEquals("Validate that expected principal is set in Jaas config", principal, + entries[0].getOptions().get("principal")); + assertEquals("Validate that expected keytab is set in Jaas config", keytab, + entries[0].getOptions().get("keyTab")); + } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java index e500ba6617..d48920a222 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java @@ -18,11 +18,11 @@ package org.apache.hadoop.registry.client.impl.zk; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Splitter; import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.util.KerberosUtil; @@ -45,11 +45,9 @@ import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.ListIterator; import java.util.Locale; -import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import static org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions.*; @@ -797,65 +795,6 @@ public void setKerberosPrincipalAndKeytab(String principal, String keytab) { this.keytab = keytab; } - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends - javax.security.auth.login.Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName The name of the entry (e.g. "Client") - * @param principal The principal of the user - * @param keytab The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[]{ - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options)}; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } - /** * Set the client properties. This forces the ZK client into * failing if it can't auth.