diff --git a/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java b/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java index 6ab722b73716..00ef4785a70d 100644 --- a/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java +++ b/jdbc/src/java/org/apache/hive/jdbc/ZooKeeperHiveClientHelper.java @@ -208,7 +208,7 @@ static void configureConnParams(JdbcConnectionParams connParams) throws ZooKeepe try (CuratorFramework zooKeeperClient = getZkClient(connParams)) { final List serverHosts = getServerHosts(connParams, zooKeeperClient); if (serverHosts.isEmpty()) { - throw new ZooKeeperHiveClientException("No more HiveServer2 URIs from ZooKeeper to attempt"); + throw new ZooKeeperHiveClientException("Empty HiveServer2 URIs found from ZooKeeper to attempt"); } // Pick a server node randomly final String serverNode = serverHosts.get(ThreadLocalRandom.current().nextInt(serverHosts.size())); diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java index c145283c1206..ced00eeb1008 100644 --- a/service/src/java/org/apache/hive/service/server/HiveServer2.java +++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java @@ -99,6 +99,7 @@ import org.apache.hadoop.hive.ql.txn.compactor.Worker; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.common.IPStackUtils; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.ExitUtil; import org.apache.hive.common.util.HiveStringUtils; import org.apache.hive.common.util.HiveVersionInfo; @@ -606,7 +607,7 @@ public static boolean isAllTransportMode(HiveConf hiveConf) { private ACLProvider getACLProvider(HiveConf hiveConf) { final boolean isSecure = - AuthType.isKerberosAuthMode(hiveConf) && + UserGroupInformation.isSecurityEnabled() && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ZOOKEEPER_USE_KERBEROS); return new ACLProvider() { @@ -683,7 +684,7 @@ private void addConfsToPublish(HiveConf hiveConf, Map confsToPub */ private static void setUpZooKeeperAuth(HiveConf hiveConf) { try { - if (AuthType.isKerberosAuthMode(hiveConf) && + if (UserGroupInformation.isSecurityEnabled() && StringUtils.isNotEmpty(HiveConf.getVar(hiveConf, HiveConf.ConfVars.HIVE_ZOOKEEPER_QUORUM)) && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_ZOOKEEPER_USE_KERBEROS)) { // Install the JAAS Configuration for the runtime