From be9377320e1292dada729b7163ce7de600bb10c6 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 12 May 2014 11:08:52 -0700 Subject: [PATCH 1/5] 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 (cherry picked from commit 3af1f386439cdddd42e545ad63d089f4dfdf9f8a) Signed-off-by: Patrick Wendell Conflicts: core/src/main/scala/org/apache/spark/ContextCleaner.scala core/src/main/scala/org/apache/spark/SparkContext.scala core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala core/src/main/scala/org/apache/spark/deploy/Client.scala core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala core/src/main/scala/org/apache/spark/deploy/master/Master.scala core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala core/src/main/scala/org/apache/spark/executor/Executor.scala core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala core/src/main/scala/org/apache/spark/util/AkkaUtils.scala core/src/main/scala/org/apache/spark/util/Utils.scala --- .../scala/org/apache/spark/SparkContext.scala | 1 + .../apache/spark/api/python/PythonRDD.scala | 1 - .../api/python/PythonWorkerFactory.scala | 4 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 38 ++++--------- .../ExecutorUncaughtExceptionHandler.scala | 53 +++++++++++++++++++ .../spark/storage/DiskBlockManager.scala | 6 +-- .../scala/org/apache/spark/util/Utils.scala | 27 +++++++++- 8 files changed, 96 insertions(+), 36 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e86e7923f8622..60f3aefb04f53 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1145,6 +1145,7 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) +<<<<<<< HEAD /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ private[spark] val executorMemoryRequested = { // TODO: Might need to add some extra memory for the non-heap parts of the JVM 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 12b4d94a567ce..2b9bf37ee7ff7 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 @@ -265,7 +265,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e => 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 67d45723badd8..4251e9d375f94 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 @@ -58,13 +58,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { new Socket(daemonHost, daemonPort) } catch { - case exc: SocketException => { + case exc: SocketException => logWarning("Python daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - } - case e => throw e } } } 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 fc1537f7963c4..530c079edd67a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -73,7 +73,7 @@ object SparkHadoopUtil { try { Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").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/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 893e05837040c..dbe8985915236 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -79,28 +79,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) @@ -258,6 +237,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).toInt val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -267,11 +251,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) - //System.exit(1) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { runningTasks.remove(taskId) 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/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index fcd2e97982955..cadb3a436bb03 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -125,14 +125,14 @@ 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") localDirs.foreach { localDir => 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/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fe932d8ede2f3..cd2100e1df2b3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,6 +26,7 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.io.Source +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -37,7 +38,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer import org.apache.spark.{SparkException, Logging} - /** * Various utility methods used by Spark. */ @@ -823,4 +823,29 @@ private[spark] object Utils extends Logging { return System.getProperties().clone() .asInstanceOf[java.util.Properties].toMap[String, String] } + + /** + * 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 7cfc5b4971d7879991817f4942d536fbb303dbdd Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 14 May 2014 10:07:25 -0700 Subject: [PATCH 2/5] [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler If the intended behavior was that uncaught exceptions thrown in functions being run by the Akka scheduler would end up being handled by the default uncaught exception handler set in Executor, and if that behavior is, in fact, correct, then this is a way to accomplish that. I'm not certain, though, that we shouldn't be doing something different to handle uncaught exceptions from some of these scheduled functions. In any event, this PR covers all of the cases I comment on in [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620). Author: Mark Hamstra Closes #622 from markhamstra/SPARK-1620 and squashes the following commits: 071d193 [Mark Hamstra] refactored post-SPARK-1772 1a6a35e [Mark Hamstra] another style fix d30eb94 [Mark Hamstra] scalastyle 3573ecd [Mark Hamstra] Use wrapped try/catch in Utils.tryOrExit 8fc0439 [Mark Hamstra] Make functions run by the Akka scheduler use Executor's UncaughtExceptionHandler (cherry picked from commit 17f3075bc4aa8cbed165f7b367f70e84b1bc8db9) Signed-off-by: Patrick Wendell Conflicts: core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala core/src/main/scala/org/apache/spark/util/Utils.scala --- .../apache/spark/deploy/client/Client.scala | 19 +++++++++++-------- .../apache/spark/deploy/worker/Worker.scala | 18 ++++++++++-------- .../apache/spark/storage/BlockManager.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 18 ++++++++++++++++++ 4 files changed, 40 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 77422f61ec901..0349919285cc1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -33,6 +33,7 @@ import org.apache.spark.Logging import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master +import org.apache.spark.util.Utils /** @@ -87,14 +88,16 @@ private[spark] class Client( var retries = 0 lazy val retryTimer: Cancellable = context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - retryTimer.cancel() - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - markDead() - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + retryTimer.cancel() + } else if (retries >= REGISTRATION_RETRIES) { + logError("All masters are unresponsive! Giving up.") + markDead() + } else { + tryRegisterAllMasters() + } } } retryTimer // start timer diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 216d9d44ac619..ba6e42408ba2c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -148,14 +148,16 @@ private[spark] class Worker( var retries = 0 lazy val retryTimer: Cancellable = context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - retryTimer.cancel() - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + logError("All masters are unresponsive! Giving up.") + System.exit(1) + } else { + tryRegisterAllMasters() + } } } retryTimer // start timer diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a34c95b6f07b6..dd78931a79e24 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -128,7 +128,7 @@ private[spark] class BlockManager( BlockManagerWorker.startBlockManagerWorker(this) if (!BlockManager.getDisableHeartBeatsForTesting) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - heartBeat() + Utils.tryOrExit { heartBeat() } } } } 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 cd2100e1df2b3..6e345ff968fae 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -31,8 +31,14 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +<<<<<<< HEAD import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} +======= +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorUncaughtExceptionHandler +>>>>>>> 9ff9078... [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer @@ -634,6 +640,18 @@ private[spark] object Utils extends Logging { output.toString } + /** + * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the + * default UncaughtExceptionHandler + */ + def tryOrExit(block: => Unit) { + try { + block + } catch { + case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + } + } + /** * A regular expression to match classes of the "core" Spark API that we want to skip when * finding the call site of a method. From 989e184461ee03cd4f30b27a63d7411d9e9e6d15 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 6 May 2014 12:53:39 -0700 Subject: [PATCH 3/5] [SPARK-1685] Cancel retryTimer on restart of Worker or AppClient See https://issues.apache.org/jira/browse/SPARK-1685 for a more complete description, but in essence: If the Worker or AppClient actor restarts before successfully registering with Master, multiple retryTimers will be running, which will lead to less than the full number of registration retries being attempted before the new actor is forced to give up. Author: Mark Hamstra Closes #602 from markhamstra/SPARK-1685 and squashes the following commits: 11cc088 [Mark Hamstra] retryTimer -> registrationRetryTimer 69c348c [Mark Hamstra] Cancel retryTimer on restart of Worker or AppClient Conflicts: core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala --- .../scala/org/apache/spark/deploy/client/Client.scala | 10 +++++++--- .../scala/org/apache/spark/deploy/worker/Worker.scala | 8 +++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 0349919285cc1..3549b13d02eee 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -62,6 +62,7 @@ private[spark] class Client( var masterAddress: Address = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times + var registrationRetryTimer: Option[Cancellable] = None override def preStart() { try { @@ -86,12 +87,12 @@ private[spark] class Client( tryRegisterAllMasters() var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { Utils.tryOrExit { retries += 1 if (registered) { - retryTimer.cancel() + registrationRetryTimer.foreach(_.cancel()) } else if (retries >= REGISTRATION_RETRIES) { logError("All masters are unresponsive! Giving up.") markDead() @@ -100,7 +101,7 @@ private[spark] class Client( } } } - retryTimer // start timer + } } def changeMaster(url: String) { @@ -177,6 +178,9 @@ private[spark] class Client( alreadyDead = true } } + + override def postStop() { + registrationRetryTimer.foreach(_.cancel()) } def start() { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ba6e42408ba2c..6b4a284038689 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -85,6 +85,8 @@ private[spark] class Worker( val metricsSystem = MetricsSystem.createMetricsSystem("worker") val workerSource = new WorkerSource(this) + var registrationRetryTimer: Option[Cancellable] = None + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -144,9 +146,8 @@ private[spark] class Worker( def registerWithMaster() { tryRegisterAllMasters() - var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { Utils.tryOrExit { retries += 1 @@ -160,7 +161,7 @@ private[spark] class Worker( } } } - retryTimer // start timer + } } override def receive = { @@ -262,6 +263,7 @@ private[spark] class Worker( } override def postStop() { + registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) webUi.stop() metricsSystem.stop() From 75c8638bd5219fe0b9b127fa91eb288b29aa1e29 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 14 May 2014 14:13:41 -0700 Subject: [PATCH 4/5] Fixed mis-merge --- .../main/scala/org/apache/spark/SparkContext.scala | 13 ++++++------- .../org/apache/spark/deploy/client/Client.scala | 1 + .../main/scala/org/apache/spark/util/Utils.scala | 13 ++++--------- 3 files changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 60f3aefb04f53..84295f64e02fe 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -218,8 +218,8 @@ class SparkContext( } 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, this.env.actorSystem) @@ -233,8 +233,8 @@ class SparkContext( cons.newInstance(this).asInstanceOf[ClusterScheduler] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -243,8 +243,8 @@ class SparkContext( val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) cons.newInstance(scheduler, this).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) } } @@ -1145,7 +1145,6 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) -<<<<<<< HEAD /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ private[spark] val executorMemoryRequested = { // TODO: Might need to add some extra memory for the non-heap parts of the JVM diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 3549b13d02eee..536eda5f72cea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -181,6 +181,7 @@ private[spark] class Client( override def postStop() { registrationRetryTimer.foreach(_.cancel()) + } } def start() { 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 6e345ff968fae..a24d5ca0b8caf 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} +import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} @@ -31,18 +32,12 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -<<<<<<< HEAD import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} -======= -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.ExecutorUncaughtExceptionHandler ->>>>>>> 9ff9078... [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import org.apache.spark.deploy.SparkHadoopUtil -import java.nio.ByteBuffer -import org.apache.spark.{SparkException, Logging} /** * Various utility methods used by Spark. @@ -852,7 +847,7 @@ private[spark] object Utils extends Logging { f } catch { case t: Throwable => - logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + logError("Uncaught exception in thread " + Thread.currentThread().getName, t) throw t } } @@ -860,7 +855,7 @@ private[spark] object Utils extends Logging { /** 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 => + case NonFatal(_) | _: InterruptedException | _: ControlThrowable => false case _ => true From 76dc266cfe5797f9ea34a931fca1c4711e5e7952 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 9 May 2014 21:50:23 -0700 Subject: [PATCH 5/5] SPARK-1686: keep schedule() calling in the main thread https://issues.apache.org/jira/browse/SPARK-1686 moved from original JIRA (by @markhamstra): In deploy.master.Master, the completeRecovery method is the last thing to be called when a standalone Master is recovering from failure. It is responsible for resetting some state, relaunching drivers, and eventually resuming its scheduling duties. There are currently four places in Master.scala where completeRecovery is called. Three of them are from within the actor's receive method, and aren't problems. The last starts from within receive when the ElectedLeader message is received, but the actual completeRecovery() call is made from the Akka scheduler. That means that it will execute on a different scheduler thread, and Master itself will end up running (i.e., schedule() ) from that Akka scheduler thread. In this PR, I added a new master message TriggerSchedule to trigger the "local" call of schedule() in the scheduler thread Author: CodingCat Closes #639 from CodingCat/SPARK-1686 and squashes the following commits: 81bb4ca [CodingCat] rename variable 69e0a2a [CodingCat] style fix 36a2ac0 [CodingCat] address Aaron's comments ec9b7bb [CodingCat] address the comments 02b37ca [CodingCat] keep schedule() calling in the main thread Conflicts: core/src/main/scala/org/apache/spark/deploy/master/Master.scala --- .../org/apache/spark/deploy/master/Master.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) 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 2b537c91b4c84..bab065b45ed19 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 @@ -85,6 +85,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var leaderElectionAgent: ActorRef = _ + private var recoveryCompletionTask: Cancellable = _ + // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. @@ -128,6 +130,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } override def postStop() { + // prevent the CompleteRecovery message sending to restarted master + if (recoveryCompletionTask != null) { + recoveryCompletionTask.cancel() + } webUi.stop() masterMetricsSystem.stop() applicationMetricsSystem.stop() @@ -147,10 +153,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedWorkers) - context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } + recoveryCompletionTask = context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, + CompleteRecovery) } } + case CompleteRecovery => completeRecovery() + case RevokedLeadership => { logError("Leadership has been revoked -- master shutting down.") System.exit(0) @@ -350,7 +359,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act * Schedule the currently available resources among waiting apps. This method will be called * every time a new app joins or resource availability changes. */ - def schedule() { + private def schedule() { if (state != RecoveryState.ALIVE) { return } // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. @@ -358,7 +367,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // Try to spread out each app among all the nodes, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(canUse(app, _)).sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)