Skip to content

Commit 8fc0439

Browse files
committed
Make functions run by the Akka scheduler use Executor's UncaughtExceptionHandler
1 parent d9c97ba commit 8fc0439

File tree

6 files changed

+50
-3
lines changed

6 files changed

+50
-3
lines changed

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

Lines changed: 2 additions & 1 deletion
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.AkkaUtils
33+
import org.apache.spark.util.{UncaughtExceptionHandler, AkkaUtils}
3434

3535
/**
3636
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
@@ -88,6 +88,7 @@ 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)
9192
retries += 1
9293
if (registered) {
9394
registrationRetryTimer.foreach(_.cancel())

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

Lines changed: 2 additions & 1 deletion
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.{AkkaUtils, Utils}
37+
import org.apache.spark.util.{UncaughtExceptionHandler, AkkaUtils, Utils}
3838

3939
/**
4040
* @param masterUrls Each url should look like spark://host:port.
@@ -166,6 +166,7 @@ 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)
169170
retries += 1
170171
if (registered) {
171172
registrationRetryTimer.foreach(_.cancel())

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.spark._
2929
import org.apache.spark.deploy.SparkHadoopUtil
3030
import org.apache.spark.scheduler._
3131
import org.apache.spark.storage.{StorageLevel, TaskResultBlockId}
32-
import org.apache.spark.util.{AkkaUtils, Utils}
32+
import org.apache.spark.util.{UncaughtExceptionHandler, AkkaUtils, Utils}
3333

3434
/**
3535
* Spark executor used with Mesos, YARN, and the standalone scheduler.

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

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +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
3435

3536
/**
3637
* Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
@@ -139,6 +140,7 @@ private[spark] class TaskSchedulerImpl(
139140
import sc.env.actorSystem.dispatcher
140141
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
141142
SPECULATION_INTERVAL milliseconds) {
143+
Thread.currentThread.setUncaughtExceptionHandler(UncaughtExceptionHandler)
142144
checkSpeculatableTasks()
143145
}
144146
}

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -155,6 +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)
158159
heartBeat()
159160
}
160161
}
Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,42 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.util
19+
20+
import org.apache.spark.executor.ExecutorExitCode
21+
import org.apache.spark.Logging
22+
23+
object UncaughtExceptionHandler extends Thread.UncaughtExceptionHandler with Logging {
24+
override def uncaughtException(thread: Thread, exception: Throwable) {
25+
try {
26+
logError("Uncaught exception in thread " + thread, exception)
27+
28+
// We may have been called from a shutdown hook. If so, we must not call System.exit().
29+
// (If we do, we will deadlock.)
30+
if (!Utils.inShutdown()) {
31+
if (exception.isInstanceOf[OutOfMemoryError]) {
32+
System.exit(ExecutorExitCode.OOM)
33+
} else {
34+
System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION)
35+
}
36+
}
37+
} catch {
38+
case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM)
39+
case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE)
40+
}
41+
}
42+
}

0 commit comments

Comments
 (0)