Skip to content

Commit

Permalink
SPARK-5425: Use synchronised methods in system properties to create S…
Browse files Browse the repository at this point in the history
…parkConf
  • Loading branch information
jacek-lewandowski committed Jan 27, 2015
1 parent 3bb5d13 commit 685780e
Show file tree
Hide file tree
Showing 2 changed files with 28 additions and 2 deletions.
5 changes: 3 additions & 2 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -49,8 +49,9 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {

if (loadDefaults) {
// Load any spark.* system properties
for ((k, v) <- System.getProperties.asScala if k.startsWith("spark.")) {
settings(k) = v
val propNames = System.getProperties.stringPropertyNames().asScala
for (k <- propNames if k.startsWith("spark.")) {
settings(k) = System.getProperty(k)
}
}

Expand Down
25 changes: 25 additions & 0 deletions core/src/test/scala/org/apache/spark/SparkConfSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,10 @@

package org.apache.spark

import java.util.concurrent.{TimeUnit, Executors}

import scala.util.{Try, Random}

import org.scalatest.FunSuite

import org.apache.spark.util.ResetSystemProperties
Expand Down Expand Up @@ -121,4 +125,25 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro
assert(conf.get("spark.test.a.b") === "A.B")
assert(conf.get("spark.test.a.b.c") === "a.b.c")
}

test("Thread safeness - SPARK-5425") {
import scala.collection.JavaConversions._
val executor = Executors.newSingleThreadScheduledExecutor()
val sf = executor.scheduleAtFixedRate(new Runnable {
override def run(): Unit =
System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString)
}, 0, 1, TimeUnit.MILLISECONDS)

try {
val t0 = System.currentTimeMillis()
while ((System.currentTimeMillis() - t0) < 1000) {
val conf = Try(new SparkConf(loadDefaults = true))
assert(conf.isSuccess === true)
}
} finally {
executor.shutdownNow()
for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425."))
System.getProperties.remove(key)
}
}
}

0 comments on commit 685780e

Please sign in to comment.