This is an automated email from the ASF dual-hosted git repository. dkuzmenko pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git
commit 0723a4bf32111064ff505a1c36fcbd838a466251 Author: Michael Smith <[email protected]> AuthorDate: Thu Oct 12 17:05:47 2023 -0700 HIVE-27886: Configure Hadoop security for Kerberos when required (Michael Smith, reviewed by Attila Turoczy, Denys Kuzmenko) Configures hadoop.security.UserGroupInformation to consider Kerberos principals when collecting information if connection requires Kerberos. This allows using Kerberos with jdbc without requiring a config file. Closes #4902 --- jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java index 054400c2024..015a9f95cd6 100644 --- a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java +++ b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hive.common.auth.HiveAuthUtils; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; @@ -298,6 +299,7 @@ public class HiveConnection implements java.sql.Connection { throw new SQLException(e); } jdbcUriString = connParams.getJdbcUriString(); + LOG.debug("Establishing connection to " + jdbcUriString); // JDBC URL: jdbc:hive2://<host>:<port>/dbName;sess_var_list?hive_conf_list#hive_var_list // each list: <key1>=<val1>;<key2>=<val2> and so on // sess_var_list -> sessConfMap @@ -306,6 +308,12 @@ public class HiveConnection implements java.sql.Connection { sessConfMap = connParams.getSessionVars(); setupLoginTimeout(); if (isKerberosAuthMode()) { + // Ensure UserGroupInformation includes any authorized Kerberos principals. + LOG.debug("Configuring Kerberos mode"); + Configuration config = new Configuration(); + config.set("hadoop.security.authentication", "Kerberos"); + UserGroupInformation.setConfiguration(config); + if (isEnableCanonicalHostnameCheck()) { host = Utils.getCanonicalHostName(connParams.getHost()); } else {
