aboutsummaryrefslogtreecommitdiff
path: root/core/src/test
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 /core/src/test
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 'core/src/test')
-rw-r--r--core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala23
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala13
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala28
-rw-r--r--core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala16
-rw-r--r--core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala34
5 files changed, 64 insertions, 50 deletions
diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
index f29160d834..26b95c0678 100644
--- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark
import java.io.File
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{SparkConfWithEnv, Utils}
class SecurityManagerSuite extends SparkFunSuite {
@@ -223,5 +223,26 @@ class SecurityManagerSuite extends SparkFunSuite {
assert(securityManager.hostnameVerifier.isDefined === false)
}
+ test("missing secret authentication key") {
+ val conf = new SparkConf().set("spark.authenticate", "true")
+ intercept[IllegalArgumentException] {
+ new SecurityManager(conf)
+ }
+ }
+
+ test("secret authentication key") {
+ val key = "very secret key"
+ val conf = new SparkConf()
+ .set(SecurityManager.SPARK_AUTH_CONF, "true")
+ .set(SecurityManager.SPARK_AUTH_SECRET_CONF, key)
+ assert(key === new SecurityManager(conf).getSecretKey())
+
+ val keyFromEnv = "very secret key from env"
+ val conf2 = new SparkConfWithEnv(Map(SecurityManager.ENV_AUTH_SECRET -> keyFromEnv))
+ .set(SecurityManager.SPARK_AUTH_CONF, "true")
+ .set(SecurityManager.SPARK_AUTH_SECRET_CONF, key)
+ assert(keyFromEnv === new SecurityManager(conf2).getSecretKey())
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
index 86eb41dd7e..8dd31b4b6f 100644
--- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala
@@ -25,6 +25,7 @@ import scala.io.Source
import org.apache.spark.scheduler.cluster.ExecutorInfo
import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener}
import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
+import org.apache.spark.util.SparkConfWithEnv
class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext {
@@ -53,17 +54,7 @@ class LogUrlsStandaloneSuite extends SparkFunSuite with LocalSparkContext {
test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") {
val SPARK_PUBLIC_DNS = "public_dns"
- class MySparkConf extends SparkConf(false) {
- override def getenv(name: String): String = {
- if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS
- else super.getenv(name)
- }
-
- override def clone: SparkConf = {
- new MySparkConf().setAll(getAll)
- }
- }
- val conf = new MySparkConf().set(
+ val conf = new SparkConfWithEnv(Map("SPARK_PUBLIC_DNS" -> SPARK_PUBLIC_DNS)).set(
"spark.extraListeners", classOf[SaveExecutorInfo].getName)
sc = new SparkContext("local-cluster[2,1,1024]", "test", conf)
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala
index 15f7ca4a6d..637e78fda0 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala
@@ -19,7 +19,7 @@
package org.apache.spark.deploy.worker
import org.apache.spark.{SparkConf, SparkFunSuite}
-
+import org.apache.spark.util.SparkConfWithEnv
class WorkerArgumentsTest extends SparkFunSuite {
@@ -34,18 +34,7 @@ class WorkerArgumentsTest extends SparkFunSuite {
test("Memory can't be set to 0 when SPARK_WORKER_MEMORY env property leaves off M or G") {
val args = Array("spark://localhost:0000 ")
-
- class MySparkConf extends SparkConf(false) {
- override def getenv(name: String): String = {
- if (name == "SPARK_WORKER_MEMORY") "50000"
- else super.getenv(name)
- }
-
- override def clone: SparkConf = {
- new MySparkConf().setAll(getAll)
- }
- }
- val conf = new MySparkConf()
+ val conf = new SparkConfWithEnv(Map("SPARK_WORKER_MEMORY" -> "50000"))
intercept[IllegalStateException] {
new WorkerArguments(args, conf)
}
@@ -53,18 +42,7 @@ class WorkerArgumentsTest extends SparkFunSuite {
test("Memory correctly set when SPARK_WORKER_MEMORY env property appends G") {
val args = Array("spark://localhost:0000 ")
-
- class MySparkConf extends SparkConf(false) {
- override def getenv(name: String): String = {
- if (name == "SPARK_WORKER_MEMORY") "5G"
- else super.getenv(name)
- }
-
- override def clone: SparkConf = {
- new MySparkConf().setAll(getAll)
- }
- }
- val conf = new MySparkConf()
+ val conf = new SparkConfWithEnv(Map("SPARK_WORKER_MEMORY" -> "5G"))
val workerArgs = new WorkerArguments(args, conf)
assert(workerArgs.memory === 5120)
}
diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
index ac6fec56bb..cc50289c7b 100644
--- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.util.Utils
import org.scalatest.BeforeAndAfter
import org.apache.spark.{SparkConf, SparkFunSuite}
-
+import org.apache.spark.util.SparkConfWithEnv
/**
* Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options.
@@ -45,20 +45,10 @@ class LocalDirsSuite extends SparkFunSuite with BeforeAndAfter {
test("SPARK_LOCAL_DIRS override also affects driver") {
// Regression test for SPARK-2975
assert(!new File("/NONEXISTENT_DIR").exists())
- // SPARK_LOCAL_DIRS is a valid directory:
- class MySparkConf extends SparkConf(false) {
- override def getenv(name: String): String = {
- if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir")
- else super.getenv(name)
- }
-
- override def clone: SparkConf = {
- new MySparkConf().setAll(getAll)
- }
- }
// spark.local.dir only contains invalid directories, but that's not a problem since
// SPARK_LOCAL_DIRS will override it on both the driver and workers:
- val conf = new MySparkConf().set("spark.local.dir", "/NONEXISTENT_PATH")
+ val conf = new SparkConfWithEnv(Map("SPARK_LOCAL_DIRS" -> System.getProperty("java.io.tmpdir")))
+ .set("spark.local.dir", "/NONEXISTENT_PATH")
assert(new File(Utils.getLocalDir(conf)).exists())
}
diff --git a/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala b/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala
new file mode 100644
index 0000000000..ddd5edf4f7
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import org.apache.spark.SparkConf
+
+/**
+ * Customized SparkConf that allows env variables to be overridden.
+ */
+class SparkConfWithEnv(env: Map[String, String]) extends SparkConf(false) {
+ override def getenv(name: String): String = {
+ env.get(name).getOrElse(super.getenv(name))
+ }
+
+ override def clone: SparkConf = {
+ new SparkConfWithEnv(env).setAll(getAll)
+ }
+
+}