diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index dd7dcae6194b8fddfdcafbef59f3c79a5965a5ec..e918a33d09deaef20ac0fcfe10f99436b1258247 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -35,7 +35,7 @@ import org.apache.spark.deploy.DeployMessages._
 import org.apache.spark.deploy.master.ui.MasterWebUI
 import org.apache.spark.metrics.MetricsSystem
 import org.apache.spark.util.{Utils, AkkaUtils}
-import akka.util.Timeout
+import akka.util.{Duration, Timeout}
 import java.util.concurrent.TimeUnit
 
 
@@ -389,9 +389,11 @@ private[spark] object Master {
   def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
     val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName)
-    implicit val timeout = Timeout(5 seconds)
+    val timeoutDuration = Duration.create(
+      System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+    implicit val timeout = Timeout(timeoutDuration)
     val respFuture = actor ? RequestWebUIPort   // ask pattern
-    val resp = Await.result(respFuture, timeout.duration).asInstanceOf[WebUIPortResponse]
+    val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse]
     (actorSystem, boundPort, resp.webUIBoundPort)
   }
 }