Skip to content

Commit

Permalink
[SPARK-1620] Handle uncaught exceptions in function run by Akka sched…
Browse files Browse the repository at this point in the history
…uler

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 <[email protected]>

Closes apache#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
  • Loading branch information
markhamstra authored and pwendell committed May 14, 2014
1 parent d58cb33 commit 17f3075
Show file tree
Hide file tree
Showing 5 changed files with 36 additions and 18 deletions.
18 changes: 10 additions & 8 deletions core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException}
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.AkkaUtils
import org.apache.spark.util.{Utils, AkkaUtils}

/**
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
Expand Down Expand Up @@ -88,13 +88,15 @@ private[spark] class AppClient(
var retries = 0
registrationRetryTimer = Some {
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
retries += 1
if (registered) {
registrationRetryTimer.foreach(_.cancel())
} else if (retries >= REGISTRATION_RETRIES) {
markDead("All masters are unresponsive! Giving up.")
} else {
tryRegisterAllMasters()
Utils.tryOrExit {
retries += 1
if (registered) {
registrationRetryTimer.foreach(_.cancel())
} else if (retries >= REGISTRATION_RETRIES) {
markDead("All masters are unresponsive! Giving up.")
} else {
tryRegisterAllMasters()
}
}
}
}
Expand Down
18 changes: 10 additions & 8 deletions core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -166,14 +166,16 @@ private[spark] class Worker(
var retries = 0
registrationRetryTimer = Some {
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
retries += 1
if (registered) {
registrationRetryTimer.foreach(_.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()
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import scala.util.Random
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.util.Utils

/**
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
Expand Down Expand Up @@ -139,7 +140,7 @@ private[spark] class TaskSchedulerImpl(
import sc.env.actorSystem.dispatcher
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
SPECULATION_INTERVAL milliseconds) {
checkSpeculatableTasks()
Utils.tryOrExit { checkSpeculatableTasks() }
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ private[spark] class BlockManager(
BlockManagerWorker.startBlockManagerWorker(this)
if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) {
heartBeat()
Utils.tryOrExit { heartBeat() }
}
}
}
Expand Down
13 changes: 13 additions & 0 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import tachyon.client.{TachyonFile,TachyonFS}

import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.executor.ExecutorUncaughtExceptionHandler
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}

/**
Expand Down Expand Up @@ -780,6 +781,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.
Expand Down

0 comments on commit 17f3075

Please sign in to comment.