Deprecate actorFor in favor of ActorSelection, see #3074

* Deprecate all actorFor methods
* resolveActorRef in provider
* Identify auto receive message
* Support ActorPath in actorSelection
* Support remote actor selections
* Additional tests of actor selection
* Update tests (keep most actorFor tests)
* Update samples to use actorSelection
* Updates to documentation
* Migration guide, including motivation
This commit is contained in:
Patrik Nordwall 2013-03-26 18:17:50 +01:00
parent 641e499cc4
commit 887af975ae
80 changed files with 1496 additions and 619 deletions

View file

@ -8,6 +8,7 @@ import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.forkjoin.ThreadLocalRandom
import java.util.concurrent.atomic.AtomicReference
import org.scalatest.BeforeAndAfterEach
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
@ -35,6 +36,8 @@ import akka.remote.testkit.MultiNodeSpec
import akka.routing.FromConfig
import akka.testkit._
import akka.testkit.TestEvent._
import akka.actor.Identify
import akka.actor.ActorIdentity
/**
* This test is intended to be used as long running stress test
@ -553,9 +556,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
* Used for remote death watch testing
*/
class Watchee extends Actor {
def receive = {
case Ping sender ! Pong
}
def receive = Actor.emptyBehavior
}
/**
@ -623,9 +624,6 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
case class ChildrenCount(numberOfChildren: Int, numberOfChildRestarts: Int)
case object Reset
case object Ping
case object Pong
}
class StressMultiJvmNode1 extends StressSpec
@ -652,6 +650,8 @@ abstract class StressSpec
val settings = new Settings(system.settings.config)
import settings._
val identifyProbe = TestProbe()
var step = 0
var nbrUsedRoles = 0
@ -687,13 +687,16 @@ abstract class StressSpec
}
enterBarrier("result-aggregator-created-" + step)
runOn(roles.take(nbrUsedRoles): _*) {
phiObserver ! ReportTo(Some(clusterResultAggregator))
phiObserver ! ReportTo(clusterResultAggregator)
statsObserver ! Reset
statsObserver ! ReportTo(Some(clusterResultAggregator))
statsObserver ! ReportTo(clusterResultAggregator)
}
}
def clusterResultAggregator: ActorRef = system.actorFor(node(roles.head) / "user" / ("result" + step))
def clusterResultAggregator: Option[ActorRef] = {
system.actorSelection(node(roles.head) / "user" / ("result" + step)).tell(Identify(step), identifyProbe.ref)
identifyProbe.expectMsgType[ActorIdentity].ref
}
lazy val clusterResultHistory = system.actorOf(Props[ClusterResultHistory], "resultHistory")
@ -703,9 +706,12 @@ abstract class StressSpec
def awaitClusterResult(): Unit = {
runOn(roles.head) {
val r = clusterResultAggregator
watch(r)
expectMsgPF(remaining) { case Terminated(a) if a.path == r.path true }
clusterResultAggregator match {
case Some(r)
watch(r)
expectMsgPF(remaining) { case Terminated(a) if a.path == r.path true }
case None // ok, already terminated
}
}
enterBarrier("cluster-result-done-" + step)
}
@ -778,9 +784,9 @@ abstract class StressSpec
}
enterBarrier("watchee-created-" + step)
runOn(roles.head) {
system.actorFor(node(removeRole) / "user" / "watchee") ! Ping
expectMsg(Pong)
watch(lastSender)
system.actorSelection(node(removeRole) / "user" / "watchee").tell(Identify("watchee"), identifyProbe.ref)
val watchee = identifyProbe.expectMsgType[ActorIdentity].ref.get
watch(watchee)
}
enterBarrier("watch-estabilished-" + step)
@ -838,8 +844,9 @@ abstract class StressSpec
val returnValue = thunk
clusterResultAggregator !
ClusterResult(cluster.selfAddress, (System.nanoTime - startTime).nanos, cluster.readView.latestStats :- startStats)
clusterResultAggregator foreach {
_ ! ClusterResult(cluster.selfAddress, (System.nanoTime - startTime).nanos, cluster.readView.latestStats :- startStats)
}
returnValue
}
@ -904,7 +911,12 @@ abstract class StressSpec
}
def master: ActorRef = system.actorFor("/user/master-" + myself.name)
def masterName: String = "master-" + myself.name
def master: Option[ActorRef] = {
system.actorSelection("/user/" + masterName).tell(Identify("master"), identifyProbe.ref)
identifyProbe.expectMsgType[ActorIdentity].ref
}
def exerciseRouters(title: String, duration: FiniteDuration, batchInterval: FiniteDuration,
expectDroppedMessages: Boolean, tree: Boolean): Unit =
@ -940,13 +952,16 @@ abstract class StressSpec
}
def awaitWorkResult: WorkResult = {
val m = master
val workResult = expectMsgType[WorkResult]
log.info("{} result, [{}] jobs/s, retried [{}] of [{}] msg", m.path.name,
log.info("{} result, [{}] jobs/s, retried [{}] of [{}] msg", masterName,
workResult.jobsPerSecond.form,
workResult.retryCount, workResult.sendCount)
watch(m)
expectMsgPF(remaining) { case Terminated(a) if a.path == m.path true }
master match {
case Some(m)
watch(m)
expectMsgPF(remaining) { case Terminated(a) if a.path == m.path true }
case None // ok, already terminated
}
workResult
}
@ -1043,7 +1058,8 @@ abstract class StressSpec
"end routers that are running while nodes are joining" taggedAs LongRunningTest in within(30.seconds) {
runOn(roles.take(3): _*) {
val m = master
m.tell(End, testActor)
m must not be (None)
m.get.tell(End, testActor)
val workResult = awaitWorkResult
workResult.retryCount must be(0)
workResult.sendCount must be > (0L)
@ -1129,7 +1145,8 @@ abstract class StressSpec
"end routers that are running while nodes are removed" taggedAs LongRunningTest in within(30.seconds) {
runOn(roles.take(3): _*) {
val m = master
m.tell(End, testActor)
m must not be (None)
m.get.tell(End, testActor)
val workResult = awaitWorkResult
workResult.sendCount must be > (0L)
workResult.ackCount must be > (0L)