From 55109d9f7aa0443dff53877e84b9705ef3566067 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 17 Mar 2017 17:16:17 +0800 Subject: [PATCH] fix proxy ugi could not get tgt to cause metastore connecting problem --- .../sql/hive/client/HiveClientImpl.scala | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 5c0e2f6ec494..3c716ce7302d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive.client import java.io.{File, PrintStream} +import java.lang.reflect.UndeclaredThrowableException +import java.security.PrivilegedExceptionAction import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -27,7 +29,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Database => HiveDatabase} import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} @@ -189,7 +191,7 @@ private[hive] class HiveClientImpl( if (clientLoader.cachedHive != null) { Hive.set(clientLoader.cachedHive.asInstanceOf[Hive]) } - SessionState.start(state) + doAsRealUser(SessionState.start(state)) state.out = new PrintStream(outputBuffer, true, "UTF-8") state.err = new PrintStream(outputBuffer, true, "UTF-8") state @@ -895,4 +897,24 @@ private[hive] class HiveClientImpl( parameters = if (hp.getParameters() != null) hp.getParameters().asScala.toMap else Map.empty) } + + + /** + * Run some code as the real logged in user (which may differ from the current user, for + * example, when using proxying). + */ + private def doAsRealUser[T](fn: => T): T = { + 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 { + realUser.doAs(new PrivilegedExceptionAction[T]() { + override def run(): T = fn + }) + } catch { + case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e) + } + } }