-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-6980] [CORE] [WIP] Akka timeout exceptions indicate which conf controls them #5741
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 all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,73 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.streaming | ||
|
|
||
| import akka.actor.{Actor, Props} | ||
| import org.apache.spark._ | ||
| import org.apache.spark.util.{AkkaUtils, ConfiguredTimeout} | ||
| import org.scalatest.concurrent.Timeouts | ||
| import org.scalatest.{BeforeAndAfter, FunSuite} | ||
|
|
||
|
|
||
| class EchoActor extends Actor { | ||
| def receive: Receive = { | ||
| case msg => | ||
| Thread.sleep(1200) | ||
| sender() ! msg | ||
| } | ||
| } | ||
|
|
||
|
|
||
| class ActorSystemSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { | ||
|
|
||
| val master = "local[2]" | ||
| val appName = this.getClass.getSimpleName | ||
| val batchDuration = Milliseconds(500) | ||
| val sparkHome = "someDir" | ||
| val envPair = "key" -> "value" | ||
|
|
||
| var ssc: StreamingContext = null | ||
|
|
||
| test("actor askWithReply using ConfiguredTimeout") { | ||
| val conf = new SparkConf() | ||
|
|
||
| val shortProp = "spark.ask.short.timeout" | ||
|
|
||
| conf.set(shortProp, "1s") | ||
|
|
||
| ssc = new StreamingContext(master, appName, batchDuration) | ||
|
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. you don't need to make a streamingContext for this test at all, it just involves akka actors. In fact, I think this test should get moved to AkkaUtilsSuite as well.
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. No problem, in my initial tests I created the Actors from the ActorSystem, so I thought that was required. I'll rework this into AkkaUtilsSuite and shorten the timeouts as you mentioned below. |
||
|
|
||
| val actorSystem = AkkaUtils.createActorSystem("EchoActors", "127.0.0.1", 9999, conf = conf, | ||
| securityManager = new SecurityManager(conf))._1 | ||
|
|
||
| val askingActor = actorSystem.actorOf(Props[EchoActor], "AskingActor") | ||
|
|
||
| AkkaUtils.askWithReply[String]("this should echo", askingActor, 1, 0, | ||
| ConfiguredTimeout.createAskTimeout(conf)) | ||
|
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. you could get around the fact that the conf timeouts have a minimum time of seconds by directly creating your timeout here: |
||
|
|
||
| try { | ||
| AkkaUtils.askWithReply[String]("this should timeout", askingActor, 1, 0, | ||
| ConfiguredTimeout(conf, shortProp)) | ||
| throw new TestException("required exception not raised during AkkaUtils.askWithReply") | ||
| } catch { | ||
| case se: SparkException => | ||
| assert(se.getCause().getMessage().contains(shortProp)) | ||
|
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. this can be simplified to:
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 nice trick, thanks! |
||
| } | ||
| } | ||
|
|
||
| } | ||
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.
this need to be scoped at least to
private[spark], possibly even tighter. Same goes for other classes & methods. We can wait till the rest of the design shakes out a little more before worrying about that too much, though