spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From yh...@apache.org
Subject spark git commit: [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization
Date Sun, 15 Nov 2015 22:54:03 GMT
Repository: spark
Updated Branches:
  refs/heads/master 3e2e1873b -> 72c1d68b4


[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.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/72c1d68b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/72c1d68b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/72c1d68b

Branch: refs/heads/master
Commit: 72c1d68b4ab6acb3f85971e10947caabb4bd846d
Parents: 3e2e187
Author: Yu Gao <ygao@us.ibm.com>
Authored: Sun Nov 15 14:53:59 2015 -0800
Committer: Yin Huai <yhuai@databricks.com>
Committed: Sun Nov 15 14:53:59 2015 -0800

----------------------------------------------------------------------
 .../org/apache/spark/deploy/SparkSubmit.scala   | 17 +++++++++++---
 .../spark/sql/hive/client/ClientWrapper.scala   | 24 +++++++++++++++++++-
 2 files changed, 37 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/72c1d68b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
----------------------------------------------------------------------
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 84ae122..09d2ec9 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)
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/72c1d68b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
index f1c2489..598ccde 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala
@@ -32,9 +32,10 @@ import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.ql.session.SessionState
 import org.apache.hadoop.hive.ql.{Driver, metadata}
 import org.apache.hadoop.hive.shims.{HadoopShims, ShimLoader}
+import org.apache.hadoop.security.UserGroupInformation
 import org.apache.hadoop.util.VersionInfo
 
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, SparkException, Logging}
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.execution.QueryExecutionException
 import org.apache.spark.util.{CircularBuffer, Utils}
@@ -149,6 +150,27 @@ private[hive] class ClientWrapper(
     val original = Thread.currentThread().getContextClassLoader
     // Switch to the initClassLoader.
     Thread.currentThread().setContextClassLoader(initClassLoader)
+
+    // Set up kerberos credentials for UserGroupInformation.loginUser within
+    // current class loader
+    // Instead of using the spark conf of the current spark context, a new
+    // instance of SparkConf is needed for the original value of spark.yarn.keytab
+    // and spark.yarn.principal set in SparkSubmit, as yarn.Client resets the
+    // keytab configuration for the link name in distributed cache
+    val sparkConf = new SparkConf
+    if (sparkConf.contains("spark.yarn.principal") && sparkConf.contains("spark.yarn.keytab"))
{
+      val principalName = sparkConf.get("spark.yarn.principal")
+      val keytabFileName = sparkConf.get("spark.yarn.keytab")
+      if (!new File(keytabFileName).exists()) {
+        throw new SparkException(s"Keytab file: ${keytabFileName}" +
+          " specified in spark.yarn.keytab does not exist")
+      } else {
+        logInfo("Attempting to login to Kerberos" +
+          s" using principal: ${principalName} and keytab: ${keytabFileName}")
+        UserGroupInformation.loginUserFromKeytab(principalName, keytabFileName)
+      }
+    }
+
     val ret = try {
       val initialConf = new HiveConf(classOf[SessionState])
       // HiveConf is a Hadoop Configuration, which has a field of classLoader and


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


Mime
View raw message