Skip to content

Commit

Permalink
Keep only WeakReference to promise in TimerTask;
Browse files Browse the repository at this point in the history
Add additional documentation.
  • Loading branch information
JoshRosen committed Nov 14, 2014
1 parent 0f0913b commit 2a2e92d
Showing 1 changed file with 22 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.network.nio

import java.io.IOException
import java.lang.ref.WeakReference
import java.net._
import java.nio._
import java.nio.channels._
Expand Down Expand Up @@ -140,7 +141,10 @@ private[nio] class ConnectionManager(
new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection]
private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection]
with SynchronizedMap[ConnectionManagerId, SendingConnection]
private val messageStatuses = new HashMap[Int, MessageStatus]
// Tracks sent messages for which we are awaiting acknowledgements. Entries are added to this
// map when messages are sent and are removed when acknowledgement messages are received or when
// acknowledgement timeouts expire
private val messageStatuses = new HashMap[Int, MessageStatus] // [MessageId, MessageStatus]
private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
private val registerRequests = new SynchronizedQueue[SendingConnection]

Expand Down Expand Up @@ -901,17 +905,30 @@ private[nio] class ConnectionManager(
val promise = Promise[Message]()

// It's important that the TimerTask doesn't capture a reference to `message`, which can cause
// memory leaks since cancelled TimerTasks won't necessarily be garbage collected until they are
// scheduled to run. Therefore, extract the message id from outside of the task:
// memory leaks since cancelled TimerTasks won't necessarily be garbage collected until the time
// at which they would originally be scheduled to run. Therefore, extract the message id
// from outside of the TimerTask closure (see SPARK-4393 for more context).
val messageId = message.id
// Keep a weak reference to the promise so that the completed promise may be garbage-collected
val promiseReference = new WeakReference(promise)
val timeoutTask: TimerTask = new TimerTask {
override def run(timeout: Timeout): Unit = {
messageStatuses.synchronized {
messageStatuses.remove(messageId).foreach ( s => {
val e = new IOException("sendMessageReliably failed because ack " +
s"was not received within $ackTimeout sec")
if (!promise.tryFailure(e)) {
logWarning("Ignore error because promise is completed", e)
Option(promiseReference.get) match {
case Some(p) =>
// Attempt to fail the promise with a Timeout exception
if (!p.tryFailure(e)) {
// If we reach here, then someone else has already signalled success or failure
// on this promise, so log a warning:
logError("Ignore error because promise is completed", e)
}
case None =>
// The WeakReference was empty, which should never happen because
// sendMessageReliably's caller should have a strong reference to promise.future;
logError("Promise was garbage collected; this should never happen!", e)
}
})
}
Expand Down

0 comments on commit 2a2e92d

Please sign in to comment.