Skip to content

Commit a7755fd

Browse files
yaooqinnMarcelo Vanzin
authored andcommitted
[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 apache#20784 from yaooqinn/SPARK-23639.
1 parent 491ec11 commit a7755fd

File tree

2 files changed

+13
-4
lines changed

2 files changed

+13
-4
lines changed

core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ import org.apache.spark.internal.Logging
3636
import org.apache.spark.internal.config.KEYTAB
3737
import org.apache.spark.util.Utils
3838

39-
private[security] class HiveDelegationTokenProvider
39+
private[spark] class HiveDelegationTokenProvider
4040
extends HadoopDelegationTokenProvider with Logging {
4141

4242
override def serviceName: String = "hive"
@@ -124,9 +124,9 @@ private[security] class HiveDelegationTokenProvider
124124
val currentUser = UserGroupInformation.getCurrentUser()
125125
val realUser = Option(currentUser.getRealUser()).getOrElse(currentUser)
126126

127-
// For some reason the Scala-generated anonymous class ends up causing an
128-
// UndeclaredThrowableException, even if you annotate the method with @throws.
129-
try {
127+
// For some reason the Scala-generated anonymous class ends up causing an
128+
// UndeclaredThrowableException, even if you annotate the method with @throws.
129+
try {
130130
realUser.doAs(new PrivilegedExceptionAction[T]() {
131131
override def run(): T = fn
132132
})

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,11 +34,13 @@ import org.apache.hadoop.hive.ql.Driver
3434
import org.apache.hadoop.hive.ql.exec.Utilities
3535
import org.apache.hadoop.hive.ql.processors._
3636
import org.apache.hadoop.hive.ql.session.SessionState
37+
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
3738
import org.apache.log4j.{Level, Logger}
3839
import org.apache.thrift.transport.TSocket
3940

4041
import org.apache.spark.SparkConf
4142
import org.apache.spark.deploy.SparkHadoopUtil
43+
import org.apache.spark.deploy.security.HiveDelegationTokenProvider
4244
import org.apache.spark.internal.Logging
4345
import org.apache.spark.sql.AnalysisException
4446
import org.apache.spark.sql.hive.HiveUtils
@@ -121,6 +123,13 @@ private[hive] object SparkSQLCLIDriver extends Logging {
121123
}
122124
}
123125

126+
val tokenProvider = new HiveDelegationTokenProvider()
127+
if (tokenProvider.delegationTokensRequired(sparkConf, hadoopConf)) {
128+
val credentials = new Credentials()
129+
tokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials)
130+
UserGroupInformation.getCurrentUser.addCredentials(credentials)
131+
}
132+
124133
SessionState.start(sessionState)
125134

126135
// Clean up after we exit

0 commit comments

Comments
 (0)