Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
97523e0
[SPARK-6980] Akka ask timeout description refactored to RPC layer
BryanCutler May 15, 2015
78a2c0a
[SPARK-6980] Using RpcTimeout.awaitResult for future in AppClient now
BryanCutler May 16, 2015
5b59a44
[SPARK-6980] Added some RpcTimeout unit tests
BryanCutler May 16, 2015
49f9f04
[SPARK-6980] Minor cleanup and scala style fix
BryanCutler May 16, 2015
23d2f26
[SPARK-6980] Fixed await result not being handled by RpcTimeout
BryanCutler May 18, 2015
a294569
[SPARK-6980] Added creation of RpcTimeout with Seq of property keys
BryanCutler May 19, 2015
f74064d
Retrieving properties from property list using iterator and while loo…
May 21, 2015
0ee5642
Changing the loop condition to halt at the first match in the propert…
May 21, 2015
4be3a8d
Modifying loop condition to find property match
May 24, 2015
b7fb99f
Merge pull request #2 from hardmettle/configTimeoutUpdates_6980
BryanCutler May 24, 2015
c07d05c
Merge branch 'master' into configTimeout-6980-tmp
BryanCutler Jun 3, 2015
235919b
[SPARK-6980] Resolved conflicts after master merge
BryanCutler Jun 3, 2015
2f94095
[SPARK-6980] Added addMessageIfTimeout for when a Future is completed…
BryanCutler Jun 4, 2015
1607a5f
[SPARK-6980] Changed addMessageIfTimeout to PartialFunction, cleanup …
BryanCutler Jun 8, 2015
4351c48
[SPARK-6980] Added UT for addMessageIfTimeout, cleaned up UTs
BryanCutler Jun 10, 2015
7774d56
[SPARK-6980] Cleaned up UT imports
BryanCutler Jun 11, 2015
995d196
[SPARK-6980] Cleaned up import ordering, comments, spacing from PR fe…
BryanCutler Jun 11, 2015
d3754d1
[SPARK-6980] Added akkaConf to prevent dead letter logging
BryanCutler Jun 11, 2015
08f5afc
[SPARK-6980] Added UT for constructing RpcTimeout with default value
BryanCutler Jun 11, 2015
1b9beab
[SPARK-6980] Cleaned up import ordering
BryanCutler Jun 12, 2015
2206b4d
[SPARK-6980] Added unit test for ask then immediat awaitReply
BryanCutler Jun 12, 2015
1517721
[SPARK-6980] RpcTimeout object scope should be private[spark]
BryanCutler Jun 15, 2015
1394de6
[SPARK-6980] Moved MessagePrefix to createRpcTimeoutException directly
BryanCutler Jun 15, 2015
c6cfd33
[SPARK-6980] Changed UT ask message timeout to explicitly intercept a…
BryanCutler Jun 23, 2015
b05d449
[SPARK-6980] Changed constructor to use val duration instead of gette…
BryanCutler Jun 23, 2015
fa6ed82
[SPARK-6980] Had to increase timeout on positive test case because a …
BryanCutler Jun 23, 2015
fadaf6f
[SPARK-6980] Put back in deprecated RpcUtils askTimeout and lookupTim…
BryanCutler Jun 24, 2015
218aa50
[SPARK-6980] Corrected issues from feedback
BryanCutler Jun 24, 2015
039afed
[SPARK-6980] Corrected import organization
BryanCutler Jun 24, 2015
be11c4e
Merge branch 'master' into configTimeout-6980
BryanCutler Jun 24, 2015
7636189
[SPARK-6980] Fixed call to askWithReply in DAGScheduler to use RpcTim…
BryanCutler Jun 26, 2015
3a168c7
[SPARK-6980] Rewrote Akka RpcTimeout UTs in RpcEnvSuite
BryanCutler Jun 26, 2015
3d8b1ff
[SPARK-6980] Cleaned up imports in AkkaRpcEnvSuite
BryanCutler Jun 26, 2015
287059a
[SPARK-6980] Removed extra import in AkkaRpcEnvSuite
BryanCutler Jun 26, 2015
7f4d78e
[SPARK-6980] Fixed scala style checks
BryanCutler Jun 26, 2015
6a1c50d
[SPARK-6980] Minor cleanup of test case
BryanCutler Jun 27, 2015
4e89c75
[SPARK-6980] Missed one usage of deprecated RpcUtils.askTimeout in Ya…
BryanCutler Jun 30, 2015
dbd5f73
[SPARK-6980] Changed RpcUtils askRpcTimeout and lookupRpcTimeout scop…
BryanCutler Jul 1, 2015
7bb70f1
Merge branch 'master' into configTimeout-6980
BryanCutler Jul 1, 2015
06afa53
[SPARK-6980] RpcTimeout class extends Serializable, was causing error…
BryanCutler Jul 2, 2015
46c8d48
[SPARK-6980] Changed RpcEnvSuite test to never reply instead of just …
BryanCutler Jul 2, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Merge branch 'master' into configTimeout-6980
Conflicts:
	core/src/main/scala/org/apache/spark/deploy/Client.scala
	core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
	core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
	core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
	core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
	core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
	core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
	core/src/test/scala/org/apache/spark/rpc/akka/AkkaRpcEnvSuite.scala
  • Loading branch information
