From af15c82bfe2c3f73142b8f310784a0e85841539d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 10:49:03 -0700 Subject: [PATCH 01/24] Revert "SPARK-1786: Edge Partition Serialization" This reverts commit a6b02fb7486356493474c7f42bb714c9cce215ca. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 +++------ .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...Map.scala => PrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ------------------ 11 files changed, 23 insertions(+), 44 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{GraphXPrimitiveKeyOpenHashMap.scala => PrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e832..d295d0127ac72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,9 +24,6 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.OpenHashSet - /** * Registers GraphX classes with Kryo for improved performance. @@ -46,8 +43,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) - kryo.register(classOf[OpenHashSet[Int]]) - kryo.register(classOf[OpenHashSet[Long]]) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index a5c9cd1f8b4e6..871e81f8d245c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - val srcIds: Array[VertexId] = null, - val dstIds: Array[VertexId] = null, - val data: Array[ED] = null, - val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, - val vertices: VertexPartition[VD] = null, - val activeSet: Option[VertexSet] = None + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 4520beb991515..ecb49bef42e45 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + val index = new PrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 56f79a7097fce..ebb0b9418d65d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index d02e9238adba5..927e32ad0f448 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..f4e221d4e05ae 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bdab..f1d174720a1ba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 34939b24440aa..8d9e0204d27f2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index a4f769b294010..21ff615feca6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala index 57b01b6f2e1fb..7b02e2ed1a9cb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 28fd112f2b124..d2e0c01bc35ef 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,9 +22,6 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer - import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -123,19 +120,4 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } - - test("Kryo serialization") { - val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) - val a: EdgePartition[Int, Int] = makeEdgePartition(aList) - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - val s = new KryoSerializer(conf).newInstance() - val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) - assert(aSer.srcIds.toList === a.srcIds.toList) - assert(aSer.dstIds.toList === a.dstIds.toList) - assert(aSer.data.toList === a.data.toList) - assert(aSer.index != null) - assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) - } } From 3af1f386439cdddd42e545ad63d089f4dfdf9f8a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 12 May 2014 11:08:52 -0700 Subject: [PATCH 02/24] SPARK-1772 Stop catching Throwable, let Executors die The main issue this patch fixes is [SPARK-1772](https://issues.apache.org/jira/browse/SPARK-1772), in which Executors may not die when fatal exceptions (e.g., OOM) are thrown. This patch causes Executors to delegate to the ExecutorUncaughtExceptionHandler when a fatal exception is thrown. This patch also continues the fight in the neverending war against `case t: Throwable =>`, by only catching Exceptions in many places, and adding a wrapper for Threads and Runnables to make sure any uncaught exceptions are at least printed to the logs. It also turns out that it is unlikely that the IndestructibleActorSystem actually works, given testing ([here](https://gist.github.com/aarondav/ca1f0cdcd50727f89c0d)). The uncaughtExceptionHandler is not called from the places that we expected it would be. [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620) deals with part of this issue, but refactoring our Actor Systems to ensure that exceptions are dealt with properly is a much bigger change, outside the scope of this PR. Author: Aaron Davidson Closes #715 from aarondav/throwable and squashes the following commits: f9b9bfe [Aaron Davidson] Remove other redundant 'throw e' e937a0a [Aaron Davidson] Address Prashant and Matei's comments 1867867 [Aaron Davidson] [RFC] SPARK-1772 Stop catching Throwable, let Executors die --- .../org/apache/spark/ContextCleaner.scala | 11 +-- .../scala/org/apache/spark/SparkContext.scala | 12 ++-- .../apache/spark/api/python/PythonRDD.scala | 3 +- .../api/python/PythonWorkerFactory.scala | 1 - .../org/apache/spark/deploy/Client.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 8 +-- .../apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/worker/DriverWrapper.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 37 +++------- .../ExecutorUncaughtExceptionHandler.scala | 53 +++++++++++++++ .../scheduler/EventLoggingListener.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 8 +-- .../spark/storage/DiskBlockManager.scala | 6 +- .../spark/storage/TachyonBlockManager.scala | 7 +- .../org/apache/spark/util/AkkaUtils.scala | 11 +-- .../util/IndestructibleActorSystem.scala | 68 ------------------- .../scala/org/apache/spark/util/Utils.scala | 26 ++++++- 19 files changed, 127 insertions(+), 140 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 54e08d7866f75..e2d2250982daa 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Classes that represent cleaning tasks. @@ -110,7 +111,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning() { + private def keepCleaning(): Unit = Utils.logUncaughtExceptions { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) @@ -128,7 +129,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } } catch { - case t: Throwable => logError("Error in cleaning thread", t) + case e: Exception => logError("Error in cleaning thread", e) } } } @@ -141,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.rddCleaned(rddId)) logInfo("Cleaned RDD " + rddId) } catch { - case t: Throwable => logError("Error cleaning RDD " + rddId, t) + case e: Exception => logError("Error cleaning RDD " + rddId, e) } } @@ -154,7 +155,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.shuffleCleaned(shuffleId)) logInfo("Cleaned shuffle " + shuffleId) } catch { - case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) } } @@ -166,7 +167,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.broadcastCleaned(broadcastId)) logInfo("Cleaned broadcast " + broadcastId) } catch { - case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71bab295442fc..e6121a705497c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1494,8 +1494,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) @@ -1510,8 +1510,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -1521,8 +1521,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 2971c277aa863..57b28b9972366 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -171,7 +171,7 @@ private[spark] class PythonRDD[T: ClassTag]( this.interrupt() } - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) @@ -282,7 +282,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 002f2acd94dee..759cbe2c46c52 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -71,7 +71,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - case e: Throwable => throw e } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7ead1171525d2..aeb159adc31d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -157,7 +157,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e2df1b8954124..148115d3ed351 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -103,7 +103,7 @@ object SparkHadoopUtil { .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { - case th: Throwable => throw new SparkException("Unable to load YARN support", th) + case e: Exception => throw new SparkException("Unable to load YARN support", e) } } else { new SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 1238bbf9da2fd..a9c11dca5678e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -70,7 +70,7 @@ class HistoryServer( * TODO: Add a mechanism to update manually. */ private val logCheckingThread = new Thread { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (!stopped) { val now = System.currentTimeMillis if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { @@ -154,7 +154,7 @@ class HistoryServer( numCompletedApplications = logInfos.size } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } else { logWarning("Attempted to check for event log updates before binding the server.") @@ -231,8 +231,8 @@ class HistoryServer( dir.getModificationTime } } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) + case e: Exception => + logError("Exception in accessing modification time of %s".format(dir.getPath), e) -1L } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f254f5585ba25..c6dec305bffcb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -684,8 +684,8 @@ private[spark] class Master( webUi.attachSparkUI(ui) return true } catch { - case t: Throwable => - logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + case e: Exception => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) } } else { logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index be15138f62406..05e242e6df702 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -31,7 +31,7 @@ object DriverWrapper { case workerUrl :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e912ae8a5d3c5..84aec65b7765d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -105,7 +105,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a // SparkEnv / Executor before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) + conf, new SecurityManager(conf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 98e7e0be813be..baee7a216a7c3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -74,28 +74,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler( - new Thread.UncaughtExceptionHandler { - override def uncaughtException(thread: Thread, exception: Throwable) { - try { - logError("Uncaught exception in thread " + thread, exception) - - // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) - if (!Utils.inShutdown()) { - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) - } - } - } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) - } - } - } - ) + Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -259,6 +238,11 @@ private[spark] class Executor( } case t: Throwable => { + // Attempt to exit cleanly by informing the driver of our failure. + // If anything goes wrong (or this was a fatal exception), we will delegate to + // the default uncaught exception handler, which will terminate the Executor. + logError("Exception in task ID " + taskId, t) + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -268,10 +252,11 @@ private[spark] class Executor( val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - // TODO: Should we exit the whole executor here? On the one hand, the failed task may - // have left some weird state around depending on when the exception was thrown, but on - // the other hand, maybe we could detect that when future tasks fail and exit then. - logError("Exception in task ID " + taskId, t) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala new file mode 100644 index 0000000000000..b0e984c03964c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala @@ -0,0 +1,53 @@ +/* + * 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.executor + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * The default uncaught exception handler for Executors terminates the whole process, to avoid + * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better + * to fail fast when things go wrong. + */ +private[spark] object ExecutorUncaughtExceptionHandler + extends Thread.UncaughtExceptionHandler with Logging { + + override def uncaughtException(thread: Thread, exception: Throwable) { + try { + logError("Uncaught exception in thread " + thread, exception) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } + } + } catch { + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + } + } + + def uncaughtException(exception: Throwable) { + uncaughtException(Thread.currentThread(), exception) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7968a0691db10..a90b0d475c04e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -206,8 +206,8 @@ private[spark] object EventLoggingListener extends Logging { applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } ) } catch { - case t: Throwable => - logError("Exception in parsing logging info from directory %s".format(logDir), t) + case e: Exception => + logError("Exception in parsing logging info from directory %s".format(logDir), e) EventLoggingInfo.empty } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index c9ad2b151daf0..99d305b36a959 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -43,7 +43,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => directResult @@ -70,7 +70,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Throwable => + case ex: Exception => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -81,7 +81,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( @@ -94,7 +94,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Throwable => {} + case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index cf6ef0029a861..3a7243a1ba19c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -148,7 +148,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def addShutdownHook() { localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") DiskBlockManager.this.stop() } @@ -162,8 +162,8 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) + case e: Exception => + logError("Exception while deleting local spark dir: " + localDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index b0b9674856568..a6cbe3aa440ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,6 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -137,7 +136,7 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") tachyonDirs.foreach { tachyonDir => try { @@ -145,8 +144,8 @@ private[spark] class TachyonBlockManager( Utils.deleteRecursively(tachyonDir, client) } } catch { - case t: Throwable => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8afe09a117ebc..a8d12bb2a0165 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} @@ -41,7 +41,7 @@ private[spark] object AkkaUtils extends Logging { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + def createActorSystem(name: String, host: String, port: Int, conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) @@ -101,12 +101,7 @@ private[spark] object AkkaUtils extends Logging { |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) - val actorSystem = if (indestructible) { - IndestructibleActorSystem(name, akkaConf) - } else { - ActorSystem(name, akkaConf) - } - + val actorSystem = ActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala deleted file mode 100644 index 4188a869c13da..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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. - */ - -// Must be in akka.actor package as ActorSystemImpl is protected[akka]. -package akka.actor - -import scala.util.control.{ControlThrowable, NonFatal} - -import com.typesafe.config.Config - -/** - * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception - * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see org.apache.spark.executor.Executor) - */ -object IndestructibleActorSystem { - def apply(name: String, config: Config): ActorSystem = - apply(name, config, ActorSystem.findClassLoader()) - - def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = - new IndestructibleActorSystemImpl(name, config, classLoader).start() -} - -private[akka] class IndestructibleActorSystemImpl( - override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) - extends ActorSystemImpl(name, applicationConfig, classLoader) { - - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { - val fallbackHandler = super.uncaughtExceptionHandler - - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable): Unit = { - if (isFatalError(cause) && !settings.JvmExitOnFatalError) { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - // shutdown() //TODO make it configurable - } else { - fallbackHandler.uncaughtException(thread, cause) - } - } - } - } - - def isFatalError(e: Throwable): Boolean = { - e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => - false - case _ => - true - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 95777fbf57d8b..8f7594ada2ba1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -29,6 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -41,7 +42,6 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - /** * Various utility methods used by Spark. */ @@ -1125,4 +1125,28 @@ private[spark] object Utils extends Logging { } } + /** + * Executes the given block, printing and re-throwing any uncaught exceptions. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } + } } From d9c97ba39723d36d3dcb53a75a3a2565357c0955 Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Mon, 12 May 2014 11:10:28 -0700 Subject: [PATCH 03/24] SPARK-1806: Upgrade Mesos dependency to 0.18.1 Enabled Mesos (0.18.1) dependency with shaded protobuf Why is this needed? Avoids any protobuf version collision between Mesos and any other dependency in Spark e.g. Hadoop HDFS 2.2+ or 1.0.4. Ticket: https://issues.apache.org/jira/browse/SPARK-1806 * Should close https://issues.apache.org/jira/browse/SPARK-1433 Author berngp Author: Bernardo Gomez Palacio Closes #741 from berngp/feature/SPARK-1806 and squashes the following commits: 5d70646 [Bernardo Gomez Palacio] SPARK-1806: Upgrade Mesos dependency to 0.18.1 --- core/pom.xml | 3 ++- .../apache/spark/executor/MesosExecutorBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- pom.xml | 10 +++++++++- project/SparkBuild.scala | 2 +- 5 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8fe215ab24289..bab50f5ce2888 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -140,6 +140,7 @@ org.apache.mesos mesos + ${mesos.classifier} io.netty @@ -322,7 +323,7 @@ - + src/main/resources diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 9b56f711e0e0b..74100498bb2bd 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c975f312324ed..2cea1c8cf154a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.util.Collections import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} diff --git a/pom.xml b/pom.xml index c4e1c6be52a1b..dd1d262881b93 100644 --- a/pom.xml +++ b/pom.xml @@ -113,7 +113,8 @@ 2.10.4 2.10 - 0.13.0 + 0.18.1 + shaded-protobuf org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -349,6 +350,13 @@ org.apache.mesos mesos ${mesos.version} + ${mesos.classifier} + + + com.google.protobuf + protobuf-java + + commons-net diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 12791e490ae2b..57b3e22f81713 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -343,7 +343,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From 0e2bde2030f8e455c5a269fc38d4ff05b395ca32 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 12 May 2014 13:05:24 -0700 Subject: [PATCH 04/24] SPARK-1786: Reopening PR 724 Addressing issue in MimaBuild.scala. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #742 from jegonzal/edge_partition_serialization and squashes the following commits: 8ba6e0d [Ankur Dave] Add concatenation operators to MimaBuild.scala cb2ed3a [Joseph E. Gonzalez] addressing missing exclusion in MimaBuild.scala 5d27824 [Ankur Dave] Disable reference tracking to fix serialization test c0a9ae5 [Ankur Dave] Add failing test for EdgePartition Kryo serialization a4a3faa [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ project/MimaBuild.scala | 8 +++++--- 12 files changed, 49 insertions(+), 26 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac72..f97f329c0e832 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245c..a5c9cd1f8b4e6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e45..4520beb991515 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65d..56f79a7097fce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f448..d02e9238adba5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05ae..dca54b8a7da86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1ba..55c7a19d1bdab 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f2..34939b24440aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6c..a4f769b294010 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9cb..57b01b6f2e1fb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35ef..28fd112f2b124 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index fafc9b36a77d3..e147be7ddaa61 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -30,7 +30,7 @@ object MimaBuild { // Read package-private excludes from file val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes") - val excludeFile = file(excludeFilePath) + val excludeFile = file(excludeFilePath) val packagePrivateList: Seq[String] = if (!excludeFile.exists()) { Seq() @@ -41,10 +41,10 @@ object MimaBuild { // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( - excludePackage(className), + excludePackage(className), ProblemFilters.exclude[MissingClassProblem](className), ProblemFilters.exclude[MissingTypesProblem](className), - excludePackage(className + "$"), + excludePackage(className + "$"), ProblemFilters.exclude[MissingClassProblem](className + "$"), ProblemFilters.exclude[MissingTypesProblem](className + "$") ) @@ -78,6 +78,8 @@ object MimaBuild { excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("graphx.impl.GraphImpl") ++ excludeSparkClass("graphx.impl.RoutingTable") ++ + excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ + excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 1e4a65e69489ff877e6da6f78b1c1306335e373c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 13:20:23 -0700 Subject: [PATCH 05/24] BUILD: Include Hive with default packages when creating a release --- dev/create-release/create-release.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33552a74920a7..b7a0a01c3bad2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -109,9 +109,9 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "--hadoop 1.0.4" -make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" +make_binary_release "hadoop1" "--with-hive --hadoop 1.0.4" +make_binary_release "cdh4" "--with-hive --hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-hive --with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" From 7120a2979d0a9f0f54a88b2416be7ca10e74f409 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:16:19 -0700 Subject: [PATCH 06/24] SPARK-1798. Tests should clean up temp files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent. Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former. The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules. Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method. _If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._ Author: Sean Owen Closes #732 from srowen/SPARK-1798 and squashes the following commits: 5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests --- bagel/src/test/resources/log4j.properties | 2 +- .../scala/org/apache/spark/TestUtils.scala | 1 + .../scala/org/apache/spark/util/Utils.scala | 18 +++--- .../java/org/apache/spark/JavaAPISuite.java | 18 ++---- core/src/test/resources/log4j.properties | 2 +- .../org/apache/spark/CheckpointSuite.scala | 5 +- .../org/apache/spark/FileServerSuite.scala | 18 ++++-- .../scala/org/apache/spark/FileSuite.scala | 56 +++++++++---------- .../WholeTextFileRecordReaderSuite.scala | 4 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 2 + .../scheduler/EventLoggingListenerSuite.scala | 15 +++-- .../spark/scheduler/ReplayListenerSuite.scala | 14 +++-- .../spark/storage/DiskBlockManagerSuite.scala | 30 +++++++--- .../apache/spark/util/FileLoggerSuite.scala | 17 ++++-- .../org/apache/spark/util/UtilsSuite.scala | 1 + .../flume/src/test/resources/log4j.properties | 2 +- .../kafka/src/test/resources/log4j.properties | 2 +- .../mqtt/src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../java/org/apache/spark/Java8APISuite.java | 3 + graphx/src/test/resources/log4j.properties | 2 +- mllib/src/test/resources/log4j.properties | 2 +- .../spark/mllib/util/MLUtilsSuite.scala | 17 ++---- pom.xml | 15 +++++ repl/src/test/resources/log4j.properties | 2 +- .../spark/repl/ExecutorClassLoaderSuite.scala | 24 ++++++-- .../org/apache/spark/repl/ReplSuite.scala | 3 + .../apache/spark/sql/InsertIntoSuite.scala | 12 ++++ .../org/apache/spark/sql/hive/TestHive.scala | 1 + .../streaming/util/MasterFailureTest.scala | 3 + .../apache/spark/streaming/JavaAPISuite.java | 5 +- streaming/src/test/resources/log4j.properties | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 + .../spark/streaming/InputStreamsSuite.scala | 1 + 35 files changed, 193 insertions(+), 114 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 5cdcf35b23a6c..30b4baa4d714a 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=bagel/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 8ae02154823ee..885c6829a2d72 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -43,6 +43,7 @@ private[spark] object TestUtils { */ def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8f7594ada2ba1..0631e54237923 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -586,15 +586,17 @@ private[spark] object Utils extends Logging { * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if ((file.isDirectory) && !isSymlink(file)) { - for (child <- listFilesSafely(file)) { - deleteRecursively(child) + if (file != null) { + if ((file.isDirectory) && !isSymlink(file)) { + for (child <- listFilesSafely(file)) { + deleteRecursively(child) + } } - } - if (!file.delete()) { - // Delete can also fail if the file simply did not exist - if (file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath) + if (!file.delete()) { + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 1912015827927..3dd79243ab5bd 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,7 +18,6 @@ package org.apache.spark; import java.io.*; -import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; @@ -49,16 +48,20 @@ import org.apache.spark.partial.PartialResult; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; +import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite implements Serializable { private transient JavaSparkContext sc; + private transient File tempDir; @Before public void setUp() { sc = new JavaSparkContext("local", "JavaAPISuite"); + tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); } @After @@ -67,6 +70,7 @@ public void tearDown() { sc = null; // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port"); + Utils.deleteRecursively(tempDir); } static class ReverseIntComparator implements Comparator, Serializable { @@ -611,7 +615,6 @@ public void glom() { @Test public void textFiles() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir); @@ -630,7 +633,6 @@ public void wholeTextFiles() throws IOException { byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); - File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); ds.write(content1); @@ -653,7 +655,6 @@ public void wholeTextFiles() throws IOException { @Test public void textFilesCompressed() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -667,7 +668,6 @@ public void textFilesCompressed() throws IOException { @SuppressWarnings("unchecked") @Test public void sequenceFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -697,7 +697,6 @@ public Tuple2 call(Tuple2 pair) { @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -728,7 +727,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -758,7 +756,6 @@ public String call(Tuple2 x) { @Test public void objectFilesOfInts() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsObjectFile(outputDir); @@ -771,7 +768,6 @@ public void objectFilesOfInts() { @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -788,7 +784,6 @@ public void objectFilesOfComplexTypes() { @SuppressWarnings("unchecked") @Test public void hadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -818,7 +813,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -948,7 +942,6 @@ public String call(Integer t) throws Exception { @Test public void checkpointAndComputation() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); @@ -960,7 +953,6 @@ public void checkpointAndComputation() { @Test public void checkpointAndRestore() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index f6fef03689a7c..26b73a1b39744 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=core/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2555b7c052c1..64933f4b1046d 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -35,6 +35,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def beforeEach() { super.beforeEach() checkpointDir = File.createTempFile("temp", "") + checkpointDir.deleteOnExit() checkpointDir.delete() sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) @@ -42,9 +43,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def afterEach() { super.afterEach() - if (checkpointDir != null) { - checkpointDir.delete() - } + Utils.deleteRecursively(checkpointDir) } test("basic checkpointing") { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index d651fbbac4e97..7e18f45de7b5b 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -24,9 +24,11 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { + @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ @@ -38,15 +40,18 @@ class FileServerSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - val tmpDir = new File(Files.createTempDir(), "test") - tmpDir.mkdir() - val textFile = new File(tmpDir, "FileServerSuite.txt") + tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + val textFile = new File(testTempDir, "FileServerSuite.txt") val pw = new PrintWriter(textFile) pw.println("100") pw.close() - val jarFile = new File(tmpDir, "test.jar") + val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) System.setProperty("spark.authenticate", "false") @@ -70,6 +75,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { tmpJarUrl = jarFile.toURI.toURL.toString } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) + } + test("Distributing files locally") { sc = new SparkContext("local[4]", "test") sc.addFile(tmpFile.toString) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index b9b668d3cc62a..1f2206b1f0379 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -30,12 +30,24 @@ import org.apache.hadoop.mapreduce.Job import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { + var tempDir: File = _ + + override def beforeEach() { + super.beforeEach() + tempDir = Files.createTempDir() + tempDir.deleteOnExit() + } + + override def afterEach() { + super.afterEach() + Utils.deleteRecursively(tempDir) + } test("text files") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) @@ -49,7 +61,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("text files (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -71,7 +82,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -82,7 +92,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -104,7 +113,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) @@ -115,7 +123,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -126,7 +133,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key and value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -137,7 +143,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("implicit conversions in reading SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -154,7 +159,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of ints") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsObjectFile(outputDir) @@ -165,7 +169,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of complex types") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) nums.saveAsObjectFile(outputDir) @@ -177,7 +180,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, Text]]( @@ -189,7 +191,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("read SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -200,7 +201,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("file caching") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -214,67 +214,61 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath) + randomRDD.saveAsTextFile(tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) - randomRDD.saveAsTextFile(tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath + "/output") + randomRDD.saveAsTextFile(tempDir.getPath + "/output") } } test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-r-00000").exists() === true) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) - job.set("mapred.output.dir", tempdir.getPath + "/outputDataset_old") + job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) - assert(new File(tempdir.getPath + "/outputDataset_old/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) } test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) - job.getConfiguration.set("mapred.output.dir", tempdir.getPath + "/outputDataset_new") + job.getConfiguration.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) - assert(new File(tempdir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 33d6de9a76405..d5ebfb3f3fae1 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** * Tests the correctness of @@ -67,6 +68,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { test("Correctness of WholeTextFileRecordReader.") { val dir = Files.createTempDir() + dir.deleteOnExit() println(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => @@ -86,7 +88,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { s"file $filename contents can not match.") } - dir.delete() + Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8f3e6bd21b752..1230565ea5b7e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -236,11 +236,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("zero-partition RDD") { val emptyDir = Files.createTempDir() + emptyDir.deleteOnExit() val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + emptyDir.delete() } test("keys and values") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 95f5bcd855665..21e3db34b8b7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.{FileStatus, Path} @@ -30,6 +29,8 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} +import java.io.File + /** * Test whether EventLoggingListener logs events properly. * @@ -43,11 +44,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "spark-events") + private var testDir: File = _ + private var logDirPath: Path = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + logDirPath = Utils.getFilePath(testDir, "spark-events") + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Parse names of special files") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d1fe1fc348961..d81499ac6abef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.scheduler -import java.io.PrintWriter - -import scala.util.Try +import java.io.{File, PrintWriter} import com.google.common.io.Files import org.json4s.jackson.JsonMethods._ @@ -39,11 +37,15 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() + private var testDir: File = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + } after { - Try { fileSystem.delete(Utils.getFilePath(testDir, "events.txt"), true) } - Try { fileSystem.delete(Utils.getFilePath(testDir, "test-replay"), true) } + Utils.deleteRecursively(testDir) } test("Simple replay") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 42bfbf1bdfc74..2167718fd2428 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -23,18 +23,16 @@ import scala.collection.mutable import scala.language.reflectiveCalls import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) - val rootDir0 = Files.createTempDir() - rootDir0.deleteOnExit() - val rootDir1 = Files.createTempDir() - rootDir1.deleteOnExit() - val rootDirs = rootDir0.getName + "," + rootDir1.getName - println("Created root dirs: " + rootDirs) + private var rootDir0: File = _ + private var rootDir1: File = _ + private var rootDirs: String = _ // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. @@ -48,6 +46,22 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def beforeAll() { + super.beforeAll() + rootDir0 = Files.createTempDir() + rootDir0.deleteOnExit() + rootDir1 = Files.createTempDir() + rootDir1.deleteOnExit() + rootDirs = rootDir0.getName + "," + rootDir1.getName + println("Created root dirs: " + rootDirs) + } + + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(rootDir0) + Utils.deleteRecursively(rootDir1) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala index f675e1e5b4981..44332fc8dbc23 100644 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.util -import java.io.IOException +import java.io.{File, IOException} import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.Path @@ -38,12 +37,18 @@ class FileLoggerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "test-file-logger") - private val logDirPathString = logDirPath.toString + private var testDir: File = _ + private var logDirPath: Path = _ + private var logDirPathString: String = _ + + before { + testDir = Files.createTempDir() + logDirPath = Utils.getFilePath(testDir, "test-file-logger") + logDirPathString = logDirPath.toString + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Simple logging") { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index eb7fb6318262b..cf9e20d347ddd 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -112,6 +112,7 @@ class UtilsSuite extends FunSuite { test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() + tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index d1bd73a8430e1..45d2ec676df66 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/flume/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 38910d113050a..45d2ec676df66 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/kafka/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index d0462c7336df5..45d2ec676df66 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/mqtt/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index c918335fcdc70..45d2ec676df66 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/twitter/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 304683dd0bac3..45d2ec676df66 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/zeromq/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index feabca6733484..84d3b6f243c72 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -39,6 +39,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; +import org.apache.spark.util.Utils; /** * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 @@ -249,6 +250,7 @@ public void mapPartitions() { @Test public void sequenceFile() { File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -265,6 +267,7 @@ public void sequenceFile() { JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) .mapToPair(pair -> new Tuple2(pair._1().get(), pair._2().toString())); Assert.assertEquals(pairs, readRDD.collect()); + Utils.deleteRecursively(tempDir); } @Test diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 85e57f0c4b504..26b73a1b39744 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=graphx/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 4265ba6e5de33..ddfc4ac6b23ed 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=mllib/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 3f64baf6fe41f..3d05fb68988c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -32,6 +32,7 @@ import com.google.common.io.Files import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.util.Utils class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -67,6 +68,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { |-1 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString @@ -90,7 +92,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(multiclassPoints(1).label === -1.0) assert(multiclassPoints(2).label === -1.0) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("saveAsLibSVMFile") { @@ -107,7 +109,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { .toSet val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") assert(lines === expected) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("appendBias") { @@ -158,16 +160,5 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - /** Delete a file/directory quietly. */ - def deleteQuietly(f: File) { - if (f.isDirectory) { - f.listFiles().foreach(deleteQuietly) - } - try { - f.delete() - } catch { - case _: Throwable => - } - } } diff --git a/pom.xml b/pom.xml index dd1d262881b93..5542a32a91368 100644 --- a/pom.xml +++ b/pom.xml @@ -796,6 +796,21 @@ + + org.apache.maven.plugins + maven-clean-plugin + 2.5 + + + + work + + + checkpoint + + + + diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index a6d33e69d21f7..9c4896e49698c 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=repl/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 336df988a1b7f..c0af7ceb6d3ef 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.repl import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite @@ -26,21 +26,35 @@ import org.scalatest.FunSuite import com.google.common.io.Files import org.apache.spark.TestUtils +import org.apache.spark.util.Utils class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") - val tempDir1 = Files.createTempDir() - val tempDir2 = Files.createTempDir() - val url1 = "file://" + tempDir1 - val urls2 = List(tempDir2.toURI.toURL).toArray + var tempDir1: File = _ + var tempDir2: File = _ + var url1: String = _ + var urls2: Array[URL] = _ override def beforeAll() { + super.beforeAll() + tempDir1 = Files.createTempDir() + tempDir1.deleteOnExit() + tempDir2 = Files.createTempDir() + tempDir2.deleteOnExit() + url1 = "file://" + tempDir1 + urls2 = List(tempDir2.toURI.toURL).toArray childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tempDir1) + Utils.deleteRecursively(tempDir2) + } + test("child first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(url1, parentLoader, true) diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 566d96e16ed83..95460aa205331 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -26,6 +26,7 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.util.Utils class ReplSuite extends FunSuite { @@ -178,6 +179,7 @@ class ReplSuite extends FunSuite { test("interacting with files") { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -196,6 +198,7 @@ class ReplSuite extends FunSuite { assertContains("res0: Long = 3", output) assertContains("res1: Long = 3", output) assertContains("res2: Long = 3", output) + Utils.deleteRecursively(tempDir) } test("local-cluster mode") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 73d87963b3a0d..4f0b85f26254b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -29,6 +29,7 @@ class InsertIntoSuite extends QueryTest { test("insertInto() created parquet file") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertTest") @@ -76,11 +77,14 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("INSERT INTO parquet table") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertSQLTest") @@ -126,23 +130,31 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertSQLTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("Double create fails when allowExisting = false") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) intercept[RuntimeException] { createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = false) } + + testFilePath.delete() } test("Double create does not fail when allowExisting = true") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = true) + + testFilePath.delete() } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 3ad66a3d7f45f..fa7d010459c63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -99,6 +99,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() + hiveFilesTemp.deleteOnExit() val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index c48a38590e060..b3ed302db6a38 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -21,6 +21,7 @@ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.util.Utils import StreamingContext._ import scala.util.Random @@ -380,6 +381,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) override def run() { val localTestDir = Files.createTempDir() + localTestDir.deleteOnExit() var fs = testDir.getFileSystem(new Configuration()) val maxTries = 3 try { @@ -421,6 +423,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) case e: Exception => logWarning("File generating in killing thread", e) } finally { fs.close() + Utils.deleteRecursively(localTestDir) } } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index f9bfb9b7444cc..ce58cb12a4564 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,7 +17,6 @@ package org.apache.spark.streaming; -import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -37,6 +36,8 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.Utils; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1606,6 +1607,7 @@ public void testCheckpointMasterRecovery() throws InterruptedException { Arrays.asList(8,7)); File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -1627,6 +1629,7 @@ public Integer call(String s) throws Exception { // will be re-processed after recovery List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); + Utils.deleteRecursively(tempDir); } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 063529a9cbc67..45d2ec676df66 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 25739956cb889..d20a7b728c741 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -232,6 +232,7 @@ class CheckpointSuite extends TestSuiteBase { test("recovery with file input stream") { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) @@ -326,6 +327,7 @@ class CheckpointSuite extends TestSuiteBase { ) // To ensure that all the inputs were received correctly assert(expectedOutput.last === output.last) + Utils.deleteRecursively(testDir) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3fa254065cc44..cd0aa4d0dce70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -98,6 +98,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() val ssc = new StreamingContext(conf, batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] From 8586bf564fe010dfc19ef26874472a6f85e355fb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:17:25 -0700 Subject: [PATCH 07/24] SPARK-1802. Audit dependency graph when Spark is built with -Phive This initial commit resolves the conflicts in the Hive profiles as noted in https://issues.apache.org/jira/browse/SPARK-1802 . Most of the fix was to note that Hive drags in Avro, and so if the hive module depends on Spark's version of the `avro-*` dependencies, it will pull in our exclusions as needed too. But I found we need to copy some exclusions between the two Avro dependencies to get this right. And then had to squash some commons-logging intrusions. This turned up another annoying find, that `hive-exec` is basically an "assembly" artifact that _also_ packages all of its transitive dependencies. This means the final assembly shows lots of collisions between itself and its dependencies, and even other project dependencies. I have a TODO to examine whether that is going to be a deal-breaker or not. In the meantime I'm going to tack on a second commit to this PR that will also fix some similar, last collisions in the YARN profile. Author: Sean Owen Closes #744 from srowen/SPARK-1802 and squashes the following commits: a856604 [Sean Owen] Resolve JAR version conflicts specific to Hive profile --- pom.xml | 16 ++++++++++++++++ sql/hive/pom.xml | 21 +++++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/pom.xml b/pom.xml index 5542a32a91368..3554efafb976a 100644 --- a/pom.xml +++ b/pom.xml @@ -535,6 +535,22 @@ io.netty netty + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 889d249146b8c..8b32451d76045 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -52,6 +52,12 @@ org.apache.hive hive-exec ${hive.version} + + + commons-logging + commons-logging + + org.codehaus.jackson @@ -61,6 +67,21 @@ org.apache.hive hive-serde ${hive.version} + + + commons-logging + commons-logging + + + commons-logging + commons-logging-api + + + + + + org.apache.avro + avro org.scalatest From 3ce526b168050c572a1feee8e0121e1426f7d9ee Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:09:13 -0700 Subject: [PATCH 08/24] Rename testExecutorEnvs --> executorEnvs. This was changed, but in fact, it's used for things other than tests. So I've changed it back. Author: Patrick Wendell Closes #747 from pwendell/executor-env and squashes the following commits: 36a60a5 [Patrick Wendell] Rename testExecutorEnvs --> executorEnvs. --- .../main/scala/org/apache/spark/SparkContext.scala | 11 +++++------ .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6121a705497c..35beff0f8ef3d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -276,19 +276,18 @@ class SparkContext(config: SparkConf) extends Logging { .getOrElse(512) // Environment variables to pass to our executors. - // NOTE: This should only be used for test related settings. - private[spark] val testExecutorEnvs = HashMap[String, String]() + private[spark] val executorEnvs = HashMap[String, String]() // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - testExecutorEnvs(envKey) = value + executorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - testExecutorEnvs ++= conf.getExecutorEnv + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -296,7 +295,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - testExecutorEnvs("SPARK_USER") = sparkUser + executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index cefa41729964a..933f6e0571518 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -54,7 +54,7 @@ private[spark] class SparkDeploySchedulerBackend( } val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 2cd9d6c12eaf7..cbe9bb093d1c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -122,7 +122,7 @@ private[spark] class CoarseMesosSchedulerBackend( val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 2cea1c8cf154a..ff8356f67681e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) From 925d8b249b84d2706c52f0d1e29fb8dcd6de452e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:27:28 -0700 Subject: [PATCH 09/24] SPARK-1623: Use File objects instead of String's in HTTPBroadcast This seems strictly better, and I think it's justified only the grounds of clean-up. It might also fix issues with path conversions, but I haven't yet isolated any instance of that happening. /cc @srowen @tdas Author: Patrick Wendell Closes #749 from pwendell/broadcast-cleanup and squashes the following commits: d6d54f2 [Patrick Wendell] SPARK-1623: Use File objects instead of string's in HTTPBroadcast --- .../scala/org/apache/spark/broadcast/HttpBroadcast.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 78fc286e5192c..4f6cabaff2b99 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -112,7 +112,7 @@ private[spark] object HttpBroadcast extends Logging { private var securityManager: SecurityManager = null // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist - private val files = new TimeStampedHashSet[String] + private val files = new TimeStampedHashSet[File] private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt private var compressionCodec: CompressionCodec = null private var cleaner: MetadataCleaner = null @@ -173,7 +173,7 @@ private[spark] object HttpBroadcast extends Logging { val serOut = ser.serializeStream(out) serOut.writeObject(value) serOut.close() - files += file.getAbsolutePath + files += file } def read[T: ClassTag](id: Long): T = { @@ -216,7 +216,7 @@ private[spark] object HttpBroadcast extends Logging { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) if (removeFromDriver) { val file = getFile(id) - files.remove(file.toString) + files.remove(file) deleteBroadcastFile(file) } } @@ -232,7 +232,7 @@ private[spark] object HttpBroadcast extends Logging { val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { iterator.remove() - deleteBroadcastFile(new File(file.toString)) + deleteBroadcastFile(file) } } } From 4b31f4ec7efab8eabf956284a99bfd96a58b79f7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 17:35:29 -0700 Subject: [PATCH 10/24] SPARK-1802. (Addendium) Audit dependency graph when Spark is built with -Pyarn Following on a few more items from SPARK-1802 -- The first commit touches up a few similar problems remaining with the YARN profile. I think this is worth cherry-picking. The second commit is more of the same for hadoop-client, although the fix is a little more complex. It may or may not be worth bothering with. Author: Sean Owen Closes #746 from srowen/SPARK-1802.2 and squashes the following commits: 52aeb41 [Sean Owen] Add more commons-logging, servlet excludes to avoid conflicts in assembly when building for YARN --- pom.xml | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pom.xml b/pom.xml index 3554efafb976a..4d4c5f63e44a3 100644 --- a/pom.xml +++ b/pom.xml @@ -583,6 +583,10 @@ org.jboss.netty netty + + commons-logging + commons-logging + @@ -606,6 +610,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + @@ -625,6 +633,14 @@ org.jboss.netty netty + + javax.servlet + servlet-api + + + commons-logging + commons-logging + @@ -648,6 +664,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + From beb9cbaca6dbbcaba77a34df692dc2e56b3e0638 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 17:39:40 -0700 Subject: [PATCH 11/24] [SPARK-1736] Spark submit for Windows Tested on Windows 7. Author: Andrew Or Closes #745 from andrewor14/windows-submit and squashes the following commits: c0b58fb [Andrew Or] Allow spaces in parameters 162e54d [Andrew Or] Merge branch 'master' of github.com:apache/spark into windows-submit 91597ce [Andrew Or] Make spark-shell.cmd use spark-submit.cmd af6fd29 [Andrew Or] Add spark submit for Windows --- bin/spark-shell.cmd | 5 ++-- bin/spark-submit.cmd | 56 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 3 deletions(-) create mode 100644 bin/spark-submit.cmd diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 99799128eb734..ca0c722c926f5 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Find the path of sbin -set BIN=%~dp0..\bin\ +set SPARK_HOME=%~dp0.. -cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd new file mode 100644 index 0000000000000..6eb702ed8c561 --- /dev/null +++ b/bin/spark-submit.cmd @@ -0,0 +1,56 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. +set ORIG_ARGS=%* + +rem Clear the values of all variables used +set DEPLOY_MODE= +set DRIVER_MEMORY= +set SPARK_SUBMIT_LIBRARY_PATH= +set SPARK_SUBMIT_CLASSPATH= +set SPARK_SUBMIT_OPTS= +set SPARK_DRIVER_MEMORY= + +:loop +if [%1] == [] goto continue + if [%1] == [--deploy-mode] ( + set DEPLOY_MODE=%2 + ) else if [%1] == [--driver-memory] ( + set DRIVER_MEMORY=%2 + ) else if [%1] == [--driver-library-path] ( + set SPARK_SUBMIT_LIBRARY_PATH=%2 + ) else if [%1] == [--driver-class-path] ( + set SPARK_SUBMIT_CLASSPATH=%2 + ) else if [%1] == [--driver-java-options] ( + set SPARK_SUBMIT_OPTS=%2 + ) + shift +goto loop +:continue + +if [%DEPLOY_MODE%] == [] ( + set DEPLOY_MODE=client +) + +if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( + set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% From 3e13b8c0bd46894392bd0fbd3e86dd55b8103858 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 15:40:48 -0700 Subject: [PATCH 12/24] Adding hadoop-2.2 profile to the build --- dev/create-release/create-release.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b7a0a01c3bad2..c4e74990860b6 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -52,14 +52,14 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. From 2f1a3373583f9b34a121236c25f5142ba8729546 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 12 May 2014 18:40:30 -0700 Subject: [PATCH 13/24] [SQL] Make Hive Metastore conversion functions publicly visible. I need this to be public for the implementation of SharkServer2. However, I think this functionality is generally useful and should be pretty stable. Author: Michael Armbrust Closes #750 from marmbrus/metastoreTypes and squashes the following commits: f51b62e [Michael Armbrust] Make Hive Metastore conversion functions publicly visible. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ba837a274c51c..fa30f596f8c67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ @@ -172,7 +173,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with override def unregisterAllTables() = {} } -private[hive] object HiveMetastoreTypes extends RegexParsers { +/** + * :: DeveloperApi :: + * Provides conversions between Spark SQL data types and Hive Metastore types. + */ +@DeveloperApi +object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | From a5150d199ca97ab2992bc2bb221a3ebf3d3450ba Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 18:46:28 -0700 Subject: [PATCH 14/24] Typo: resond -> respond Author: Andrew Ash Closes #743 from ash211/patch-4 and squashes the following commits: c959f3b [Andrew Ash] Typo: resond -> respond --- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index ff8356f67681e..f08b19e6782e3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -175,7 +175,7 @@ private[spark] class MesosSchedulerBackend( override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} /** - * Method called by Mesos to offer resources on slaves. We resond by asking our active task sets + * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ From 5c2275d6e4639946fd11ff6403338c8a9ade3d1e Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 12 May 2014 19:20:24 -0700 Subject: [PATCH 15/24] L-BFGS Documentation Documentation for L-BFGS, and an example of training binary L2 logistic regression using L-BFGS. Author: DB Tsai Closes #702 from dbtsai/dbtsai-lbfgs-doc and squashes the following commits: 0712215 [DB Tsai] Update 38fdfa1 [DB Tsai] Removed extra empty line 5745b64 [DB Tsai] Update again e9e418e [DB Tsai] Update 7381521 [DB Tsai] L-BFGS Documentation --- docs/mllib-optimization.md | 120 +++++++++++++++++++++++++++++++++++-- 1 file changed, 116 insertions(+), 4 deletions(-) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index bec3912b55dc7..aa0dec2130593 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -28,7 +28,6 @@ title: MLlib - Optimization ## Mathematical description ### Gradient descent - The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -128,10 +127,19 @@ is sampled, i.e. `$|S|=$ miniBatchFraction $\cdot n = 1$`, then the algorithm is standard SGD. In that case, the step direction depends from the uniformly random sampling of the point. - +### Limited-memory BFGS (L-BFGS) +[L-BFGS](http://en.wikipedia.org/wiki/Limited-memory_BFGS) is an optimization +algorithm in the family of quasi-Newton methods to solve the optimization problems of the form +`$\min_{\wv \in\R^d} \; f(\wv)$`. The L-BFGS method approximates the objective function locally as a +quadratic without evaluating the second partial derivatives of the objective function to construct the +Hessian matrix. The Hessian matrix is approximated by previous gradient evaluations, so there is no +vertical scalability issue (the number of training features) when computing the Hessian matrix +explicitly in Newton's method. As a result, L-BFGS often achieves rapider convergence compared with +other first-order optimization. ## Implementation in MLlib +### Gradient descent and stochastic gradient descent Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the @@ -142,12 +150,12 @@ The SGD method [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: -* `gradient` is a class that computes the stochastic gradient of the function +* `Gradient` is a class that computes the stochastic gradient of the function being optimized, i.e., with respect to a single training example, at the current parameter value. MLlib includes gradient classes for common loss functions, e.g., hinge, logistic, least-squares. The gradient class takes as input a training example, its label, and the current parameter value. -* `updater` is a class that performs the actual gradient descent step, i.e. +* `Updater` is a class that performs the actual gradient descent step, i.e. updating the weights in each iteration, for a given gradient of the loss part. The updater is also responsible to perform the update from the regularization part. MLlib includes updaters for cases without regularization, as well as @@ -163,3 +171,107 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: * [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) + +### L-BFGS +L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various +ML algorithms such as Linear Regression, and Logistic Regression, you have to pass the gradient of objective +function, and updater into optimizer yourself instead of using the training APIs like +[LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +See the example below. It will be addressed in the next release. + +The L1 regularization by using +[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater) will not work since the +soft-thresholding logic in L1Updater is designed for gradient descent. See the developer's note. + +The L-BFGS method +[LBFGS.runLBFGS](api/scala/index.html#org.apache.spark.mllib.optimization.LBFGS) +has the following parameters: + +* `Gradient` is a class that computes the gradient of the objective function +being optimized, i.e., with respect to a single training example, at the +current parameter value. MLlib includes gradient classes for common loss +functions, e.g., hinge, logistic, least-squares. The gradient class takes as +input a training example, its label, and the current parameter value. +* `Updater` is a class that computes the gradient and loss of objective function +of the regularization part for L-BFGS. MLlib includes updaters for cases without +regularization, as well as L2 regularizer. +* `numCorrections` is the number of corrections used in the L-BFGS update. 10 is +recommended. +* `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. +* `regParam` is the regularization parameter when using regularization. + +The `return` is a tuple containing two elements. The first element is a column matrix +containing weights for every feature, and the second element is an array containing +the loss computed for every iteration. + +Here is an example to train binary logistic regression with L2 regularization using +L-BFGS optimizer. +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.classification.LogisticRegressionModel + +val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val numFeatures = data.take(1)(0).features.size + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + +// Append 1 into the training data as intercept. +val training = splits(0).map(x => (x.label, MLUtils.appendBias(x.features))).cache() + +val test = splits(1) + +// Run training algorithm to build the model +val numCorrections = 10 +val convergenceTol = 1e-4 +val maxNumIterations = 20 +val regParam = 0.1 +val initialWeightsWithIntercept = Vectors.dense(new Array[Double](numFeatures + 1)) + +val (weightsWithIntercept, loss) = LBFGS.runLBFGS( + training, + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept) + +val model = new LogisticRegressionModel( + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)), + weightsWithIntercept(weightsWithIntercept.size - 1)) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Loss of each step in training process") +loss.foreach(println) +println("Area under ROC = " + auROC) +{% endhighlight %} + +#### Developer's note +Since the Hessian is constructed approximately from previous gradient evaluations, +the objective function can not be changed during the optimization process. +As a result, Stochastic L-BFGS will not work naively by just using miniBatch; +therefore, we don't provide this until we have better understanding. + +* `Updater` is a class originally designed for gradient decent which computes +the actual gradient descent step. However, we're able to take the gradient and +loss of objective function of regularization for L-BFGS by ignoring the part of logic +only for gradient decent such as adaptive step size stuff. We will refactorize +this into regularizer to replace updater to separate the logic between +regularization and step update later. \ No newline at end of file From 9cf9f18973840f7287f7cfa5ce90efed3225bb30 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 12 May 2014 19:21:06 -0700 Subject: [PATCH 16/24] Modify a typo in monitoring.md As I mentioned in SPARK-1765, there is a word 'JXM' in monitoring.md. I think it's typo for 'JMX'. Author: Kousuke Saruta Closes #698 from sarutak/SPARK-1765 and squashes the following commits: bae9843 [Kousuke Saruta] modified a typoe in monitoring.md --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 6f35fc37c4075..fffc58ac39230 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -156,7 +156,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the * `ConsoleSink`: Logs metrics information to the console. * `CSVSink`: Exports metrics data to CSV files at regular intervals. -* `JmxSink`: Registers metrics for viewing in a JXM console. +* `JmxSink`: Registers metrics for viewing in a JMX console. * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data. * `GraphiteSink`: Sends metrics to a Graphite node. From 156df87e7ca0e6cda2cc970ecd1466ce06f7576f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 19:23:39 -0700 Subject: [PATCH 17/24] SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile() https://issues.apache.org/jira/browse/SPARK-1757 The first test succeeds, but the second test fails with exception: ``` [info] - save and load case class RDD with Nones as parquet *** FAILED *** (14 milliseconds) [info] java.lang.RuntimeException: Unsupported datatype StructType(List()) [info] at scala.sys.package$.error(package.scala:27) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.fromDataType(ParquetRelation.scala:201) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.immutable.List.foreach(List.scala:318) [info] at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) [info] at scala.collection.AbstractTraversable.map(Traversable.scala:105) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.convertFromAttributes(ParquetRelation.scala:234) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:267) [info] at org.apache.spark.sql.parquet.ParquetRelation$.createEmpty(ParquetRelation.scala:143) [info] at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:122) [info] at org.apache.spark.sql.execution.SparkStrategies$ParquetOperations$.apply(SparkStrategies.scala:139) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:268) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:268) [info] at org.apache.spark.sql.SchemaRDDLike$class.saveAsParquetFile(SchemaRDDLike.scala:66) [info] at org.apache.spark.sql.SchemaRDD.saveAsParquetFile(SchemaRDD.scala:98) ``` Author: Andrew Ash Author: Michael Armbrust Closes #690 from ash211/rdd-parquet-save and squashes the following commits: 747a0b9 [Andrew Ash] Merge pull request #1 from marmbrus/pr/690 54bd00e [Michael Armbrust] Need to put Option first since Option <: Seq. 8f3f281 [Andrew Ash] SPARK-1757 Add failing test for saving SparkSQL Schemas with Option[?] fields as parquet --- .../spark/sql/catalyst/ScalaReflection.scala | 6 +-- .../spark/sql/parquet/ParquetQuerySuite.scala | 44 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 792ef6cee6f5d..196695a0a188f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -41,6 +41,9 @@ object ScalaReflection { /** Returns a catalyst DataType for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): DataType = tpe match { + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + schemaFor(optType) case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss StructType( @@ -59,9 +62,6 @@ object ScalaReflection { case t if t <:< typeOf[String] => StringType case t if t <:< typeOf[Timestamp] => TimestampType case t if t <:< typeOf[BigDecimal] => DecimalType - case t if t <:< typeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - schemaFor(optType) case t if t <:< typeOf[java.lang.Integer] => IntegerType case t if t <:< typeOf[java.lang.Long] => LongType case t if t <:< typeOf[java.lang.Double] => DoubleType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d9c9b9a076ab9..ff1677eb8a480 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -42,6 +42,20 @@ import org.apache.spark.sql.test.TestSQLContext._ case class TestRDDEntry(key: Int, value: String) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + booleanField: Option[Boolean]) + class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -195,5 +209,35 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() } + + test("save and load case class RDD with nulls as parquet") { + val data = NullReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } + + test("save and load case class RDD with Nones as parquet") { + val data = OptionalReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } } From ba96bb3d591130075763706526f86fb2aaffa3ae Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:42:35 -0700 Subject: [PATCH 18/24] [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking around What they really mean is SPARK_DAEMON_***JAVA***_OPTS Author: Andrew Or Closes #751 from andrewor14/spark-daemon-opts and squashes the following commits: 70c41f9 [Andrew Or] SPARK_DAEMON_OPTS -> SPARK_DAEMON_JAVA_OPTS --- conf/spark-env.sh.template | 2 +- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index f906be611a931..4479e1e34cd4a 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -39,5 +39,5 @@ # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") -# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y") # - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index bd21fdc5a18e4..800616622d7bf 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -247,7 +247,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application | - ./spark-submit with --driver-java-options to set -X options for a driver | - spark.executor.extraJavaOptions to set -X options for executors - | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) """.stripMargin logError(error) From 2ffd1eafd28635dcecc0ac738d4a62c05d740925 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:44:14 -0700 Subject: [PATCH 19/24] [SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs --- conf/spark-defaults.conf.template | 3 +- conf/spark-env.sh.template | 4 +- docs/building-with-maven.md | 7 +++ docs/cluster-overview.md | 73 +++++++++++++++--------- docs/configuration.md | 64 +++++++++++++-------- docs/hadoop-third-party-distributions.md | 14 +++-- docs/index.md | 34 +++++++---- docs/java-programming-guide.md | 5 +- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 4 +- docs/running-on-yarn.md | 15 +++-- docs/scala-programming-guide.md | 13 +++-- docs/spark-standalone.md | 71 ++++++++++------------- 13 files changed, 184 insertions(+), 125 deletions(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index f840ff681d019..2779342769c14 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -2,6 +2,7 @@ # This is useful for setting default environmental settings. # Example: -# spark.master spark://master:7077 +# spark.master spark://master:7077 # spark.eventLog.enabled true # spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 4479e1e34cd4a..f8ffbf64278fb 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -30,11 +30,11 @@ # Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname -# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) -# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT +# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index b6dd553bbe06b..8b44535d82404 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -129,6 +129,13 @@ Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spi For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +## Building for PySpark on YARN ## + +PySpark on YARN is only supported if the jar is built with maven. Further, there is a known problem +with building this assembly jar on Red Hat based operating systems (see SPARK-1753). If you wish to +run PySpark on a YARN cluster with Red Hat installed, we recommend that you build the jar elsewhere, +then ship it over to the cluster. We are investigating the exact cause for this. + ## Packaging without Hadoop dependencies for deployment on YARN ## The assembly jar produced by "mvn package" will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The "hadoop-provided" profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 162c415b5883e..f05a755de7fec 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -66,62 +66,76 @@ script as shown here while passing your jar. For Python, you can use the `pyFiles` argument of SparkContext or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. -### Launching Applications with ./bin/spark-submit +### Launching Applications with Spark submit Once a user application is bundled, it can be launched using the `spark-submit` script located in the bin directory. This script takes care of setting up the classpath with Spark and its -dependencies, and can support different cluster managers and deploy modes that Spark supports. -It's usage is +dependencies, and can support different cluster managers and deploy modes that Spark supports: - ./bin/spark-submit --class path.to.your.Class [options] [app options] + ./bin/spark-submit \ + --class + --master \ + --deploy-mode \ + ... // other options + + [application-arguments] -When calling `spark-submit`, `[app options]` will be passed along to your application's -main class. To enumerate all options available to `spark-submit` run it with -the `--help` flag. Here are a few examples of common options: + main-class: The entry point for your application (e.g. org.apache.spark.examples.SparkPi) + master-url: The URL of the master node (e.g. spark://23.195.26.187:7077) + deploy-mode: Whether to deploy this application within the cluster or from an external client (e.g. client) + application-jar: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. + application-arguments: Space delimited arguments passed to the main method of , if any + +To enumerate all options available to `spark-submit` run it with the `--help` flag. Here are a few +examples of common options: {% highlight bash %} # Run application locally ./bin/spark-submit \ - --class my.main.ClassName + --class org.apache.spark.examples.SparkPi --master local[8] \ - my-app.jar + /path/to/examples.jar \ + 100 # Run on a Spark standalone cluster ./bin/spark-submit \ - --class my.main.ClassName - --master spark://mycluster:7077 \ + --class org.apache.spark.examples.SparkPi + --master spark://207.184.161.138:7077 \ --executor-memory 20G \ --total-executor-cores 100 \ - my-app.jar + /path/to/examples.jar \ + 1000 # Run on a YARN cluster -HADOOP_CONF_DIR=XX /bin/spark-submit \ - --class my.main.ClassName +HADOOP_CONF_DIR=XX ./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi --master yarn-cluster \ # can also be `yarn-client` for client mode --executor-memory 20G \ --num-executors 50 \ - my-app.jar + /path/to/examples.jar \ + 1000 {% endhighlight %} ### Loading Configurations from a File -The `spark-submit` script can load default `SparkConf` values from a properties file and pass them -onto your application. By default it will read configuration options from -`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the -application when run. They can obviate the need for certain flags to `spark-submit`: for -instance, if `spark.master` property is set, you can safely omit the +The `spark-submit` script can load default [Spark configuration values](configuration.html) from a +properties file and pass them on to your application. By default it will read configuration options +from `conf/spark-defaults.conf`. For more detail, see the section on +[loading default configurations](configuration.html#loading-default-configurations). + +Loading default Spark configurations this way can obviate the need for certain flags to +`spark-submit`. For instance, if the `spark.master` property is set, you can safely omit the `--master` flag from `spark-submit`. In general, configuration values explicitly set on a -`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values -in the defaults file. +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values in the +defaults file. -If you are ever unclear where configuration options are coming from. fine-grained debugging -information can be printed by adding the `--verbose` option to `./spark-submit`. +If you are ever unclear where configuration options are coming from, you can print out fine-grained +debugging information by running `spark-submit` with the `--verbose` option. ### Advanced Dependency Management -When using `./bin/spark-submit` the app jar along with any jars included with the `--jars` option -will be automatically transferred to the cluster. `--jars` can also be used to distribute .egg and .zip -libraries for Python to executors. Spark uses the following URL scheme to allow different -strategies for disseminating jars: +When using `spark-submit`, the application jar along with any jars included with the `--jars` option +will be automatically transferred to the cluster. Spark uses the following URL scheme to allow +different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor pulls the file from the driver HTTP server. @@ -135,6 +149,9 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. +For python, the equivalent `--py-files` option can be used to distribute .egg and .zip libraries +to executors. + # Monitoring Each driver program has a web UI, typically on port 4040, that displays information about running diff --git a/docs/configuration.md b/docs/configuration.md index 5b034e3cb3d47..2eed96f704a4f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -5,9 +5,9 @@ title: Spark Configuration Spark provides three locations to configure the system: -* [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java - system properties. +* [Spark properties](#spark-properties) control most application parameters and can be set by + passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, + or through the `conf/spark-defaults.conf` properties file. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. * [Logging](#configuring-logging) can be configured through `log4j.properties`. @@ -15,25 +15,41 @@ Spark provides three locations to configure the system: # Spark Properties -Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) -class to your SparkContext constructor. -Alternatively, Spark will also load them from Java system properties, for compatibility with old versions -of Spark. - -SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and -application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could -initialize an application as follows: +Spark properties control most application settings and are configured separately for each +application. The preferred way is to set them through +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passing it as an argument to your +SparkContext. SparkConf allows you to configure most of the common properties to initialize a +cluster (e.g. master URL and application name), as well as arbitrary key-value pairs through the +`set()` method. For example, we could initialize an application as follows: {% highlight scala %} -val conf = new SparkConf(). - setMaster("local"). - setAppName("My application"). - set("spark.executor.memory", "1g") +val conf = new SparkConf + .setMaster("local") + .setAppName("CountingSheep") + .set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} -Most of the properties control internal settings that have reasonable default values. However, +## Loading Default Configurations + +In the case of `spark-shell`, a SparkContext has already been created for you, so you cannot control +the configuration properties through SparkConf. However, you can still set configuration properties +through a default configuration file. By default, `spark-shell` (and more generally `spark-submit`) +will read configuration options from `conf/spark-defaults.conf`, in which each line consists of a +key and a value separated by whitespace. For example, + + spark.master spark://5.6.7.8:7077 + spark.executor.memory 512m + spark.eventLog.enabled true + spark.serializer org.apache.spark.serializer.KryoSerializer + +Any values specified in the file will be passed on to the application, and merged with those +specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf` +and SparkConf, then the latter will take precedence as it is the most application-specific. + +## All Configuration Properties + +Most of the properties that control internal settings have reasonable default values. However, there are at least five properties that you will commonly want to control: @@ -101,9 +117,9 @@ Apart from these, the following properties are also available, and may be useful @@ -696,7 +712,9 @@ Apart from these, the following properties are also available, and may be useful ## Viewing Spark Properties The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. -This is a useful place to check to make sure that your properties have been set correctly. +This is a useful place to check to make sure that your properties have been set correctly. Note +that only values explicitly specified through either `spark-defaults.conf` or SparkConf will +appear. For all other configuration properties, you can assume the default value is used. # Environment Variables @@ -714,8 +732,8 @@ The following variables can be set in `spark-env.sh`: * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. * `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. -* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores - to use on each machine and maximum memory. +* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), + such as number of cores to use on each machine and maximum memory. Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 454877a7fa8a5..a0aeab5727bde 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -9,12 +9,14 @@ with these distributions: # Compile-time Hadoop Version -When compiling Spark, you'll need to -[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): +When compiling Spark, you'll need to specify the Hadoop version by defining the `hadoop.version` +property. For certain versions, you will need to specify additional profiles. For more detail, +see the guide on [building with maven](building-with-maven.html#specifying-the-hadoop-version): - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + mvn -Dhadoop.version=1.0.4 -DskipTests clean package + mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that +The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors. @@ -46,6 +48,10 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
spark.default.parallelism
    +
  • Local mode: number of cores on the local machine
  • Mesos fine grained mode: 8
  • -
  • Local mode: core number of the local machine
  • -
  • Others: total core number of all executor nodes or 2, whichever is larger
  • +
  • Others: total number of cores on all executor nodes or 2, whichever is larger
@@ -187,7 +203,7 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of filter class names to apply to the Spark web ui. The filter should be a standard javax servlet Filter. Parameters to each filter can also be specified by setting a java system property of spark.<class name of filter>.params='param1=value1,param2=value2' - (e.g.-Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') + (e.g. -Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing')
+In SBT, the equivalent can be achieved by setting the SPARK_HADOOP_VERSION flag: + + SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + # Linking Applications to the Hadoop Version In addition to compiling Spark itself against the right version, you need to add a Maven dependency on that diff --git a/docs/index.md b/docs/index.md index a2f1a84371ff4..48182a27d28ae 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,21 +24,31 @@ right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/ # Running the Examples and Shell -Spark comes with several sample programs. Scala, Java and Python examples are in the `examples/src/main` directory. -To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory -(the `bin/run-example` script sets up the appropriate paths and launches that program). -For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. -To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./examples/src/main/python/pi.py local`. +Spark comes with several sample programs. Scala, Java and Python examples are in the +`examples/src/main` directory. To run one of the Java or Scala sample programs, use +`bin/run-example [params]` in the top-level Spark directory. (Behind the scenes, this +invokes the more general +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) for +launching applications). For example, -Each example prints usage help when run with no parameters. + ./bin/run-example SparkPi 10 -Note that all of the sample programs take a `` parameter specifying the cluster URL -to connect to. This can be a [URL for a distributed cluster](scala-programming-guide.html#master-urls), -or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using -`local` for testing. +You can also run Spark interactively through modified versions of the Scala shell. This is a +great way to learn the framework. -Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or -Python interpreter (`./bin/pyspark`). These are a great way to learn the framework. + ./bin/spark-shell --master local[2] + +The `--master` option specifies the +[master URL for a distributed cluster](scala-programming-guide.html#master-urls), or `local` to run +locally with one thread, or `local[N]` to run locally with N threads. You should start by using +`local` for testing. For a full list of options, run Spark shell with the `--help` option. + +Spark also provides a Python interface. To run an example Spark application written in Python, use +`bin/pyspark [params]`. For example, + + ./bin/pyspark examples/src/main/python/pi.py local[2] 10 + +or simply `bin/pyspark` without any arguments to run Spark interactively in a python interpreter. # Launching on a Cluster diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index c34eb28fc06a2..943fdd9d019ff 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -215,7 +215,4 @@ Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/apache/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.JavaWordCount - -Each example program prints usage help when run -without any arguments. + ./bin/run-example JavaWordCount README.md diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39fb5f0c99ca3..2ce2c346d7b04 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -164,6 +164,6 @@ some example applications. PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/spark-submit examples/src/main/python/wordcount.py + ./bin/spark-submit examples/src/main/python/wordcount.py local[2] README.md Each program prints usage help when run without arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 478b790f92e17..a4d01487bb494 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -18,7 +18,9 @@ you can download a package for any version of Hadoop. ## Basics Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running `./bin/spark-shell` in the Spark directory. +Start the shell by running the following in the Spark directory. + + ./bin/spark-shell Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c563594296802..66c330fdee734 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -54,13 +54,13 @@ For example: --executor-memory 2g \ --executor-cores 1 lib/spark-examples*.jar \ - yarn-cluster 5 + 10 The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell: - $ MASTER=yarn-client ./bin/spark-shell + $ ./bin/spark-shell --master yarn-client ## Adding additional jars @@ -70,9 +70,9 @@ In yarn-cluster mode, the driver runs on a different machine than the client, so --master yarn-cluster \ --jars my-other-jar.jar,my-other-other-jar.jar my-main-jar.jar - yarn-cluster 5 + app_arg1 app_arg2 -# Viewing logs +# Debugging your Application In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. @@ -82,6 +82,13 @@ will print out the contents of all log files from all containers from the given When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. +To review per-container launch environment, increase yarn.nodemanager.delete.debug-delay-sec to a +large value (e.g. 36000), and then access the application cache through yarn.nodemanager.local-dirs +on the nodes on which containers are launched. This directory contains the launch script, jars, and +all environment variables used for launching each container. This process is useful for debugging +classpath problems in particular. (Note that enabling this requires admin privileges on cluster +settings and a restart of all node managers. Thus, this is not applicable to hosted clusters). + # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index f25e9cca88524..3ed86e460c01c 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -56,7 +56,7 @@ The `master` parameter is a string specifying a [Spark, Mesos or YARN cluster UR to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. It's also possible to set these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) -which avoids hard-coding the master name in your application. +which avoids hard-coding the master url in your application. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the @@ -74,6 +74,11 @@ Or, to also add `code.jar` to its classpath, use: $ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} +For a complete list of options, run Spark shell with the `--help` option. Behind the scenes, +Spark shell invokes the more general [Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) +used for launching applications, and passes on all of its parameters. As a result, these two scripts +share the same parameters. + ### Master URLs The master URL passed to Spark can be in one of the following formats: @@ -98,7 +103,7 @@ cluster mode. The cluster location will be inferred based on the local Hadoop co -If no master URL is specified, the spark shell defaults to "local[*]". +If no master URL is specified, the spark shell defaults to `local[*]`. # Resilient Distributed Datasets (RDDs) @@ -432,9 +437,7 @@ res2: Int = 10 You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.SparkPi - -Each example program prints usage help when run without any arguments. + ./bin/run-example SparkPi For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index dc7f206e03996..eb3211b6b0e4e 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -70,7 +70,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. -- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`. +- `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. - `sbin/stop-all.sh` - Stops both the master and the slaves as described above. Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine. @@ -92,12 +92,8 @@ You can optionally configure the cluster further by setting environment variable Port for the master web UI (default: 8080). - SPARK_WORKER_PORT - Start the Spark worker on a specific port (default: random). - - - SPARK_WORKER_DIR - Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + SPARK_MASTER_OPTS + Configuration properties that apply only to the master in the form "-Dx=y" (default: none). SPARK_WORKER_CORES @@ -107,6 +103,10 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_MEMORY Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. + + SPARK_WORKER_PORT + Start the Spark worker on a specific port (default: random). + SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081). @@ -120,13 +120,25 @@ You can optionally configure the cluster further by setting environment variable or else each worker will try to use all the cores. + + SPARK_WORKER_DIR + Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + + + SPARK_WORKER_OPTS + Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). + SPARK_DAEMON_MEMORY Memory to allocate to the Spark master and worker daemons themselves (default: 512m). SPARK_DAEMON_JAVA_OPTS - JVM options for the Spark master and worker daemons themselves (default: none). + JVM options for the Spark master and worker daemons themselves in the form "-Dx=y" (default: none). + + + SPARK_PUBLIC_DNS + The public DNS name of the Spark master and workers (default: none). @@ -148,38 +160,17 @@ You can also pass an option `--cores ` to control the number of cores # Launching Compiled Spark Applications -Spark supports two deploy modes. Spark applications may run with the driver inside the client process or entirely inside the cluster. - -The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to the cluster in either deploy mode. For info on the lower-level invocations used to launch an app inside the cluster, read ahead. - -## Launching Applications Inside the Cluster - - ./bin/spark-class org.apache.spark.deploy.Client launch - [client-options] \ - \ - [application-options] - - cluster-url: The URL of the master node. - application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. - main-class: The entry point for your application. - - Client Options: - --memory (amount of memory, in MB, allocated for your driver program) - --cores (number of cores allocated for your driver program) - --supervise (whether to automatically restart your driver on application or node failure) - --verbose (prints increased logging output) - -Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways: - - * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program. - * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client. - * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes. - -Once you submit a driver program, it will appear in the cluster management UI at port 8080 and -be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using -the same client: +Spark supports two deploy modes: applications may run with the driver inside the client process or +entirely inside the cluster. The +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) provides the +most straightforward way to submit a compiled Spark application to the cluster in either deploy +mode. - ./bin/spark-class org.apache.spark.deploy.Client kill +If your application is launched through Spark submit, then the application jar is automatically +distributed to all worker nodes. For any additional jars that your application depends on, you +should specify them through the `--jars` flag using comma as a delimiter (e.g. `--jars jar1,jar2`). +To control the application's configuration or execution environment, see +[Spark Configuration](configuration.html). # Resource Scheduling @@ -203,7 +194,7 @@ default for applications that don't set `spark.cores.max` to something less than Do this by adding the following to `conf/spark-env.sh`: {% highlight bash %} -export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=" +export SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores=" {% endhighlight %} This is useful on shared clusters where users might not have configured a maximum number of cores From 2792bd016af2a67848e6f403c4e1e05e9f3e3c2a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 12 May 2014 20:08:30 -0700 Subject: [PATCH 20/24] SPARK-1815. SparkContext should not be marked DeveloperApi Author: Sandy Ryza Closes #753 from sryza/sandy-spark-1815 and squashes the following commits: 957a8ac [Sandy Ryza] SPARK-1815. SparkContext should not be marked DeveloperApi --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 35beff0f8ef3d..c43b4fd6d926f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -50,7 +50,6 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** - * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * @@ -58,7 +57,6 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * this config overrides the default configs as well as system properties. */ -@DeveloperApi class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, From 52d905296fd75b3bff34a16cb69e604b68b3c246 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 23:02:54 -0700 Subject: [PATCH 21/24] BUILD: Add more content to make-distribution.sh. --- make-distribution.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 1cc2844703fbb..7a08d6b9151c4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -171,10 +171,22 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +# Copy example sources (needed for python and SQL) +mkdir -p "$DISTDIR/examples/src/main" +cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" + if [ "$SPARK_HIVE" == "true" ]; then cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi +# Copy license and ASF files +cp "$FWDIR/LICENSE" "$DISTDIR" +cp "$FWDIR/NOTICE" "$DISTDIR" + +if [ -e $FWDIR/CHANGES.txt ]; then + cp "$FWDIR/CHANGES.txt" "$DISTDIR" +fi + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf @@ -182,6 +194,7 @@ cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" +cp -r "$FWDIR/ec2" "$DISTDIR" # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then From 16ffadcc4af21430b5079dc555bcd9d8cf1fa1fa Mon Sep 17 00:00:00 2001 From: William Benton Date: Tue, 13 May 2014 13:45:23 -0700 Subject: [PATCH 22/24] SPARK-571: forbid return statements in cleaned closures This patch checks top-level closure arguments to `ClosureCleaner.clean` for `return` statements and raises an exception if it finds any. This is mainly a user-friendliness addition, since programs with return statements in closure arguments will currently fail upon RDD actions with a less-than-intuitive error message. Author: William Benton Closes #717 from willb/spark-571 and squashes the following commits: c41eb7d [William Benton] Another test case for SPARK-571 30c42f4 [William Benton] Stylistic cleanups 559b16b [William Benton] Stylistic cleanups from review de13b79 [William Benton] Style fixes 295b6a5 [William Benton] Forbid return statements in closure arguments. b017c47 [William Benton] Added a test for SPARK-571 --- .../apache/spark/util/ClosureCleaner.scala | 23 ++++++++++- .../spark/util/ClosureCleanerSuite.scala | 39 ++++++++++++++++++- 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 2d05e09b10948..4916d9b86cca5 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.Set import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -108,6 +108,9 @@ private[spark] object ClosureCleaner extends Logging { val outerObjects = getOuterObjects(func) val accessedFields = Map[Class[_], Set[String]]() + + getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) + for (cls <- outerClasses) accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) @@ -180,6 +183,24 @@ private[spark] object ClosureCleaner extends Logging { } } +private[spark] +class ReturnStatementFinder extends ClassVisitor(ASM4) { + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name.contains("apply")) { + new MethodVisitor(ASM4) { + override def visitTypeInsn(op: Int, tp: String) { + if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) { + throw new SparkException("Return statements aren't allowed in Spark closures") + } + } + } + } else { + new MethodVisitor(ASM4) {} + } + } +} + private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index d7e48e633e0ee..054ef54e746a5 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.apache.spark.LocalSparkContext._ -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { @@ -50,6 +50,19 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } + + test("toplevel return statements in closures are identified at cleaning time") { + val ex = intercept[SparkException] { + TestObjectWithBogusReturns.run() + } + + assert(ex.getMessage.contains("Return statements aren't allowed in Spark closures")) + } + + test("return statements from named functions nested in closures don't raise exceptions") { + val result = TestObjectWithNestedReturns.run() + assert(result == 1) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -108,6 +121,30 @@ class TestClassWithoutFieldAccess { } } +object TestObjectWithBogusReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + // this return is invalid since it will transfer control outside the closure + nums.map {x => return 1 ; x * 2} + 1 + } + } +} + +object TestObjectWithNestedReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map {x => + // this return is fine since it will not transfer control outside the closure + def foo(): Int = { return 5; 1 } + foo() + } + 1 + } + } +} object TestObjectWithNesting { def run(): Int = { From d1e487473fd509f28daf28dcda856f3c2f1194ec Mon Sep 17 00:00:00 2001 From: Andrew Tulloch Date: Tue, 13 May 2014 17:31:27 -0700 Subject: [PATCH 23/24] SPARK-1791 - SVM implementation does not use threshold parameter Summary: https://issues.apache.org/jira/browse/SPARK-1791 Simple fix, and backward compatible, since - anyone who set the threshold was getting completely wrong answers. - anyone who did not set the threshold had the default 0.0 value for the threshold anyway. Test Plan: Unit test added that is verified to fail under the old implementation, and pass under the new implementation. Reviewers: CC: Author: Andrew Tulloch Closes #725 from ajtulloch/SPARK-1791-SVM and squashes the following commits: 770f55d [Andrew Tulloch] SPARK-1791 - SVM implementation does not use threshold parameter --- .../spark/mllib/classification/SVM.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 37 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e05213536e64a..316ecd713b715 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -65,7 +65,7 @@ class SVMModel private[mllib] ( intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { - case Some(t) => if (margin < 0) 0.0 else 1.0 + case Some(t) => if (margin < t) 0.0 else 1.0 case None => margin } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 77d6f04b32320..886c71dde3af7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -69,6 +69,43 @@ class SVMSuite extends FunSuite with LocalSparkContext { assert(numOffPredictions < input.length / 5) } + test("SVM with threshold") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMWithSGD().setIntercept(true) + svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.run(testRDD) + + val validationData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + + // Test prediction on RDD. + + var predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) != predictions.length) + + // High threshold makes all the predictions 0.0 + model.setThreshold(10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) == predictions.length) + + // Low threshold makes all the predictions 1.0 + model.setThreshold(-10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 1.0) == predictions.length) + } + test("SVM using local random SGD") { val nPoints = 10000 From 5c0dafc2c8734a421206a808b73be67b66264dd7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 13 May 2014 18:32:32 -0700 Subject: [PATCH 24/24] [SPARK-1816] LiveListenerBus dies if a listener throws an exception The solution is to wrap a try / catch / log around the posting of each event to each listener. Author: Andrew Or Closes #759 from andrewor14/listener-die and squashes the following commits: aee5107 [Andrew Or] Merge branch 'master' of github.com:apache/spark into listener-die 370939f [Andrew Or] Remove two layers of indirection 422d278 [Andrew Or] Explicitly throw an exception instead of 1 / 0 0df0e2a [Andrew Or] Try/catch and log exceptions when posting events --- .../spark/scheduler/LiveListenerBus.scala | 36 ++++++++++--- .../spark/scheduler/SparkListenerBus.scala | 50 +++++++++++++------ .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 50 +++++++++++++++++-- 4 files changed, 109 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index dec3316bf7745..36a6e6338faa6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging +import org.apache.spark.util.Utils /** * Asynchronously passes SparkListenerEvents to registered SparkListeners. @@ -42,7 +43,7 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (true) { eventLock.acquire() // Atomically remove and process this event @@ -77,11 +78,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { val eventAdded = eventQueue.offer(event) if (eventAdded) { eventLock.release() - } else if (!queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true + } else { + logQueueFullErrorMessage() } } @@ -96,13 +94,18 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (System.currentTimeMillis > finishTime) { return false } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ + /* Sleep rather than using wait/notify, because this is used only for testing and + * wait/notify add overhead in the general case. */ Thread.sleep(10) } true } + /** + * For testing only. Return whether the listener daemon thread is still alive. + */ + def listenerThreadIsAlive: Boolean = synchronized { listenerThread.isAlive } + /** * Return whether the event queue is empty. * @@ -111,6 +114,23 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { */ def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + /** + * Log an error message to indicate that the event queue is full. Do this only once. + */ + private def logQueueFullErrorMessage(): Unit = { + if (!queueFullErrorMessageLogged) { + if (listenerThread.isAlive) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with" + + "the rate at which tasks are being started by the scheduler.") + } else { + logError("SparkListenerBus thread is dead! This means SparkListenerEvents have not" + + "been (and will no longer be) propagated to listeners for some time.") + } + queueFullErrorMessageLogged = true + } + } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 0286aac8769b2..ed9fb24bc8ce8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -20,10 +20,13 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.util.Utils + /** * A SparkListenerEvent bus that relays events to its listeners */ -private[spark] trait SparkListenerBus { +private[spark] trait SparkListenerBus extends Logging { // SparkListeners attached to this event bus protected val sparkListeners = new ArrayBuffer[SparkListener] @@ -34,38 +37,53 @@ private[spark] trait SparkListenerBus { } /** - * Post an event to all attached listeners. This does nothing if the event is - * SparkListenerShutdown. + * Post an event to all attached listeners. + * This does nothing if the event is SparkListenerShutdown. */ def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + foreachListener(_.onStageSubmitted(stageSubmitted)) case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + foreachListener(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) + foreachListener(_.onJobStart(jobStart)) case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) + foreachListener(_.onJobEnd(jobEnd)) case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) + foreachListener(_.onTaskStart(taskStart)) case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + foreachListener(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) + foreachListener(_.onTaskEnd(taskEnd)) case environmentUpdate: SparkListenerEnvironmentUpdate => - sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + foreachListener(_.onEnvironmentUpdate(environmentUpdate)) case blockManagerAdded: SparkListenerBlockManagerAdded => - sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + foreachListener(_.onBlockManagerAdded(blockManagerAdded)) case blockManagerRemoved: SparkListenerBlockManagerRemoved => - sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + foreachListener(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => - sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + foreachListener(_.onUnpersistRDD(unpersistRDD)) case applicationStart: SparkListenerApplicationStart => - sparkListeners.foreach(_.onApplicationStart(applicationStart)) + foreachListener(_.onApplicationStart(applicationStart)) case applicationEnd: SparkListenerApplicationEnd => - sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) + foreachListener(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } + + /** + * Apply the given function to all attached listeners, catching and logging any exception. + */ + private def foreachListener(f: SparkListener => Unit): Unit = { + sparkListeners.foreach { listener => + try { + f(listener) + } catch { + case e: Exception => + logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + } + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0631e54237923..99ef6dd1fa6e3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1128,7 +1128,7 @@ private[spark] object Utils extends Logging { } /** - * Executes the given block, printing and re-throwing any uncaught exceptions. + * Execute the given block, logging and re-throwing any uncaught exception. * This is particularly useful for wrapping code that runs in a thread, to ensure * that exceptions are printed, and to avoid having to catch Throwable. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 4e9fd07e68a21..5426e578a9ddd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -331,16 +331,47 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - def checkNonZeroAvg(m: Traversable[Long], msg: String) { + test("SparkListener moves on if a listener throws an exception") { + val badListener = new BadListener + val jobCounter1 = new BasicJobCounter + val jobCounter2 = new BasicJobCounter + val bus = new LiveListenerBus + + // Propagate events to bad listener first + bus.addListener(badListener) + bus.addListener(jobCounter1) + bus.addListener(jobCounter2) + bus.start() + + // Post events to all listeners, and wait until the queue is drained + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + + // The exception should be caught, and the event should be propagated to other listeners + assert(bus.listenerThreadIsAlive) + assert(jobCounter1.count === 5) + assert(jobCounter2.count === 5) + } + + /** + * Assert that the given list of numbers has an average that is greater than zero. + */ + private def checkNonZeroAvg(m: Traversable[Long], msg: String) { assert(m.sum / m.size.toDouble > 0.0, msg) } - class BasicJobCounter extends SparkListener { + /** + * A simple listener that counts the number of jobs observed. + */ + private class BasicJobCounter extends SparkListener { var count = 0 override def onJobEnd(job: SparkListenerJobEnd) = count += 1 } - class SaveStageAndTaskInfo extends SparkListener { + /** + * A simple listener that saves all task infos and task metrics. + */ + private class SaveStageAndTaskInfo extends SparkListener { val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() @@ -358,7 +389,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - class SaveTaskEvents extends SparkListener { + /** + * A simple listener that saves the task indices for all task events. + */ + private class SaveTaskEvents extends SparkListener { val startedTasks = new mutable.HashSet[Int]() val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() @@ -377,4 +411,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc startedGettingResultTasks += taskGettingResult.taskInfo.index } } + + /** + * A simple listener that throws an exception on job end. + */ + private class BadListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + } + }