aboutsummaryrefslogtreecommitdiff
path: root/yarn
diff options
context:
space:
mode:
authorMarcelo Vanzin <vanzin@cloudera.com>2015-11-01 15:57:42 -0800
committerMarcelo Vanzin <vanzin@cloudera.com>2015-11-01 15:57:42 -0800
commitf8d93edec82eedab59d50aec06ca2de7e4cf14f6 (patch)
tree47a943e70e2c794c61e4905d4850cfa88e08450f /yarn
parentcf04fdfe71abc395163a625cc1f99ec5e54cc07e (diff)
downloadspark-f8d93edec82eedab59d50aec06ca2de7e4cf14f6.tar.gz
spark-f8d93edec82eedab59d50aec06ca2de7e4cf14f6.tar.bz2
spark-f8d93edec82eedab59d50aec06ca2de7e4cf14f6.zip
[SPARK-11073][CORE][YARN] Remove akka dependency in secret key generation.
Use standard JDK APIs for that (with a little help from Guava). Most of the changes here are in test code, since there were no tests specific to that part of the code. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9257 from vanzin/SPARK-11073.
Diffstat (limited to 'yarn')
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala3
-rw-r--r--yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala32
2 files changed, 33 insertions, 2 deletions
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index 5924daf3ec..561ad79ee0 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -18,6 +18,7 @@
package org.apache.spark.deploy.yarn
import java.io.File
+import java.nio.charset.StandardCharsets.UTF_8
import java.util.regex.Matcher
import java.util.regex.Pattern
@@ -81,7 +82,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil {
override def addSecretKeyToUserCredentials(key: String, secret: String) {
val creds = new Credentials()
- creds.addSecretKey(new Text(key), secret.getBytes("utf-8"))
+ creds.addSecretKey(new Text(key), secret.getBytes(UTF_8))
addCurrentUserCredentials(creds)
}
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 9132c56a91..a70e66d39a 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
@@ -24,8 +24,10 @@ 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.io.Text
import org.apache.hadoop.yarn.api.ApplicationConstants
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.scalatest.Matchers
@@ -263,7 +265,7 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
})
}
- def assertNestedHiveException(e: InvocationTargetException): Throwable = {
+ private def assertNestedHiveException(e: InvocationTargetException): Throwable = {
val inner = e.getCause
if (inner == null) {
fail("No inner cause", e)
@@ -274,4 +276,32 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging
inner
}
+ // This test needs to live here because it depends on isYarnMode returning true, which can only
+ // happen in the YARN module.
+ test("security manager token generation") {
+ try {
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val initial = SparkHadoopUtil.get
+ .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+ assert(initial === null || initial.length === 0)
+
+ val conf = new SparkConf()
+ .set(SecurityManager.SPARK_AUTH_CONF, "true")
+ .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused")
+ val sm = new SecurityManager(conf)
+
+ val generated = SparkHadoopUtil.get
+ .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY)
+ assert(generated != null)
+ val genString = new Text(generated).toString()
+ assert(genString != "unused")
+ assert(sm.getSecretKey() === genString)
+ } finally {
+ // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret
+ // to an empty string.
+ SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "")
+ System.clearProperty("SPARK_YARN_MODE")
+ }
+ }
+
}