diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index a074ab8ece1b7c4485e2085f7b17a0e1788cb386..6e4edc7c80d7ae788f1af622e68ce27b51034321 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -76,6 +76,8 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
  */
 private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging {
   private val timeout = AkkaUtils.askTimeout(conf)
+  private val retryAttempts = AkkaUtils.numRetries(conf)
+  private val retryIntervalMs = AkkaUtils.retryWaitMs(conf)
 
   /** Set to the MapOutputTrackerActor living on the driver. */
   var trackerActor: ActorRef = _
@@ -108,8 +110,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
    */
   protected def askTracker(message: Any): Any = {
     try {
-      val future = trackerActor.ask(message)(timeout)
-      Await.result(future, timeout)
+      AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout)
     } catch {
       case e: Exception =>
         logError("Error communicating with MapOutputTracker", e)