Skip to content

Commit fbfe69d

Browse files
markhamstramateiz
authored andcommitted
[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 <markhamstra@gmail.com> 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
1 parent 7b978c1 commit fbfe69d

File tree

2 files changed

+15
-8
lines changed

2 files changed

+15
-8
lines changed

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

Lines changed: 9 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@ private[spark] class AppClient(
6060
var master: ActorSelection = null
6161
var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times
6262
var alreadyDead = false // To avoid calling listener.dead() multiple times
63+
var registrationRetryTimer: Option[Cancellable] = None
6364

6465
override def preStart() {
6566
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
@@ -83,21 +84,20 @@ private[spark] class AppClient(
8384

8485
def registerWithMaster() {
8586
tryRegisterAllMasters()
86-
8787
import context.dispatcher
8888
var retries = 0
89-
lazy val retryTimer: Cancellable =
89+
registrationRetryTimer = Some {
9090
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
9191
retries += 1
9292
if (registered) {
93-
retryTimer.cancel()
93+
registrationRetryTimer.foreach(_.cancel())
9494
} else if (retries >= REGISTRATION_RETRIES) {
9595
markDead("All masters are unresponsive! Giving up.")
9696
} else {
9797
tryRegisterAllMasters()
9898
}
9999
}
100-
retryTimer // start timer
100+
}
101101
}
102102

103103
def changeMaster(url: String) {
@@ -177,6 +177,11 @@ private[spark] class AppClient(
177177
alreadyDead = true
178178
}
179179
}
180+
181+
override def postStop() {
182+
registrationRetryTimer.foreach(_.cancel())
183+
}
184+
180185
}
181186

182187
def start() {

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

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -100,6 +100,8 @@ private[spark] class Worker(
100100
val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr)
101101
val workerSource = new WorkerSource(this)
102102

103+
var registrationRetryTimer: Option[Cancellable] = None
104+
103105
def coresFree: Int = cores - coresUsed
104106
def memoryFree: Int = memory - memoryUsed
105107

@@ -161,21 +163,20 @@ private[spark] class Worker(
161163

162164
def registerWithMaster() {
163165
tryRegisterAllMasters()
164-
165166
var retries = 0
166-
lazy val retryTimer: Cancellable =
167+
registrationRetryTimer = Some {
167168
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
168169
retries += 1
169170
if (registered) {
170-
retryTimer.cancel()
171+
registrationRetryTimer.foreach(_.cancel())
171172
} else if (retries >= REGISTRATION_RETRIES) {
172173
logError("All masters are unresponsive! Giving up.")
173174
System.exit(1)
174175
} else {
175176
tryRegisterAllMasters()
176177
}
177178
}
178-
retryTimer // start timer
179+
}
179180
}
180181

181182
override def receive = {
@@ -344,6 +345,7 @@ private[spark] class Worker(
344345
}
345346

346347
override def postStop() {
348+
registrationRetryTimer.foreach(_.cancel())
347349
executors.values.foreach(_.kill())
348350
drivers.values.foreach(_.kill())
349351
webUi.stop()

0 commit comments

Comments
 (0)