diff options
author | Yu Gao <ygao@us.ibm.com> | 2015-11-15 14:53:59 -0800 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2015-11-15 14:53:59 -0800 |
commit | 72c1d68b4ab6acb3f85971e10947caabb4bd846d (patch) | |
tree | 337e24a65fa1c455fa8b019db9fb8b118e4b126c /core/src/main/scala/org | |
parent | 3e2e1873b2762d07e49de8f9ea709bf3fa2d171c (diff) | |
download | spark-72c1d68b4ab6acb3f85971e10947caabb4bd846d.tar.gz spark-72c1d68b4ab6acb3f85971e10947caabb4bd846d.tar.bz2 spark-72c1d68b4ab6acb3f85971e10947caabb4bd846d.zip |
[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 #9272 from yolandagao/master.
Diffstat (limited to 'core/src/main/scala/org')
-rw-r--r-- | core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 17 |
1 files changed, 14 insertions, 3 deletions
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 84ae122f44..09d2ec90c9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -39,7 +39,7 @@ import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.repository.file.FileRepository import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, IBiblioResolver} -import org.apache.spark.{SparkUserAppException, SPARK_VERSION} +import org.apache.spark.{SparkException, SparkUserAppException, SPARK_VERSION} import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} @@ -521,8 +521,19 @@ object SparkSubmit { sysProps.put("spark.yarn.isPython", "true") } if (args.principal != null) { - require(args.keytab != null, "Keytab must be specified when the keytab is specified") - UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) + require(args.keytab != null, "Keytab must be specified when principal is specified") + if (!new File(args.keytab).exists()) { + throw new SparkException(s"Keytab file: ${args.keytab} does not exist") + } else { + // Add keytab and principal configurations in sysProps to make them available + // for later use; e.g. in spark sql, the isolated class loader used to talk + // to HiveMetastore will use these settings. They will be set as Java system + // properties and then loaded by SparkConf + sysProps.put("spark.yarn.keytab", args.keytab) + sysProps.put("spark.yarn.principal", args.principal) + + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) + } } } |