Skip to content

Commit 72c1d68

Browse files
yolandagaoyhuai
authored andcommitted
[SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization
On driver process start up, UserGroupInformation.loginUserFromKeytab is called with the principal and keytab passed in, and therefore static var UserGroupInfomation,loginUser is set to that principal with kerberos credentials saved in its private credential set, and all threads within the driver process are supposed to see and use this login credentials to authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, UserGroupInformation class is not shared for hive metastore clients, and instead it is loaded separately and of course not able to see the prepared kerberos login credentials in the main thread. The first proposed fix would cause other classloader conflict errors, and is not an appropriate solution. This new change does kerberos login during hive client initialization, which will make credentials ready for the particular hive client instance. yhuai Please take a look and let me know. If you are not the right person to talk to, could you point me to someone responsible for this? Author: Yu Gao <ygao@us.ibm.com> Author: gaoyu <gaoyu@gaoyu-macbookpro.roam.corp.google.com> Author: Yu Gao <crystalgaoyu@gmail.com> Closes apache#9272 from yolandagao/master.
1 parent 3e2e187 commit 72c1d68

File tree

2 files changed

+37
-4
lines changed

2 files changed

+37
-4
lines changed

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher
3939
import org.apache.ivy.plugins.repository.file.FileRepository
4040
import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, IBiblioResolver}
4141

42-
import org.apache.spark.{SparkUserAppException, SPARK_VERSION}
42+
import org.apache.spark.{SparkException, SparkUserAppException, SPARK_VERSION}
4343
import org.apache.spark.api.r.RUtils
4444
import org.apache.spark.deploy.rest._
4545
import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils}
@@ -521,8 +521,19 @@ object SparkSubmit {
521521
sysProps.put("spark.yarn.isPython", "true")
522522
}
523523
if (args.principal != null) {
524-
require(args.keytab != null, "Keytab must be specified when the keytab is specified")
525-
UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
524+
require(args.keytab != null, "Keytab must be specified when principal is specified")
525+
if (!new File(args.keytab).exists()) {
526+
throw new SparkException(s"Keytab file: ${args.keytab} does not exist")
527+
} else {
528+
// Add keytab and principal configurations in sysProps to make them available
529+
// for later use; e.g. in spark sql, the isolated class loader used to talk
530+
// to HiveMetastore will use these settings. They will be set as Java system
531+
// properties and then loaded by SparkConf
532+
sysProps.put("spark.yarn.keytab", args.keytab)
533+
sysProps.put("spark.yarn.principal", args.principal)
534+
535+
UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
536+
}
526537
}
527538
}
528539

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala

Lines changed: 23 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,10 @@ import org.apache.hadoop.hive.ql.processors._
3232
import org.apache.hadoop.hive.ql.session.SessionState
3333
import org.apache.hadoop.hive.ql.{Driver, metadata}
3434
import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader}
35+
import org.apache.hadoop.security.UserGroupInformation
3536
import org.apache.hadoop.util.VersionInfo
3637

37-
import org.apache.spark.Logging
38+
import org.apache.spark.{SparkConf, SparkException, Logging}
3839
import org.apache.spark.sql.catalyst.expressions.Expression
3940
import org.apache.spark.sql.execution.QueryExecutionException
4041
import org.apache.spark.util.{CircularBuffer, Utils}
@@ -149,6 +150,27 @@ private[hive] class ClientWrapper(
149150
val original = Thread.currentThread().getContextClassLoader
150151
// Switch to the initClassLoader.
151152
Thread.currentThread().setContextClassLoader(initClassLoader)
153+
154+
// Set up kerberos credentials for UserGroupInformation.loginUser within
155+
// current class loader
156+
// Instead of using the spark conf of the current spark context, a new
157+
// instance of SparkConf is needed for the original value of spark.yarn.keytab
158+
// and spark.yarn.principal set in SparkSubmit, as yarn.Client resets the
159+
// keytab configuration for the link name in distributed cache
160+
val sparkConf = new SparkConf
161+
if (sparkConf.contains("spark.yarn.principal") && sparkConf.contains("spark.yarn.keytab")) {
162+
val principalName = sparkConf.get("spark.yarn.principal")
163+
val keytabFileName = sparkConf.get("spark.yarn.keytab")
164+
if (!new File(keytabFileName).exists()) {
165+
throw new SparkException(s"Keytab file: ${keytabFileName}" +
166+
" specified in spark.yarn.keytab does not exist")
167+
} else {
168+
logInfo("Attempting to login to Kerberos" +
169+
s" using principal: ${principalName} and keytab: ${keytabFileName}")
170+
UserGroupInformation.loginUserFromKeytab(principalName, keytabFileName)
171+
}
172+
}
173+
152174
val ret = try {
153175
val initialConf = new HiveConf(classOf[SessionState])
154176
// HiveConf is a Hadoop Configuration, which has a field of classLoader and

0 commit comments

Comments
 (0)