Skip to content

Commit 3200c33

Browse files
committed
Use Netty HashedWheelTimer
1 parent f847dd4 commit 3200c33

File tree

1 file changed

+9
-7
lines changed

1 file changed

+9
-7
lines changed

core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala

Lines changed: 9 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -24,14 +24,14 @@ import java.nio.channels._
2424
import java.nio.channels.spi._
2525
import java.util.concurrent.atomic.AtomicInteger
2626
import java.util.concurrent.{LinkedBlockingDeque, ThreadPoolExecutor, TimeUnit}
27-
import java.util.{Timer, TimerTask}
2827

2928
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, SynchronizedMap, SynchronizedQueue}
3029
import scala.concurrent.duration._
3130
import scala.concurrent.{Await, ExecutionContext, Future, Promise}
3231
import scala.language.postfixOps
3332

3433
import com.google.common.base.Charsets.UTF_8
34+
import io.netty.util.{Timeout, TimerTask, HashedWheelTimer}
3535

3636
import org.apache.spark._
3737
import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer}
@@ -77,7 +77,8 @@ private[nio] class ConnectionManager(
7777
}
7878

7979
private val selector = SelectorProvider.provider.openSelector()
80-
private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true)
80+
private val ackTimeoutMonitor =
81+
new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor"))
8182

8283
private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60)
8384

@@ -903,8 +904,8 @@ private[nio] class ConnectionManager(
903904
// memory leaks since cancelled TimerTasks won't necessarily be garbage collected until they are
904905
// scheduled to run. Therefore, extract the message id from outside of the task:
905906
val messageId = message.id
906-
val timeoutTask = new TimerTask {
907-
override def run(): Unit = {
907+
val timeoutTask: TimerTask = new TimerTask {
908+
override def run(timeout: Timeout): Unit = {
908909
messageStatuses.synchronized {
909910
messageStatuses.remove(messageId).foreach ( s => {
910911
val e = new IOException("sendMessageReliably failed because ack " +
@@ -917,8 +918,10 @@ private[nio] class ConnectionManager(
917918
}
918919
}
919920

921+
val timoutTaskHandle = ackTimeoutMonitor.newTimeout(timeoutTask, ackTimeout, TimeUnit.SECONDS)
922+
920923
val status = new MessageStatus(message, connectionManagerId, s => {
921-
timeoutTask.cancel()
924+
timoutTaskHandle.cancel()
922925
s match {
923926
case scala.util.Failure(e) =>
924927
// Indicates a failure where we either never sent or never got ACK'd
@@ -947,7 +950,6 @@ private[nio] class ConnectionManager(
947950
messageStatuses += ((message.id, status))
948951
}
949952

950-
ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000)
951953
sendMessage(connectionManagerId, message)
952954
promise.future
953955
}
@@ -957,7 +959,7 @@ private[nio] class ConnectionManager(
957959
}
958960

959961
def stop() {
960-
ackTimeoutMonitor.cancel()
962+
ackTimeoutMonitor.stop()
961963
selectorThread.interrupt()
962964
selectorThread.join()
963965
selector.close()

0 commit comments

Comments
 (0)