BryanCutler committed Jul 1, 2015
commit 7bb70f1b862099666784930ffefe92ce0fff1df3
11 changes: 4 additions & 7 deletions core/src/main/scala/org/apache/spark/deploy/Client.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,8 @@ private class ClientEndpoint(
System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
})

val timeout = RpcUtils.askRpcTimeout(conf)

override def preStart(): Unit = {
context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
private val lostMasters = new HashSet[RpcAddress]
private var activeMasterEndpoint: RpcEndpointRef = null

override def onStart(): Unit = {
driverArgs.cmd match {
Expand Down Expand Up @@ -123,9 +121,8 @@ private class ClientEndpoint(
println("... waiting before polling master for driver state")
Thread.sleep(5000)
println("... polling master for driver state")
val statusFuture = (activeMasterActor ? RequestDriverStatus(driverId))(timeout.duration)
.mapTo[DriverStatusResponse]
val statusResponse = timeout.awaitResult(statusFuture)
val statusResponse =
activeMasterEndpoint.askWithRetry[DriverStatusResponse](RequestDriverStatus(driverId))
statusResponse.found match {
case false =>
println(s"ERROR: Cluster master did not recognize $driverId")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -248,9 +248,7 @@ private[spark] class AppClient(
def stop() {
if (endpoint != null) {
try {
val timeout = RpcUtils.askRpcTimeout(conf)
val future = actor.ask(StopAppClient)(timeout.duration)
timeout.awaitResult(future)
endpoint.askWithRetry[Boolean](StopAppClient)
} catch {
case e: TimeoutException =>
logInfo("Stop request to Master timed out; it may already be shut down.")
Expand Down
14 changes: 5 additions & 9 deletions core/src/main/scala/org/apache/spark/deploy/master/Master.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import java.util.Date
import java.util.concurrent.{ScheduledFuture, TimeUnit}

import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.Random

Expand Down Expand Up @@ -928,13 +927,10 @@ private[deploy] object Master extends Logging {
webUiPort: Int,
conf: SparkConf): (RpcEnv, Int, Option[Int]) = {
val securityMgr = new SecurityManager(conf)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf,
securityManager = securityMgr)
val actor = actorSystem.actorOf(
Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName)
val timeout = RpcUtils.askRpcTimeout(conf)
val portsRequest = actor.ask(BoundPortsRequest)(timeout.duration)
val portsResponse = timeout.awaitResult(portsRequest).asInstanceOf[BoundPortsResponse]
(actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort)
val rpcEnv = RpcEnv.create(SYSTEM_NAME, host, port, conf, securityMgr)
val masterEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME,
new Master(rpcEnv, rpcEnv.address, webUiPort, securityMgr, conf))
val portsResponse = masterEndpoint.askWithRetry[BoundPortsResponse](BoundPortsRequest)
(rpcEnv, portsResponse.webUIPort, portsResponse.restPort)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
/** Executor details for a particular application */
def render(request: HttpServletRequest): Seq[Node] = {
val appId = request.getParameter("appId")
val stateFuture = (master ? RequestMasterState)(timeout.duration).mapTo[MasterStateResponse]
val state = timeout.awaitResult(stateFuture)
val state = master.askWithRetry[MasterStateResponse](RequestMasterState)
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
private val master = parent.masterEndpointRef

def getMasterState: MasterStateResponse = {
val stateFuture = (master ? RequestMasterState)(timeout.duration).mapTo[MasterStateResponse]
timeout.awaitResult(stateFuture)
master.askWithRetry[MasterStateResponse](RequestMasterState)
}

override def renderJson(request: HttpServletRequest): JValue = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,7 @@ class MasterWebUI(val master: Master, requestedPort: Int)
extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging
with UIRoot {

val masterActorRef = master.self
val timeout = RpcUtils.askRpcTimeout(master.conf)
val masterEndpointRef = master.self
val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true)

val masterPage = new MasterPage(this)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,9 +71,8 @@ private[rest] class StandaloneKillRequestServlet(masterEndpoint: RpcEndpointRef,
extends KillRequestServlet {

protected def handleKill(submissionId: String): KillSubmissionResponse = {
val askTimeout = RpcUtils.askRpcTimeout(conf)
val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse](
DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout)
val response = masterEndpoint.askWithRetry[DeployMessages.KillDriverResponse](
DeployMessages.RequestKillDriver(submissionId))
val k = new KillSubmissionResponse
k.serverSparkVersion = sparkVersion
k.message = response.message
Expand All @@ -90,9 +89,8 @@ private[rest] class StandaloneStatusRequestServlet(masterEndpoint: RpcEndpointRe
extends StatusRequestServlet {

protected def handleStatus(submissionId: String): SubmissionStatusResponse = {
val askTimeout = RpcUtils.askRpcTimeout(conf)
val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse](
DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout)
val response = masterEndpoint.askWithRetry[DeployMessages.DriverStatusResponse](
DeployMessages.RequestDriverStatus(submissionId))
val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) }
val d = new SubmissionStatusResponse
d.serverSparkVersion = sparkVersion
Expand Down Expand Up @@ -175,7 +173,6 @@ private[rest] class StandaloneSubmitRequestServlet(
responseServlet: HttpServletResponse): SubmitRestProtocolResponse = {
requestMessage match {
case submitRequest: CreateSubmissionRequest =>
val askTimeout = RpcUtils.askRpcTimeout(conf)
val driverDescription = buildDriverDescription(submitRequest)
val response = masterEndpoint.askWithRetry[DeployMessages.SubmitDriverResponse](
DeployMessages.RequestSubmitDriver(driverDescription))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,16 +33,12 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
private val workerEndpoint = parent.worker.self

override def renderJson(request: HttpServletRequest): JValue = {
val stateFuture = (workerActor ? RequestWorkerState)(timeout.duration).
mapTo[WorkerStateResponse]
val workerState = timeout.awaitResult(stateFuture)
val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState)
JsonProtocol.writeWorkerState(workerState)
}

def render(request: HttpServletRequest): Seq[Node] = {
val stateFuture = (workerActor ? RequestWorkerState)(timeout.duration).
mapTo[WorkerStateResponse]
val workerState = timeout.awaitResult(stateFuture)
val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState)

val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs")
val runningExecutors = workerState.executors
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.spark.rpc.akka

import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.rpc._
import org.apache.spark.{SSLSampleConfigs, SecurityManager, SparkConf}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: delete extra newline

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

class AkkaRpcEnvSuite extends RpcEnvSuite {

Expand Down
You are viewing a condensed version of this merge commit. You can view the full changes here.