Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down