aboutsummaryrefslogtreecommitdiff
path: root/yarn/src/test
diff options
context:
space:
mode:
authorSteve Loughran <stevel@hortonworks.com>2015-10-31 18:23:15 -0700
committerMarcelo Vanzin <vanzin@cloudera.com>2015-10-31 18:23:15 -0700
commit40d3c6797a3dfd037eb69b2bcd336d8544deddf5 (patch)
tree39243aae057c6d6314861579e536773e952cf6a9 /yarn/src/test
parentfc27dfbf0f8d3f96c70e27d88f7d0316c97ddb1e (diff)
downloadspark-40d3c6797a3dfd037eb69b2bcd336d8544deddf5.tar.gz
spark-40d3c6797a3dfd037eb69b2bcd336d8544deddf5.tar.bz2
spark-40d3c6797a3dfd037eb69b2bcd336d8544deddf5.zip
[SPARK-11265][YARN] YarnClient can't get tokens to talk to Hive 1.2.1 in a secure cluster
This is a fix for SPARK-11265; the introspection code to get Hive delegation tokens failing on Spark 1.5.1+, due to changes in the Hive codebase Author: Steve Loughran <stevel@hortonworks.com> Closes #9232 from steveloughran/stevel/patches/SPARK-11265-hive-tokens.
Diffstat (limited to 'yarn/src/test')
-rw-r--r--yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala29
1 files changed, 29 insertions, 0 deletions
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
index e1c67db765..9132c56a91 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala
@@ -18,10 +18,12 @@
package org.apache.spark.deploy.yarn
import java.io.{File, IOException}
+import java.lang.reflect.InvocationTargetException
import com.google.common.io.{ByteStreams, Files}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.ql.metadata.HiveException
import org.apache.hadoop.yarn.api.ApplicationConstants
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -245,4 +247,31 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
System.clearProperty("SPARK_YARN_MODE")
}
}
+
+ test("Obtain tokens For HiveMetastore") {
+ val hadoopConf = new Configuration()
+ hadoopConf.set("hive.metastore.kerberos.principal", "bob")
+ // thrift picks up on port 0 and bails out, without trying to talk to endpoint
+ hadoopConf.set("hive.metastore.uris", "http://localhost:0")
+ val util = new YarnSparkHadoopUtil
+ assertNestedHiveException(intercept[InvocationTargetException] {
+ util.obtainTokenForHiveMetastoreInner(hadoopConf, "alice")
+ })
+ // expect exception trapping code to unwind this hive-side exception
+ assertNestedHiveException(intercept[InvocationTargetException] {
+ util.obtainTokenForHiveMetastore(hadoopConf)
+ })
+ }
+
+ def assertNestedHiveException(e: InvocationTargetException): Throwable = {
+ val inner = e.getCause
+ if (inner == null) {
+ fail("No inner cause", e)
+ }
+ if (!inner.isInstanceOf[HiveException]) {
+ fail("Not a hive exception", inner)
+ }
+ inner
+ }
+
}