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
[SPARK-6980] Minor cleanup and scala style fix
  • Loading branch information
BryanCutler committed May 16, 2015
commit 49f9f04e13c25d86f7f058ff757c098cc9e60ebb
Original file line number Diff line number Diff line change
Expand Up @@ -35,13 +35,15 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
private val timeout = parent.timeout

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

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

val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs")
Expand Down
15 changes: 8 additions & 7 deletions core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -188,8 +188,8 @@ private[spark] object RpcAddress {


/**
* Associates a timeout with a configuration property so that a TimeoutException can be
* traced back to the controlling property.
* Associates a timeout with a description so that a when a TimeoutException occurs, additional
* context about the timeout can be amended to the exception message.
* @param timeout timeout duration in seconds
* @param description description to be displayed in a timeout exception
*/
Expand All @@ -212,14 +212,16 @@ private[spark] class RpcTimeout(timeout: FiniteDuration, description: String) {
Await.result(future, duration)
}
catch {
case te: TimeoutException =>
throw amend(te)
case te: TimeoutException => throw amend(te)
}
}

// TODO(bryanc) wrap Await.ready also
}


/**
* Create an RpcTimeout using a configuration property that controls the timeout duration so when
* a TimeoutException is thrown, the property key will be indicated in the message.
*/
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this comment can be deleted, it doesn't really apply to object RpcTimeout, really its for the individual apply methods which you've already got good docs for.

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

object RpcTimeout {
Copy link
Contributor

Choose a reason for hiding this comment

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

private[spark]

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


private[this] val messagePrefix = "This timeout is controlled by "
Copy link
Contributor

Choose a reason for hiding this comment

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

instead of putting messagePrefix in each of the apply methods, what if its just used by RpcTimeout itself when building its description? Then the second arg to the RpctTimeout constructor could just be the property, not the full description.

The reason I'm asking is because in the tests, where you directly construct an RpcTimeout, the messagePrefix is missing, so if you look at the exception its kinda weird. And I think that the real msgs are missing a "." before the message prefix, so it would be nice to look at in the test.

Copy link
Member Author

Choose a reason for hiding this comment

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

That's a good point about the message in the tests. Way back when I first made the unit test for AkkaUtils the config was being read using conf.getTimeAsSeconds like here and if I tried to configure the tests in the millisecond range, it wouldn't work. Because of that I couldn't create a RpcTimeout with apply and just created a simple message without the messagePrefix.

I don't think this is the case anymore and maybe we should use conf.getTimeAsMs to read the property values? I think it would then automatically convert seconds to milliseconds, and I could fix the tests to use this instead and get the proper message.

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think this is related to whether or not we get the time as seconds or millis. I'm thinking that we still leave the FiniteDuration in the constructor, but just change that second param to the constructor to be just the name of the conf, not the full description. eg. :

diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
index 5149cf1..e0de397 100644
--- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala
@@ -197,19 +197,16 @@ private[rpc] class RpcTimeoutException(message: String, cause: TimeoutException)
  * Associates a timeout with a description so that a when a TimeoutException occurs, additional
  * context about the timeout can be amended to the exception message.
  * @param timeout timeout duration in seconds
- * @param description description to be displayed in a timeout exception
+ * @param conf the configuration parameter that controls this timeout
  */
-private[spark] class RpcTimeout(timeout: FiniteDuration, description: String) {
+private[spark] class RpcTimeout(timeout: FiniteDuration, val conf: String) {

   /** Get the timeout duration */
   def duration: FiniteDuration = timeout

-  /** Get the message associated with this timeout */
-  def message: String = description
-
   /** Amends the standard message of TimeoutException to include the description */
   private def createRpcTimeoutException(te: TimeoutException): RpcTimeoutException = {
-    new RpcTimeoutException(te.getMessage() + " " + description, te)
+    new RpcTimeoutException(te.getMessage() + ". This timeout is controlled by " + conf, te)
   }

(and corresponding changes to the apply methods.)

I do kinda wish that the values were read w/ conf.getTimeAsMs to allow finer grained resolution, but (a) I'd be nervous about changing any default and (b) in any case that should just be a separate issue.

Copy link
Contributor

Choose a reason for hiding this comment

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

in fact we could also mark the constructor as private[rpc] to be clear that normal use should really go through the RpcTimeout.apply methods.

Copy link
Member Author

Choose a reason for hiding this comment

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

I see what you're saying. I kept the messagePrefix separated so that if there was ever a need to create a RpcTimeout without a conf, it would still be possible and the message could be fit accordingly. But not a big deal, I'll put in the above patch. Right now it's always tied to a conf, so it does make sense.

Quick Scala question: would it be better to make the constructor use val timeout: FiniteDuration and remove that getter method too?

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

Expand Down Expand Up @@ -248,5 +250,4 @@ object RpcTimeout {
val timeout = { conf.getTimeAsSeconds(timeoutProp, defaultValue) seconds }
Copy link
Member Author

Choose a reason for hiding this comment

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

Should this be conf.getTimeAsMs instead?

Copy link
Contributor

Choose a reason for hiding this comment

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

as mentioned above, I don't think so -- lets stick w/ the current behavior of using conf.getTimeAsSeconds for now.

new RpcTimeout(timeout, messagePrefix + timeoutProp)
}

}