Skip to content

Commit 3573ecd

Browse files
committed
Use wrapped try/catch in Utils.tryOrExit
1 parent 8fc0439 commit 3573ecd

File tree

6 files changed

+37
-25
lines changed

6 files changed

+37
-25
lines changed

core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException}
3030
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
3131
import org.apache.spark.deploy.DeployMessages._
3232
import org.apache.spark.deploy.master.Master
33-
import org.apache.spark.util.{UncaughtExceptionHandler, AkkaUtils}
33+
import org.apache.spark.util.{Utils, AkkaUtils}
3434

3535
/**
3636
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
@@ -88,14 +88,15 @@ private[spark] class AppClient(
8888
var retries = 0
8989
registrationRetryTimer = Some {
9090
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
91-
Thread.currentThread.setUncaughtExceptionHandler(UncaughtExceptionHandler)
92-
retries += 1
93-
if (registered) {
94-
registrationRetryTimer.foreach(_.cancel())
95-
} else if (retries >= REGISTRATION_RETRIES) {
96-
markDead("All masters are unresponsive! Giving up.")
97-
} else {
98-
tryRegisterAllMasters()
91+
Utils.tryOrExit {
92+
retries += 1
93+
if (registered) {
94+
registrationRetryTimer.foreach(_.cancel())
95+
} else if (retries >= REGISTRATION_RETRIES) {
96+
markDead("All masters are unresponsive! Giving up.")
97+
} else {
98+
tryRegisterAllMasters()
99+
}
99100
}
100101
}
101102
}

core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@ import org.apache.spark.deploy.DeployMessages._
3434
import org.apache.spark.deploy.master.{DriverState, Master}
3535
import org.apache.spark.deploy.worker.ui.WorkerWebUI
3636
import org.apache.spark.metrics.MetricsSystem
37-
import org.apache.spark.util.{UncaughtExceptionHandler, AkkaUtils, Utils}
37+
import org.apache.spark.util.{AkkaUtils, Utils}
3838

3939
/**
4040
* @param masterUrls Each url should look like spark://host:port.
@@ -166,15 +166,16 @@ private[spark] class Worker(
166166
var retries = 0
167167
registrationRetryTimer = Some {
168168
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
169-
Thread.currentThread.setUncaughtExceptionHandler(UncaughtExceptionHandler)
170-
retries += 1
171-
if (registered) {
172-
registrationRetryTimer.foreach(_.cancel())
173-
} else if (retries >= REGISTRATION_RETRIES) {
174-
logError("All masters are unresponsive! Giving up.")
175-
System.exit(1)
176-
} else {
177-
tryRegisterAllMasters()
169+
Utils.tryOrExit {
170+
retries += 1
171+
if (registered) {
172+
registrationRetryTimer.foreach(_.cancel())
173+
} else if (retries >= REGISTRATION_RETRIES) {
174+
logError("All masters are unresponsive! Giving up.")
175+
System.exit(1)
176+
} else {
177+
tryRegisterAllMasters()
178+
}
178179
}
179180
}
180181
}

core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import scala.util.Random
3131
import org.apache.spark._
3232
import org.apache.spark.TaskState.TaskState
3333
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
34-
import org.apache.spark.util.UncaughtExceptionHandler
34+
import org.apache.spark.util.Utils
3535

3636
/**
3737
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
@@ -140,8 +140,7 @@ private[spark] class TaskSchedulerImpl(
140140
import sc.env.actorSystem.dispatcher
141141
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
142142
SPECULATION_INTERVAL milliseconds) {
143-
Thread.currentThread.setUncaughtExceptionHandler(UncaughtExceptionHandler)
144-
checkSpeculatableTasks()
143+
Utils.tryOrExit { checkSpeculatableTasks() }
145144
}
146145
}
147146
}

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -155,8 +155,7 @@ private[spark] class BlockManager(
155155
BlockManagerWorker.startBlockManagerWorker(this)
156156
if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
157157
heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) {
158-
Thread.currentThread.setUncaughtExceptionHandler(UncaughtExceptionHandler)
159-
heartBeat()
158+
Utils.tryOrExit { heartBeat() }
160159
}
161160
}
162161
}

core/src/main/scala/org/apache/spark/util/UncaughtExceptionHandler.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ package org.apache.spark.util
2020
import org.apache.spark.executor.ExecutorExitCode
2121
import org.apache.spark.Logging
2222

23-
object UncaughtExceptionHandler extends Thread.UncaughtExceptionHandler with Logging {
23+
private[spark] object UncaughtExceptionHandler extends Thread.UncaughtExceptionHandler with Logging {
2424
override def uncaughtException(thread: Thread, exception: Throwable) {
2525
try {
2626
logError("Uncaught exception in thread " + thread, exception)

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -778,6 +778,18 @@ private[spark] object Utils extends Logging {
778778
output.toString
779779
}
780780

781+
/**
782+
* Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the
783+
* default UncaughtExceptionHandler
784+
*/
785+
def tryOrExit(block: => Unit) {
786+
try {
787+
block
788+
} catch {
789+
case t: Throwable => UncaughtExceptionHandler.uncaughtException(Thread.currentThread, t)
790+
}
791+
}
792+
781793
/**
782794
* A regular expression to match classes of the "core" Spark API that we want to skip when
783795
* finding the call site of a method.

0 commit comments

Comments
 (0)