Repository: spark
Updated Branches:
  refs/heads/master 491ec114f -> a7755fd8c


[SPARK-23639][SQL] Obtain token before init metastore client in SparkSQL CLI

## What changes were proposed in this pull request?

In SparkSQLCLI, SessionState generates before SparkContext instantiating. When 
we use --proxy-user to impersonate, it's unable to initializing a metastore 
client to talk to the secured metastore for no kerberos ticket.

This PR use real user ugi to obtain token for owner before talking to 
kerberized metastore.

## How was this patch tested?

Manually verified with kerberized hive metasotre / hdfs.

Author: Kent Yao <[email protected]>

Closes #20784 from yaooqinn/SPARK-23639.


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

Branch: refs/heads/master
Commit: a7755fd8ce2f022118b9827aaac7d5d59f0f297a
Parents: 491ec11
Author: Kent Yao <[email protected]>
Authored: Thu Mar 29 10:46:28 2018 -0700
Committer: Marcelo Vanzin <[email protected]>
Committed: Thu Mar 29 10:46:28 2018 -0700

----------------------------------------------------------------------
 .../spark/deploy/security/HiveDelegationTokenProvider.scala | 8 ++++----
 .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala     | 9 +++++++++
 2 files changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a7755fd8/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
 
b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
index ece5ce7..7249eb8 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
@@ -36,7 +36,7 @@ import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.KEYTAB
 import org.apache.spark.util.Utils
 
-private[security] class HiveDelegationTokenProvider
+private[spark] class HiveDelegationTokenProvider
     extends HadoopDelegationTokenProvider with Logging {
 
   override def serviceName: String = "hive"
@@ -124,9 +124,9 @@ private[security] class HiveDelegationTokenProvider
     val currentUser = UserGroupInformation.getCurrentUser()
     val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser)
 
-   // For some reason the Scala-generated anonymous class ends up causing an
-   // UndeclaredThrowableException, even if you annotate the method with 
@throws.
-   try {
+    // For some reason the Scala-generated anonymous class ends up causing an
+    // UndeclaredThrowableException, even if you annotate the method with 
@throws.
+    try {
       realUser.doAs(new PrivilegedExceptionAction[T]() {
         override def run(): T = fn
       })

http://git-wip-us.apache.org/repos/asf/spark/blob/a7755fd8/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index 832a15d..084f820 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -34,11 +34,13 @@ import org.apache.hadoop.hive.ql.Driver
 import org.apache.hadoop.hive.ql.exec.Utilities
 import org.apache.hadoop.hive.ql.processors._
 import org.apache.hadoop.hive.ql.session.SessionState
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
 import org.apache.log4j.{Level, Logger}
 import org.apache.thrift.transport.TSocket
 
 import org.apache.spark.SparkConf
 import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.security.HiveDelegationTokenProvider
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.hive.HiveUtils
@@ -121,6 +123,13 @@ private[hive] object SparkSQLCLIDriver extends Logging {
       }
     }
 
+    val tokenProvider = new HiveDelegationTokenProvider()
+    if (tokenProvider.delegationTokensRequired(sparkConf, hadoopConf)) {
+      val credentials = new Credentials()
+      tokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials)
+      UserGroupInformation.getCurrentUser.addCredentials(credentials)
+    }
+
     SessionState.start(sessionState)
 
     // Clean up after we exit


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to