-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-6980] [CORE] Akka timeout exceptions indicate which conf controls them (RPC Layer) #6205
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
97523e0
78a2c0a
5b59a44
49f9f04
23d2f26
a294569
f74064d
0ee5642
4be3a8d
b7fb99f
c07d05c
235919b
2f94095
1607a5f
4351c48
7774d56
995d196
d3754d1
08f5afc
1b9beab
2206b4d
1517721
1394de6
c6cfd33
b05d449
fa6ed82
fadaf6f
218aa50
039afed
be11c4e
7636189
3a168c7
3d8b1ff
287059a
7f4d78e
6a1c50d
4e89c75
dbd5f73
7bb70f1
06afa53
46c8d48
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,7 +18,10 @@ | |
| package org.apache.spark.rpc | ||
|
|
||
| import java.net.URI | ||
| import java.util.concurrent.TimeoutException | ||
|
|
||
| import scala.concurrent.duration.FiniteDuration | ||
| import scala.concurrent.duration._ | ||
| import scala.concurrent.{Await, Future} | ||
| import scala.language.postfixOps | ||
|
|
||
|
|
@@ -94,7 +97,7 @@ private[spark] abstract class RpcEnv(conf: SparkConf) { | |
| * Retrieve the [[RpcEndpointRef]] represented by `uri`. This is a blocking action. | ||
| */ | ||
| def setupEndpointRefByURI(uri: String): RpcEndpointRef = { | ||
| Await.result(asyncSetupEndpointRefByURI(uri), defaultLookupTimeout) | ||
| Await.result(asyncSetupEndpointRefByURI(uri), defaultLookupTimeout.duration) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -182,3 +185,68 @@ private[spark] object RpcAddress { | |
| RpcAddress(host, port) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Associates a timeout with a configuration property so that a TimeoutException can be | ||
| * traced back to the controlling property. | ||
| * @param timeout timeout duration in seconds | ||
| * @param description description to be displayed in a timeout exception | ||
| */ | ||
| private[spark] class RpcTimeout(timeout: FiniteDuration, description: String) { | ||
|
|
||
| /** Get the timeout duration */ | ||
| def duration: FiniteDuration = timeout | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this necessary or should we just use
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah I think you should just change the constructor
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure, that sounds good. One other minor question @squito , the constructor has the parameter class RpcTimeout(val duration: FiniteDuration, val timeoutProp: String)what do you think?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, I agree that would be more consistent, good point |
||
|
|
||
| /** Get the message associated with this timeout */ | ||
| def message: String = description | ||
|
|
||
| /** Amends the standard message of TimeoutException to include the description */ | ||
| def amend(te: TimeoutException): TimeoutException = { | ||
| new TimeoutException(te.getMessage() + " " + description) | ||
| } | ||
|
|
||
| /** Wait on a future result to catch and amend a TimeoutException */ | ||
| def awaitResult[T](future: Future[T]): T = { | ||
| try { | ||
| Await.result(future, duration) | ||
| } | ||
| catch { | ||
| case te: TimeoutException => | ||
| throw amend(te) | ||
| } | ||
| } | ||
|
|
||
| // TODO(bryanc) wrap Await.ready also | ||
| } | ||
|
|
||
| object RpcTimeout { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
|
|
||
| private[this] val messagePrefix = "This timeout is controlled by " | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. instead of putting The reason I'm asking is because in the tests, where you directly construct an
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 I don't think this is the case anymore and maybe we should use
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 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 I do kinda wish that the values were read w/
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. in fact we could also mark the constructor as
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see what you're saying. I kept the Quick Scala question: would it be better to make the constructor use
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
|
|
||
| /** | ||
| * Lookup the timeout property in the configuration and create | ||
| * a RpcTimeout with the property key in the description. | ||
| * @param conf configuration properties containing the timeout | ||
| * @param timeoutProp property key for the timeout in seconds | ||
| * @throws NoSuchElementException if property is not set | ||
| */ | ||
| def apply(conf: SparkConf, timeoutProp: String): RpcTimeout = { | ||
| val timeout = { conf.getTimeAsSeconds(timeoutProp) seconds } | ||
| new RpcTimeout(timeout, messagePrefix + timeoutProp) | ||
| } | ||
|
|
||
| /** | ||
| * Lookup the timeout property in the configuration and create | ||
| * a RpcTimeout with the property key in the description. | ||
| * Uses the given default value if property is not set | ||
| * @param conf configuration properties containing the timeout | ||
| * @param timeoutProp property key for the timeout in seconds | ||
| * @param defaultValue default timeout value in seconds if property not found | ||
| */ | ||
| def apply(conf: SparkConf, timeoutProp: String, defaultValue: String): RpcTimeout = { | ||
| val timeout = { conf.getTimeAsSeconds(timeoutProp, defaultValue) seconds } | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should this be
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
| new RpcTimeout(timeout, messagePrefix + timeoutProp) | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,7 +20,6 @@ package org.apache.spark.rpc.akka | |
| import java.util.concurrent.ConcurrentHashMap | ||
|
|
||
| import scala.concurrent.Future | ||
| import scala.concurrent.duration._ | ||
| import scala.language.postfixOps | ||
| import scala.reflect.ClassTag | ||
| import scala.util.control.NonFatal | ||
|
|
@@ -212,7 +211,7 @@ private[spark] class AkkaRpcEnv private[akka] ( | |
|
|
||
| override def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { | ||
| import actorSystem.dispatcher | ||
| actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout). | ||
| actorSystem.actorSelection(uri).resolveOne(defaultLookupTimeout.duration). | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. though this doesn't follow the same pattern of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I feel there are two ways :
In my opinion 2nd one is the best solution as the previous one will require modifications where ever the usages are.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The timeout is a little trickier for Futures. From what I understand, creating the future is non-blocking, so we can't just call I think we might be able to use the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what you are describing sounds reasonable. We certainly don't want to change the return type or await on the result here. You could potentially fix the doubly-appended msg by throwing a subclass of (Honestly I thiink it will also be fine to not stress too much about this case, it may not be worth it.)
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm going to do some research on this and see what our options are while still keeping the same return type. |
||
| map(new AkkaRpcEndpointRef(defaultAddress, _, conf)) | ||
| } | ||
|
|
||
|
|
@@ -293,9 +292,9 @@ private[akka] class AkkaRpcEndpointRef( | |
| actorRef ! AkkaMessage(message, false) | ||
| } | ||
|
|
||
| override def ask[T: ClassTag](message: Any, timeout: FiniteDuration): Future[T] = { | ||
| override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { | ||
| import scala.concurrent.ExecutionContext.Implicits.global | ||
| actorRef.ask(AkkaMessage(message, true))(timeout).flatMap { | ||
| actorRef.ask(AkkaMessage(message, true))(timeout.duration).flatMap { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same thing here about catching the timeout |
||
| case msg @ AkkaMessage(message, reply) => | ||
| if (reply) { | ||
| logError(s"Receive $msg but the sender cannot reply") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,9 +17,9 @@ | |
|
|
||
| package org.apache.spark.util | ||
|
|
||
| import org.apache.spark.rpc.RpcTimeout | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: ordering
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
|
|
||
| import scala.collection.JavaConversions.mapAsJavaMap | ||
| import scala.concurrent.Await | ||
| import scala.concurrent.duration.FiniteDuration | ||
|
|
||
| import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} | ||
| import akka.pattern.ask | ||
|
|
@@ -147,7 +147,7 @@ private[spark] object AkkaUtils extends Logging { | |
| def askWithReply[T]( | ||
| message: Any, | ||
| actor: ActorRef, | ||
| timeout: FiniteDuration): T = { | ||
| timeout: RpcTimeout): T = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I suggest reverting the changes to
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hmm, I disagree. No harm in making these changes. If we're really certain Maybe it wasn't worth the effort in the first place (my fault!) but I dont' see the harm ...
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm OK with the changes. Not a big deal. |
||
| askWithReply[T](message, actor, maxAttempts = 1, retryInterval = Int.MaxValue, timeout) | ||
| } | ||
|
|
||
|
|
@@ -160,7 +160,7 @@ private[spark] object AkkaUtils extends Logging { | |
| actor: ActorRef, | ||
| maxAttempts: Int, | ||
| retryInterval: Long, | ||
| timeout: FiniteDuration): T = { | ||
| timeout: RpcTimeout): T = { | ||
| // TODO: Consider removing multiple attempts | ||
| if (actor == null) { | ||
| throw new SparkException(s"Error sending message [message = $message]" + | ||
|
|
@@ -171,8 +171,8 @@ private[spark] object AkkaUtils extends Logging { | |
| while (attempts < maxAttempts) { | ||
| attempts += 1 | ||
| try { | ||
| val future = actor.ask(message)(timeout) | ||
| val result = Await.result(future, timeout) | ||
| val future = actor.ask(message)(timeout.duration) | ||
| val result = timeout.awaitResult(future) | ||
| if (result == null) { | ||
| throw new SparkException("Actor returned null") | ||
| } | ||
|
|
@@ -200,7 +200,7 @@ private[spark] object AkkaUtils extends Logging { | |
| val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) | ||
| val timeout = RpcUtils.lookupTimeout(conf) | ||
| logInfo(s"Connecting to $name: $url") | ||
| Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) | ||
| timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration)) | ||
| } | ||
|
|
||
| def makeExecutorRef( | ||
|
|
@@ -214,7 +214,7 @@ private[spark] object AkkaUtils extends Logging { | |
| val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) | ||
| val timeout = RpcUtils.lookupTimeout(conf) | ||
| logInfo(s"Connecting to $name: $url") | ||
| Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) | ||
| timeout.awaitResult(actorSystem.actorSelection(url).resolveOne(timeout.duration)) | ||
| } | ||
|
|
||
| def protocol(actorSystem: ActorSystem): String = { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
any reason not to use
timeout.awaitResulthere, so we get the better exception msg?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nope - I must have missed that one, it's fixed now.