Merge commit

This commit is contained in:
Viktor Klang 2011-09-29 13:11:35 +02:00
commit a12ee36151
57 changed files with 596 additions and 275 deletions

View file

@ -18,6 +18,8 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
object ActorRefSpec {
case class ReplyTo(channel: Channel[Any])
val latch = TestLatch(4)
class ReplyActor extends Actor {
@ -31,7 +33,7 @@ object ActorRefSpec {
}
case "complexRequest2"
val worker = actorOf(Props[WorkerActor])
worker ! channel
worker ! ReplyTo(channel)
case "workDone" replyTo ! "complexReply"
case "simpleRequest" reply("simpleReply")
}
@ -44,7 +46,7 @@ object ActorRefSpec {
reply("workDone")
self.stop()
}
case replyTo: Channel[Any] {
case ReplyTo(replyTo) {
work
replyTo ! "complexReply"
}

View file

@ -45,7 +45,7 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
"Actor Registry" must {
"get actor by address from registry" in {
"get actor by address from registry" ignore {
val started = TestLatch(1)
val stopped = TestLatch(1)
val actor = actorOf(new StartStopTestActor(started, stopped), "test-actor-1")
@ -59,7 +59,7 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
Actor.registry.actorFor(actor.address).isEmpty must be(true)
}
"get actor by uuid from local registry" in {
"get actor by uuid from local registry" ignore {
val started = TestLatch(1)
val stopped = TestLatch(1)
val actor = actorOf(new StartStopTestActor(started, stopped), "test-actor-1")
@ -74,7 +74,7 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
Actor.registry.local.actorFor(uuid).isEmpty must be(true)
}
"find things from local registry" in {
"find things from local registry" ignore {
val actor = actorOf[TestActor]("test-actor-1")
val found: Option[LocalActorRef] = Actor.registry.local.find({ case a: LocalActorRef if a.underlyingActorInstance.isInstanceOf[TestActor] a })
found.isDefined must be(true)
@ -83,7 +83,7 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
actor.stop
}
"get all actors from local registry" in {
"get all actors from local registry" ignore {
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
val actors = Actor.registry.local.actors
@ -94,7 +94,7 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
actor2.stop
}
"get response from all actors in local registry using foreach" in {
"get response from all actors in local registry using foreach" ignore {
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
val results = new ConcurrentLinkedQueue[Future[String]]
@ -108,14 +108,14 @@ class ActorRegistrySpec extends WordSpec with MustMatchers with BeforeAndAfterAl
actor2.stop()
}
"shutdown all actors in local registry" in {
"shutdown all actors in local registry" ignore {
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
Actor.registry.local.shutdownAll
Actor.registry.local.actors.size must be(0)
}
"remove when unregistering actors from local registry" in {
"remove when unregistering actors from local registry" ignore {
val actor1 = actorOf[TestActor]("test-actor-1")
val actor2 = actorOf[TestActor]("test-actor-2")
Actor.registry.local.actors.size must be(2)

View file

@ -20,11 +20,12 @@ class DeployerSpec extends WordSpec with MustMatchers {
"service-ping",
None,
LeastCPU,
NrOfInstances(3),
BannagePeriodFailureDetector(10),
RemoteScope("localhost", 2552))))
// ClusterScope(
// List(Node("node1")),
// new ReplicationFactor(3),
// new NrOfInstances(3),
// Replication(
// TransactionLog,
// WriteThrough)))))

View file

@ -51,7 +51,7 @@ class SchedulerSpec extends JUnitSuite {
collectFuture(Scheduler.schedule(() countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS))
// after max 1 second it should be executed at least the 3 times already
assert(countDownLatch2.await(1, TimeUnit.SECONDS))
assert(countDownLatch2.await(2, TimeUnit.SECONDS))
}
@Test
@ -66,7 +66,7 @@ class SchedulerSpec extends JUnitSuite {
collectFuture(Scheduler.scheduleOnce(() countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS))
// after 1 second the wait should fail
assert(countDownLatch.await(1, TimeUnit.SECONDS) == false)
assert(countDownLatch.await(2, TimeUnit.SECONDS) == false)
// should still be 1 left
assert(countDownLatch.getCount == 1)
}

View file

@ -0,0 +1,343 @@
package akka.routing
import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers
import akka.actor._
import Actor._
import DeploymentConfig._
import akka.routing._
import Routing.Broadcast
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.{ CountDownLatch, TimeUnit }
class ConfiguredLocalRoutingSpec extends WordSpec with MustMatchers {
// "direct router" must {
// "be able to shut down its instance" in {
// val address = "direct-0"
// Deployer.deploy(
// Deploy(
// address,
// None,
// Direct,
// NrOfInstances(1),
// RemoveConnectionOnFirstFailureLocalFailureDetector,
// LocalScope))
// val helloLatch = new CountDownLatch(1)
// val stopLatch = new CountDownLatch(1)
// val actor = actorOf(new Actor {
// def receive = {
// case "hello" helloLatch.countDown()
// }
// override def postStop() {
// stopLatch.countDown()
// }
// }, address)
// actor ! "hello"
// helloLatch.await(5, TimeUnit.SECONDS) must be(true)
// actor.stop()
// stopLatch.await(5, TimeUnit.SECONDS) must be(true)
// }
// "send message to connection" in {
// val address = "direct-1"
// Deployer.deploy(
// Deploy(
// address,
// None,
// Direct,
// NrOfInstances(1),
// RemoveConnectionOnFirstFailureLocalFailureDetector,
// LocalScope))
// val doneLatch = new CountDownLatch(1)
// val counter = new AtomicInteger(0)
// val actor = actorOf(new Actor {
// def receive = {
// case "end" doneLatch.countDown()
// case _ counter.incrementAndGet()
// }
// }, address)
// actor ! "hello"
// actor ! "end"
// doneLatch.await(5, TimeUnit.SECONDS) must be(true)
// counter.get must be(1)
// }
// "deliver a broadcast message" in {
// val address = "direct-2"
// Deployer.deploy(
// Deploy(
// address,
// None,
// Direct,
// NrOfInstances(1),
// RemoveConnectionOnFirstFailureLocalFailureDetector,
// LocalScope))
// val doneLatch = new CountDownLatch(1)
// val counter1 = new AtomicInteger
// val actor = actorOf(new Actor {
// def receive = {
// case "end" doneLatch.countDown()
// case msg: Int counter1.addAndGet(msg)
// }
// }, address)
// actor ! Broadcast(1)
// actor ! "end"
// doneLatch.await(5, TimeUnit.SECONDS) must be(true)
// counter1.get must be(1)
// }
// }
"round robin router" must {
"be able to shut down its instance" in {
val address = "round-robin-0"
Deployer.deploy(
Deploy(
address,
None,
RoundRobin,
NrOfInstances(5),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5)
val actor = actorOf(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}
override def postStop() {
stopLatch.countDown()
}
}, address)
actor ! "hello"
actor ! "hello"
actor ! "hello"
actor ! "hello"
actor ! "hello"
helloLatch.await(5, TimeUnit.SECONDS) must be(true)
actor.stop()
stopLatch.await(5, TimeUnit.SECONDS) must be(true)
}
"deliver messages in a round robin fashion" in {
val address = "round-robin-1"
Deployer.deploy(
Deploy(
address,
None,
RoundRobin,
NrOfInstances(10),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val connectionCount = 10
val iterationCount = 10
val doneLatch = new CountDownLatch(connectionCount)
val counter = new AtomicInteger
var replies = Map.empty[Int, Int]
for (i 0 until connectionCount) {
replies = replies + (i -> 0)
}
val actor = actorOf(new Actor {
lazy val id = counter.getAndIncrement()
def receive = {
case "hit" reply(id)
case "end" doneLatch.countDown()
}
}, address)
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
val id = (actor ? "hit").as[Int].getOrElse(fail("No id returned by actor"))
replies = replies + (id -> (replies(id) + 1))
}
}
counter.get must be(connectionCount)
actor ! Broadcast("end")
doneLatch.await(5, TimeUnit.SECONDS) must be(true)
replies.values foreach { _ must be(10) }
}
"deliver a broadcast message using the !" in {
val address = "round-robin-2"
Deployer.deploy(
Deploy(
address,
None,
RoundRobin,
NrOfInstances(5),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(5)
val stopLatch = new CountDownLatch(5)
val actor = actorOf(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}
override def postStop() {
stopLatch.countDown()
}
}, address)
actor ! Broadcast("hello")
helloLatch.await(5, TimeUnit.SECONDS) must be(true)
actor.stop()
stopLatch.await(5, TimeUnit.SECONDS) must be(true)
}
}
"random router" must {
"be able to shut down its instance" in {
val address = "random-0"
Deployer.deploy(
Deploy(
address,
None,
Random,
NrOfInstances(7),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val stopLatch = new CountDownLatch(7)
val actor = actorOf(new Actor {
def receive = {
case "hello" {}
}
override def postStop() {
stopLatch.countDown()
}
}, address)
actor ! "hello"
actor ! "hello"
actor ! "hello"
actor ! "hello"
actor ! "hello"
actor.stop()
stopLatch.await(5, TimeUnit.SECONDS) must be(true)
}
"deliver messages in a random fashion" in {
val address = "random-1"
Deployer.deploy(
Deploy(
address,
None,
Random,
NrOfInstances(10),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val connectionCount = 10
val iterationCount = 10
val doneLatch = new CountDownLatch(connectionCount)
val counter = new AtomicInteger
var replies = Map.empty[Int, Int]
for (i 0 until connectionCount) {
replies = replies + (i -> 0)
}
val actor = actorOf(new Actor {
lazy val id = counter.getAndIncrement()
def receive = {
case "hit" reply(id)
case "end" doneLatch.countDown()
}
}, address)
for (i 0 until iterationCount) {
for (k 0 until connectionCount) {
val id = (actor ? "hit").as[Int].getOrElse(fail("No id returned by actor"))
replies = replies + (id -> (replies(id) + 1))
}
}
counter.get must be(connectionCount)
actor ! Broadcast("end")
doneLatch.await(5, TimeUnit.SECONDS) must be(true)
replies.values foreach { _ must be > (0) }
}
"deliver a broadcast message using the !" in {
val address = "random-2"
Deployer.deploy(
Deploy(
address,
None,
Random,
NrOfInstances(6),
RemoveConnectionOnFirstFailureLocalFailureDetector,
LocalScope))
val helloLatch = new CountDownLatch(6)
val stopLatch = new CountDownLatch(6)
val actor = actorOf(new Actor {
def receive = {
case "hello" helloLatch.countDown()
}
override def postStop() {
stopLatch.countDown()
}
}, address)
actor ! Broadcast("hello")
helloLatch.await(5, TimeUnit.SECONDS) must be(true)
actor.stop()
stopLatch.await(5, TimeUnit.SECONDS) must be(true)
}
}
}

View file

@ -28,13 +28,15 @@ class RoutingSpec extends WordSpec with MustMatchers {
"be started when constructed" in {
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.Direct)
val props = RoutedProps(() new DirectRouter, List(actor1))
val actor = Routing.actorOf(props, "foo")
actor.isRunning must be(true)
}
"throw IllegalArgumentException at construction when no connections" in {
try {
Routing.actorOf("foo", List(), RouterType.Direct)
val props = RoutedProps(() new DirectRouter, List())
Routing.actorOf(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
@ -52,7 +54,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val routedActor = Routing.actorOf("foo", List(connection1), RouterType.Direct)
val props = RoutedProps(() new DirectRouter, List(connection1))
val routedActor = Routing.actorOf(props, "foo")
routedActor ! "hello"
routedActor ! "end"
@ -72,7 +75,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.Direct)
val props = RoutedProps(() new DirectRouter, List(connection1))
val actor = Routing.actorOf(props, "foo")
actor ! Broadcast(1)
actor ! "end"
@ -88,13 +92,15 @@ class RoutingSpec extends WordSpec with MustMatchers {
"be started when constructed" in {
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.RoundRobin)
val props = RoutedProps(() new RoundRobinRouter, List(actor1))
val actor = Routing.actorOf(props, "foo")
actor.isRunning must be(true)
}
"throw IllegalArgumentException at construction when no connections" in {
try {
Routing.actorOf("foo", List(), RouterType.RoundRobin)
val props = RoutedProps(() new RoundRobinRouter, List())
Routing.actorOf(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
@ -126,7 +132,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
//create the routed actor.
val actor = Routing.actorOf("foo", connections, RouterType.RoundRobin)
val props = RoutedProps(() new RoundRobinRouter, connections)
val actor = Routing.actorOf(props, "foo")
//send messages to the actor.
for (i 0 until iterationCount) {
@ -164,7 +171,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val actor = Routing.actorOf("foo", List(connection1, connection2), RouterType.RoundRobin)
val props = RoutedProps(() new RoundRobinRouter, List(connection1, connection2))
val actor = Routing.actorOf(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -186,7 +194,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.RoundRobin)
val props = RoutedProps(() new RoundRobinRouter, List(connection1))
val actor = Routing.actorOf(props, "foo")
try {
actor ? Broadcast(1)
@ -207,13 +216,15 @@ class RoutingSpec extends WordSpec with MustMatchers {
val actor1 = Actor.actorOf[TestActor]
val actor = Routing.actorOf("foo", List(actor1), RouterType.Random)
val props = RoutedProps(() new RandomRouter, List(actor1))
val actor = Routing.actorOf(props, "foo")
actor.isRunning must be(true)
}
"throw IllegalArgumentException at construction when no connections" in {
try {
Routing.actorOf("foo", List(), RouterType.Random)
val props = RoutedProps(() new RandomRouter, List())
Routing.actorOf(props, "foo")
fail()
} catch {
case e: IllegalArgumentException
@ -243,7 +254,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val actor = Routing.actorOf("foo", List(connection1, connection2), RouterType.Random)
val props = RoutedProps(() new RandomRouter, List(connection1, connection2))
val actor = Routing.actorOf(props, "foo")
actor ! Broadcast(1)
actor ! Broadcast("end")
@ -265,7 +277,8 @@ class RoutingSpec extends WordSpec with MustMatchers {
}
})
val actor = Routing.actorOf("foo", List(connection1), RouterType.Random)
val props = RoutedProps(() new RandomRouter, List(connection1))
val actor = Routing.actorOf(props, "foo")
try {
actor ? Broadcast(1)
@ -279,40 +292,4 @@ class RoutingSpec extends WordSpec with MustMatchers {
counter1.get must be(0)
}
}
"least cpu router" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastCPU)
} catch {
case e: IllegalArgumentException
}
}
}
"least ram router" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastRAM)
} catch {
case e: IllegalArgumentException
}
}
}
"smallest mailbox" must {
"throw IllegalArgumentException when constructed" in {
val actor1 = Actor.actorOf[TestActor]
try {
Routing.actorOf("foo", List(actor1), RouterType.LeastMessages)
} catch {
case e: IllegalArgumentException
}
}
}
}

View file

@ -28,7 +28,7 @@ class Ticket703Spec extends WordSpec with MustMatchers {
}
})
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000)))
(actorPool.?("Ping", 7000)).await.result must be === Some("Response")
(actorPool.?("Ping", 10000)).await.result must be === Some("Response")
}
}
}

View file

@ -359,6 +359,7 @@ private[akka] class ActorCell(
}
} catch {
case e
EventHandler.error(e, this, "error while creating actor")
envelope.channel.sendException(e)
if (supervisor.isDefined) supervisor.get ! Failed(self, e) else throw e
}

View file

@ -268,8 +268,8 @@ class LocalActorRef private[akka] (
protected[akka] def underlying: ActorCell = actorCell
//FIXME TODO REMOVE THIS
@deprecated("This method does a spin-lock to block for the actor, which might never be there, do not use this")
// FIXME TODO: remove this method
// @deprecated("This method does a spin-lock to block for the actor, which might never be there, do not use this", "2.0")
protected[akka] def underlyingActorInstance: Actor = {
var instance = actorCell.actor
while ((instance eq null) && actorCell.isRunning) {

View file

@ -7,6 +7,7 @@ package akka.actor
import DeploymentConfig._
import akka.event.EventHandler
import akka.AkkaException
import akka.routing._
/**
* Interface for all ActorRef providers to implement.
@ -59,7 +60,7 @@ private[akka] class ActorRefProviders(
providers match {
case Nil None
case provider :: rest
provider.actorOf(props, address) match { //WARNING FIXME RACE CONDITION NEEDS TO BE SOLVED
provider.actorOf(props, address) match {
case None actorOf(props, address, rest) // recur
case ref ref
}
@ -124,8 +125,8 @@ private[akka] class ActorRefProviders(
class LocalActorRefProvider extends ActorRefProvider {
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
import com.eaio.uuid.UUID
// FIXME who evicts this registry, and when? Should it be used instead of ActorRegistry?
private val actors = new ConcurrentHashMap[String, Promise[Option[ActorRef]]]
def actorOf(props: Props, address: String): Option[ActorRef] = actorOf(props, address, false)
@ -145,13 +146,36 @@ class LocalActorRefProvider extends ActorRefProvider {
if (oldFuture eq null) { // we won the race -- create the actor and resolve the future
def newActor() = Some(new LocalActorRef(props, address, systemService))
val actor = try {
Deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor
case Some(Deploy(_, _, router, _, LocalScope)) newActor() // create a local actor
case None newActor() // create a local actor
case _ None // non-local actor
// create a local actor
case None | Some(Deploy(_, _, Direct, _, _, LocalScope))
Some(new LocalActorRef(props, address, systemService)) // create a local actor
// create a routed actor ref
case deploy @ Some(Deploy(_, _, router, nrOfInstances, _, LocalScope))
val routerType = DeploymentConfig.routerTypeFor(router)
val routerFactory: () Router = routerType match {
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")
case RouterType.Custom sys.error("Router Custom not supported yet")
}
val connections: Iterable[ActorRef] =
if (nrOfInstances.factor > 0)
Vector.fill(nrOfInstances.factor)(new LocalActorRef(props, new UUID().toString, systemService))
else Nil
Some(Routing.actorOf(RoutedProps(
routerFactory = routerFactory,
connections = connections)))
case _ None // non-local actor - pass it on
}
} catch {
case e: Exception

View file

@ -46,7 +46,7 @@ object Deployer extends ActorDeployer {
def deploy(deployment: Deploy): Unit = instance.deploy(deployment)
def isLocal(deployment: Deploy): Boolean = deployment match {
case Deploy(_, _, _, _, LocalScope) | Deploy(_, _, _, _, _: LocalScope) true
case Deploy(_, _, _, _, _, LocalScope) | Deploy(_, _, _, _, _, _: LocalScope) true
case _ false
}
@ -122,7 +122,7 @@ object Deployer extends ActorDeployer {
val addressPath = "akka.actor.deployment." + address
configuration.getSection(addressPath) match {
case None
Some(Deploy(address, None, Direct, RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope))
Some(Deploy(address, None, Direct, NrOfInstances(1), RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope))
case Some(addressConfig)
@ -144,6 +144,30 @@ object Deployer extends ActorDeployer {
CustomRouter(_))
}
// --------------------------------
// akka.actor.deployment.<address>.nr-of-instances
// --------------------------------
val nrOfInstances = {
if (router == Direct) NrOfInstances(1)
else {
addressConfig.getAny("nr-of-instances", "1") match {
case "auto" AutoNrOfInstances
case "1" NrOfInstances(1)
case "0" ZeroNrOfInstances
case nrOfReplicas: String
try {
new NrOfInstances(nrOfReplicas.toInt)
} catch {
case e: Exception
throw new ConfigurationException(
"Config option [" + addressPath +
".nr-of-instances] needs to be either [\"auto\"] or [1-N] - was [" +
nrOfReplicas + "]")
}
}
}
}
// --------------------------------
// akka.actor.deployment.<address>.failure-detector.xxx
// --------------------------------
@ -210,7 +234,7 @@ object Deployer extends ActorDeployer {
val hostname = remoteConfig.getString("hostname", "localhost")
val port = remoteConfig.getInt("port", 2552)
Some(Deploy(address, recipe, router, failureDetector, RemoteScope(hostname, port)))
Some(Deploy(address, recipe, router, nrOfInstances, failureDetector, RemoteScope(hostname, port)))
case None // check for 'cluster' config section
@ -219,7 +243,7 @@ object Deployer extends ActorDeployer {
// --------------------------------
addressConfig.getSection("cluster") match {
case None
Some(Deploy(address, recipe, router, RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope)) // deploy locally
Some(Deploy(address, recipe, router, nrOfInstances, RemoveConnectionOnFirstFailureLocalFailureDetector, LocalScope)) // deploy locally
case Some(clusterConfig)
@ -251,35 +275,12 @@ object Deployer extends ActorDeployer {
}
}
// --------------------------------
// akka.actor.deployment.<address>.cluster.replicas
// --------------------------------
val replicationFactor = {
if (router == Direct) new ReplicationFactor(1)
else {
clusterConfig.getAny("replication-factor", "0") match {
case "auto" AutoReplicationFactor
case "0" ZeroReplicationFactor
case nrOfReplicas: String
try {
new ReplicationFactor(nrOfReplicas.toInt)
} catch {
case e: Exception
throw new ConfigurationException(
"Config option [" + addressPath +
".cluster.replicas] needs to be either [\"auto\"] or [0-N] - was [" +
nrOfReplicas + "]")
}
}
}
}
// --------------------------------
// akka.actor.deployment.<address>.cluster.replication
// --------------------------------
clusterConfig.getSection("replication") match {
case None
Some(Deploy(address, recipe, router, failureDetector, ClusterScope(preferredNodes, replicationFactor, Transient)))
Some(Deploy(address, recipe, router, nrOfInstances, failureDetector, ClusterScope(preferredNodes, Transient)))
case Some(replicationConfig)
val storage = replicationConfig.getString("storage", "transaction-log") match {
@ -298,7 +299,7 @@ object Deployer extends ActorDeployer {
".cluster.replication.strategy] needs to be either [\"write-through\"] or [\"write-behind\"] - was [" +
unknown + "]")
}
Some(Deploy(address, recipe, router, failureDetector, ClusterScope(preferredNodes, replicationFactor, Replication(storage, strategy))))
Some(Deploy(address, recipe, router, nrOfInstances, failureDetector, ClusterScope(preferredNodes, Replication(storage, strategy))))
}
}
}
@ -319,7 +320,7 @@ object Deployer extends ActorDeployer {
}
/**
* TODO: Improved documentation
* Simple local deployer, only for internal use.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
@ -335,15 +336,7 @@ object LocalDeployer extends ActorDeployer {
}
private[akka] def deploy(deployment: Deploy) {
deployments.putIfAbsent(deployment.address, deployment) /* match {
case null
deployment match {
case Deploy(address, Some(recipe), routing, _) Actor.actorOf(recipe.implementationClass, address) //FIXME use routing?
case _
}
case `deployment` //Already deployed TODO should it be like this?
case preexists Deployer.throwDeploymentBoundException(deployment)
}*/
deployments.putIfAbsent(deployment.address, deployment)
}
private[akka] def lookupDeploymentFor(address: String): Option[Deploy] = Option(deployments.get(address))

View file

@ -23,6 +23,7 @@ object DeploymentConfig {
address: String,
recipe: Option[ActorRecipe],
routing: Routing = Direct,
nrOfInstances: NrOfInstances = ZeroNrOfInstances,
failureDetector: FailureDetector = RemoveConnectionOnFirstFailureLocalFailureDetector,
scope: Scope = LocalScope) {
Address.validate(address)
@ -76,7 +77,6 @@ object DeploymentConfig {
sealed trait Scope
case class ClusterScope(
preferredNodes: Iterable[Home] = Vector(Node(Config.nodename)),
replicas: ReplicationFactor = ZeroReplicationFactor,
replication: ReplicationScheme = Transient) extends Scope
case class RemoteScope(
@ -101,28 +101,28 @@ object DeploymentConfig {
// --- Replicas
// --------------------------------
class ReplicationFactor(val factor: Int) extends Serializable {
if (factor < 0) throw new IllegalArgumentException("replication-factor can not be negative")
class NrOfInstances(val factor: Int) extends Serializable {
if (factor < 0) throw new IllegalArgumentException("nr-of-instances can not be negative")
override def hashCode = 0 + factor.##
override def equals(other: Any) = ReplicationFactor.unapply(this) == ReplicationFactor.unapply(other)
override def toString = "ReplicationFactor(" + factor + ")"
override def equals(other: Any) = NrOfInstances.unapply(this) == NrOfInstances.unapply(other)
override def toString = "NrOfInstances(" + factor + ")"
}
object ReplicationFactor {
def apply(factor: Int): ReplicationFactor = new ReplicationFactor(factor)
object NrOfInstances {
def apply(factor: Int): NrOfInstances = new NrOfInstances(factor)
def unapply(other: Any) = other match {
case x: ReplicationFactor import x._; Some(factor)
case x: NrOfInstances import x._; Some(factor)
case _ None
}
}
// For Java API
class AutoReplicationFactor extends ReplicationFactor(-1)
class ZeroReplicationFactor extends ReplicationFactor(0)
class AutoNrOfInstances extends NrOfInstances(-1)
class ZeroNrOfInstances extends NrOfInstances(0)
// For Scala API
case object AutoReplicationFactor extends AutoReplicationFactor
case object ZeroReplicationFactor extends ZeroReplicationFactor
case object AutoNrOfInstances extends AutoNrOfInstances
case object ZeroNrOfInstances extends ZeroNrOfInstances
// --------------------------------
// --- Replication
@ -206,7 +206,7 @@ object DeploymentConfig {
}
def replicationSchemeFor(deployment: Deploy): Option[ReplicationScheme] = deployment match {
case Deploy(_, _, _, _, ClusterScope(_, _, replicationScheme)) Some(replicationScheme)
case Deploy(_, _, _, _, _, ClusterScope(_, replicationScheme)) Some(replicationScheme)
case _ None
}

View file

@ -423,6 +423,7 @@ trait FSM[S, D] extends ListenerManagement {
*/
private val handleEventDefault: StateFunction = {
case Event(value, stateData)
EventHandler.warning(this, "unhandled event " + value + " in state " + stateName)
stay
}
private var handleEvent: StateFunction = handleEventDefault

View file

@ -248,14 +248,14 @@ trait ClusterNode {
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializer: Serializer): ClusterNode
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, serializer: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -276,14 +276,14 @@ trait ClusterNode {
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
def store[T <: Actor](address: String, actorClass: Class[T], nrOfInstances: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -311,14 +311,14 @@ trait ClusterNode {
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializer: Serializer): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializer: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -329,26 +329,26 @@ trait ClusterNode {
/**
* Needed to have reflection through structural typing work.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode
/**
* Needed to have reflection through structural typing work.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode
/**
* Removes actor from the cluster.
@ -442,28 +442,28 @@ trait ClusterNode {
def inetSocketAddressesForActor(actorAddress: String): Array[(UUID, InetSocketAddress)]
/**
* Send a function 'Function0[Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
* Send a function 'Function0[Unit]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument).
*/
def send(f: Function0[Unit], replicationFactor: Int)
def send(f: Function0[Unit], nrOfInstances: Int)
/**
* Send a function 'Function0[Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
* Send a function 'Function0[Any]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument).
* Returns an 'Array' with all the 'Future's from the computation.
*/
def send(f: Function0[Any], replicationFactor: Int): List[Future[Any]]
def send(f: Function0[Any], nrOfInstances: Int): List[Future[Any]]
/**
* Send a function 'Function1[Any, Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
* Send a function 'Function1[Any, Unit]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument)
* with the argument speficied.
*/
def send(f: Function1[Any, Unit], arg: Any, replicationFactor: Int)
def send(f: Function1[Any, Unit], arg: Any, nrOfInstances: Int)
/**
* Send a function 'Function1[Any, Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
* Send a function 'Function1[Any, Any]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument)
* with the argument speficied.
* Returns an 'Array' with all the 'Future's from the computation.
*/
def send(f: Function1[Any, Any], arg: Any, replicationFactor: Int): List[Future[Any]]
def send(f: Function1[Any, Any], arg: Any, nrOfInstances: Int): List[Future[Any]]
/**
* Stores a configuration element under a specific key.

View file

@ -85,17 +85,12 @@ object RoutedProps {
*/
case class RoutedProps(
routerFactory: () Router,
failureDetectorFactory: (Map[InetSocketAddress, ActorRef]) FailureDetector,
connections: Iterable[ActorRef],
timeout: Timeout,
localOnly: Boolean) {
failureDetectorFactory: (Map[InetSocketAddress, ActorRef]) FailureDetector = RoutedProps.defaultFailureDetectorFactory,
timeout: Timeout = RoutedProps.defaultTimeout,
localOnly: Boolean = RoutedProps.defaultLocalOnly) {
def this() = this(
routerFactory = RoutedProps.defaultRouterFactory,
failureDetectorFactory = RoutedProps.defaultFailureDetectorFactory,
connections = List(),
timeout = RoutedProps.defaultTimeout,
localOnly = RoutedProps.defaultLocalOnly)
def this() = this(RoutedProps.defaultRouterFactory, List())
/**
* Returns a new RoutedProps configured with a random router.

View file

@ -191,8 +191,8 @@ object Routing {
new RoutedActorRef(
new RoutedProps(
() router,
RoutedProps.defaultFailureDetectorFactory,
connections,
RoutedProps.defaultFailureDetectorFactory,
RoutedProps.defaultTimeout, true),
actorAddress)
}
@ -240,7 +240,7 @@ private[akka] class RoutedActorRef(val routedProps: RoutedProps, val address: St
synchronized {
if (running) {
running = false
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
router.route(Routing.Broadcast(PoisonPill))(Some(this))
}
}
}

View file

@ -55,12 +55,12 @@ trait Consumer { this: Actor ⇒
/**
* Java-friendly Consumer.
*
* @see UntypedConsumerActor
* @see RemoteUntypedConsumerActor
* Subclass this abstract class to create an MDB-style untyped consumer actor. This
* class is meant to be used from Java.
*
* @author Martin Krasser
*/
trait UntypedConsumer extends Consumer { self: UntypedActor
abstract class UntypedConsumerActor extends UntypedActor with Consumer {
final override def endpointUri = getEndpointUri
final override def blocking = isBlocking
final override def autoack = isAutoack
@ -84,12 +84,6 @@ trait UntypedConsumer extends Consumer { self: UntypedActor ⇒
def isAutoack() = super.autoack
}
/**
* Subclass this abstract class to create an MDB-style untyped consumer actor. This
* class is meant to be used from Java.
*/
abstract class UntypedConsumerActor extends UntypedActor with UntypedConsumer
/**
* A callback handler for route definitions to consumer actors.
*

View file

@ -191,13 +191,11 @@ trait Producer extends ProducerSupport { this: Actor ⇒
}
/**
* Java-friendly ProducerSupport.
*
* @see UntypedProducerActor
* Subclass this abstract class to create an untyped producer actor. This class is meant to be used from Java.
*
* @author Martin Krasser
*/
trait UntypedProducer extends ProducerSupport { this: UntypedActor
abstract class UntypedProducerActor extends UntypedActor with ProducerSupport {
final override def endpointUri = getEndpointUri
final override def oneway = isOneway
@ -244,13 +242,6 @@ trait UntypedProducer extends ProducerSupport { this: UntypedActor ⇒
def onReceiveAfterProduce(message: Any): Unit = super.receiveAfterProduce(message)
}
/**
* Subclass this abstract class to create an untyped producer actor. This class is meant to be used from Java.
*
* @author Martin Krasser
*/
abstract class UntypedProducerActor extends UntypedActor with UntypedProducer
/**
* @author Martin Krasser
*/

View file

@ -67,7 +67,7 @@ public class MessageJavaTestBase {
@Test public void shouldTransformBodyAndPreserveHeaders() {
assertEquals(
new Message("ab", createMap("A", "1")),
new Message("a" , createMap("A", "1")).transformBody((Function)new TestTransformer()));
new Message("a" , createMap("A", "1")).transformBody((Function<String, Object>) new TestTransformer()));
}
@Test public void shouldConvertBodyAndPreserveHeaders() {
@ -120,7 +120,7 @@ public class MessageJavaTestBase {
return map;
}
private static class TestTransformer implements Function<String, String> {
private static class TestTransformer implements Function<String, Object> {
public String apply(String param) {
return param + "b";
}

View file

@ -517,16 +517,16 @@ class DefaultClusterNode private[akka] (
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, Transient, false, serializer)
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), nrOfInstances, Transient, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, replicationScheme, false, serializer)
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), nrOfInstances, replicationScheme, false, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
@ -549,16 +549,16 @@ class DefaultClusterNode private[akka] (
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, Transient, serializeMailbox, serializer)
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), nrOfInstances, Transient, serializeMailbox, serializer)
/**
* Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store[T <: Actor](actorAddress: String, actorClass: Class[T], replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), replicationFactor, replicationScheme, serializeMailbox, serializer)
def store[T <: Actor](actorAddress: String, actorClass: Class[T], nrOfInstances: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, () Actor.actorOf(actorClass, actorAddress), nrOfInstances, replicationScheme, serializeMailbox, serializer)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -589,24 +589,24 @@ class DefaultClusterNode private[akka] (
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, replicationFactor, Transient, false, serializer)
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, nrOfInstances, Transient, false, serializer)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, replicationFactor, replicationScheme, false, serializer)
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, replicationScheme: ReplicationScheme, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, nrOfInstances, replicationScheme, false, serializer)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
* with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly
* available durable store.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, replicationFactor, Transient, serializeMailbox, serializer)
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializeMailbox: Boolean, serializer: Serializer): ClusterNode =
store(actorAddress, actorFactory, nrOfInstances, Transient, serializeMailbox, serializer)
/**
* Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated
@ -619,14 +619,14 @@ class DefaultClusterNode private[akka] (
/**
* Needed to have reflection through structural typing work.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode =
store(actorAddress, actorFactory, replicationFactor, replicationScheme, serializeMailbox, serializer.asInstanceOf[Serializer])
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, replicationScheme: ReplicationScheme, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode =
store(actorAddress, actorFactory, nrOfInstances, replicationScheme, serializeMailbox, serializer.asInstanceOf[Serializer])
/**
* Needed to have reflection through structural typing work.
*/
def store(actorAddress: String, actorFactory: () ActorRef, replicationFactor: Int, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode =
store(actorAddress, actorFactory, replicationFactor, Transient, serializeMailbox, serializer)
def store(actorAddress: String, actorFactory: () ActorRef, nrOfInstances: Int, serializeMailbox: Boolean, serializer: AnyRef): ClusterNode =
store(actorAddress, actorFactory, nrOfInstances, Transient, serializeMailbox, serializer)
/**
* Clusters an actor. If the actor is already clustered then the clustered version will be updated
@ -636,7 +636,7 @@ class DefaultClusterNode private[akka] (
def store(
actorAddress: String,
actorFactory: () ActorRef,
replicationFactor: Int,
nrOfInstances: Int,
replicationScheme: ReplicationScheme,
serializeMailbox: Boolean,
serializer: Serializer): ClusterNode = {
@ -686,7 +686,7 @@ class DefaultClusterNode private[akka] (
// create ADDRESS -> UUIDs mapping
ignore[ZkNodeExistsException](zkClient.createPersistent(actorAddressToUuidsPathFor(actorAddress)))
useActorOnNodes(nodesForReplicationFactor(replicationFactor, Some(actorAddress)).toArray, actorAddress)
useActorOnNodes(nodesForNrOfInstances(nrOfInstances, Some(actorAddress)).toArray, actorAddress)
this
}
@ -1025,9 +1025,9 @@ class DefaultClusterNode private[akka] (
// =======================================
/**
* Send a function 'Function0[Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
* Send a function 'Function0[Unit]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument).
*/
def send(f: Function0[Unit], replicationFactor: Int) {
def send(f: Function0[Unit], nrOfInstances: Int) {
Serialization.serialize(f) match {
case Left(error) throw error
case Right(bytes)
@ -1035,15 +1035,15 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN0_UNIT)
.setPayload(ByteString.copyFrom(bytes))
.build
nodeConnectionsForReplicationFactor(replicationFactor) foreach (_ ! message)
nodeConnectionsForNrOfInstances(nrOfInstances) foreach (_ ! message)
}
}
/**
* Send a function 'Function0[Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument).
* Send a function 'Function0[Any]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument).
* Returns an 'Array' with all the 'Future's from the computation.
*/
def send(f: Function0[Any], replicationFactor: Int): List[Future[Any]] = {
def send(f: Function0[Any], nrOfInstances: Int): List[Future[Any]] = {
Serialization.serialize(f) match {
case Left(error) throw error
case Right(bytes)
@ -1051,16 +1051,16 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN0_ANY)
.setPayload(ByteString.copyFrom(bytes))
.build
val results = nodeConnectionsForReplicationFactor(replicationFactor) map (_ ? message)
val results = nodeConnectionsForNrOfInstances(nrOfInstances) map (_ ? message)
results.toList.asInstanceOf[List[Future[Any]]]
}
}
/**
* Send a function 'Function1[Any, Unit]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
* Send a function 'Function1[Any, Unit]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument)
* with the argument speficied.
*/
def send(f: Function1[Any, Unit], arg: Any, replicationFactor: Int) {
def send(f: Function1[Any, Unit], arg: Any, nrOfInstances: Int) {
Serialization.serialize((f, arg)) match {
case Left(error) throw error
case Right(bytes)
@ -1068,16 +1068,16 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN1_ARG_UNIT)
.setPayload(ByteString.copyFrom(bytes))
.build
nodeConnectionsForReplicationFactor(replicationFactor) foreach (_ ! message)
nodeConnectionsForNrOfInstances(nrOfInstances) foreach (_ ! message)
}
}
/**
* Send a function 'Function1[Any, Any]' to be invoked on a random number of nodes (defined by 'replicationFactor' argument)
* Send a function 'Function1[Any, Any]' to be invoked on a random number of nodes (defined by 'nrOfInstances' argument)
* with the argument speficied.
* Returns an 'Array' with all the 'Future's from the computation.
*/
def send(f: Function1[Any, Any], arg: Any, replicationFactor: Int): List[Future[Any]] = {
def send(f: Function1[Any, Any], arg: Any, nrOfInstances: Int): List[Future[Any]] = {
Serialization.serialize((f, arg)) match {
case Left(error) throw error
case Right(bytes)
@ -1085,7 +1085,7 @@ class DefaultClusterNode private[akka] (
.setMessageType(FUNCTION_FUN1_ARG_ANY)
.setPayload(ByteString.copyFrom(bytes))
.build
val results = nodeConnectionsForReplicationFactor(replicationFactor) map (_ ? message)
val results = nodeConnectionsForNrOfInstances(nrOfInstances) map (_ ? message)
results.toList.asInstanceOf[List[Future[Any]]]
}
}
@ -1211,16 +1211,16 @@ class DefaultClusterNode private[akka] (
private[cluster] def actorAddressToUuidsPathFor(actorAddress: String, uuid: UUID): String = "%s/%s".format(actorAddressToUuidsPathFor(actorAddress), uuid)
/**
* Returns a random set with node names of size 'replicationFactor'.
* Default replicationFactor is 0, which returns the empty Set.
* Returns a random set with node names of size 'nrOfInstances'.
* Default nrOfInstances is 0, which returns the empty Set.
*/
private def nodesForReplicationFactor(replicationFactor: Int = 0, actorAddress: Option[String] = None): Set[String] = {
private def nodesForNrOfInstances(nrOfInstances: Int = 0, actorAddress: Option[String] = None): Set[String] = {
var replicaNames = Set.empty[String]
val nrOfClusterNodes = nodeConnections.get.connections.size
if (replicationFactor < 1) return replicaNames
if (nrOfClusterNodes < replicationFactor) throw new IllegalArgumentException(
"Replication factor [" + replicationFactor +
if (nrOfInstances < 1) return replicaNames
if (nrOfClusterNodes < nrOfInstances) throw new IllegalArgumentException(
"Replication factor [" + nrOfInstances +
"] is greater than the number of available nodeNames [" + nrOfClusterNodes + "]")
val preferredNodes =
@ -1228,7 +1228,7 @@ class DefaultClusterNode private[akka] (
// use 'preferred-nodes' in deployment config for the actor
Deployer.deploymentFor(actorAddress.get) match {
case Deploy(_, _, _, _, Cluster(nodes, _, _))
nodes map (node DeploymentConfig.nodeNameFor(node)) take replicationFactor
nodes map (node DeploymentConfig.nodeNameFor(node)) take nrOfInstances
case _
throw new ClusterException("Actor [" + actorAddress.get + "] is not configured as clustered")
}
@ -1243,11 +1243,11 @@ class DefaultClusterNode private[akka] (
val nrOfCurrentReplicaNames = replicaNames.size
val replicaSet =
if (nrOfCurrentReplicaNames > replicationFactor) throw new IllegalStateException("Replica set is larger than replication factor")
else if (nrOfCurrentReplicaNames == replicationFactor) replicaNames
if (nrOfCurrentReplicaNames > nrOfInstances) throw new IllegalStateException("Replica set is larger than replication factor")
else if (nrOfCurrentReplicaNames == nrOfInstances) replicaNames
else {
val random = new java.util.Random(System.currentTimeMillis)
while (replicaNames.size < replicationFactor) {
while (replicaNames.size < nrOfInstances) {
replicaNames = replicaNames + membershipNodes(random.nextInt(nrOfClusterNodes))
}
replicaNames
@ -1260,12 +1260,12 @@ class DefaultClusterNode private[akka] (
}
/**
* Returns a random set with replica connections of size 'replicationFactor'.
* Default replicationFactor is 0, which returns the empty Set.
* Returns a random set with replica connections of size 'nrOfInstances'.
* Default nrOfInstances is 0, which returns the empty Set.
*/
private def nodeConnectionsForReplicationFactor(replicationFactor: Int = 0, actorAddress: Option[String] = None): Set[ActorRef] = {
private def nodeConnectionsForNrOfInstances(nrOfInstances: Int = 0, actorAddress: Option[String] = None): Set[ActorRef] = {
for {
node nodesForReplicationFactor(replicationFactor, actorAddress)
node nodesForNrOfInstances(nrOfInstances, actorAddress)
connectionOption nodeConnections.get.connections(node)
connection connectionOption
actorRef connection._2

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 1
akka.actor.deployment.service-hello.nr-of-instances = 1

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 1
akka.actor.deployment.service-hello.nr-of-instances = 1

View file

@ -3,5 +3,5 @@ akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-test.router = "round-robin"
akka.actor.deployment.service-test.cluster.preferred-nodes = ["node:node2","node:node3"]
akka.actor.deployment.service-test.cluster.replication-factor = 2
akka.actor.deployment.service-test.nr-of-instances = 2
akka.remote.client.buffering.retry-message-send-on-failure = false

View file

@ -2,5 +2,5 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-test.router = "round-robin"
akka.actor.deployment.service-test.cluster.preferred-nodes = ["node:node2","node:node3"]
akka.actor.deployment.service-test.cluster.replication-factor = 2
akka.actor.deployment.service-test.nr-of-instances = 2
akka.remote.client.buffering.retry-message-send-on-failure = false

View file

@ -2,5 +2,5 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-test.router = "round-robin"
akka.actor.deployment.service-test.cluster.preferred-nodes = ["node:node2","node:node3"]
akka.actor.deployment.service-test.cluster.replication-factor = 2
akka.actor.deployment.service-test.nr-of-instances = 2
akka.remote.client.buffering.retry-message-send-on-failure = false

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world-write-behind-nosnapshot.router = "direct"
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-behind-nosnapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication.strategy = "write-behind"
akka.cluster.replication.snapshot-frequency = 1000

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world-write-behind-nosnapshot.router = "direct"
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-behind-nosnapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-behind-nosnapshot.cluster.replication.strategy = "write-behind"
akka.cluster.replication.snapshot-frequency = 1000

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world.router = "direct"
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-behind-snapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication.strategy = "write-behind"
akka.cluster.replication.snapshot-frequency = 7

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world.router = "direct"
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-behind-snapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-behind-snapshot.cluster.replication.strategy = "write-behind"
akka.cluster.replication.snapshot-frequency = 7

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world-write-through-snapshot.router = "direct"
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-through-snapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication.strategy = "write-through"
akka.cluster.replication.snapshot-frequency = 7

View file

@ -1,7 +1,7 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.hello-world-write-through-snapshot.router = "direct"
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication-factor = 1
akka.actor.deployment.hello-world-write-through-snapshot.nr-of-instances = 1
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication.storage = "transaction-log"
akka.actor.deployment.hello-world-write-through-snapshot.cluster.replication.strategy = "write-through"
akka.cluster.replication.snapshot-frequency = 7

View file

@ -3,6 +3,6 @@ akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1", "node:node3"]
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.timeout = 30
akka.cluster.session-timeout = 10

View file

@ -3,6 +3,6 @@ akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1", "node:node3"]
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.timeout = 30
akka.cluster.session-timeout = 10

View file

@ -3,6 +3,6 @@ akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1", "node:node3"]
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.timeout = 30
akka.cluster.session-timeout = 10

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-node1.router = "random"
akka.actor.deployment.service-node1.cluster.preferred-nodes = ["node:node1"]
akka.actor.deployment.service-node1.cluster.replication-factor = 1
akka.actor.deployment.service-node1.nr-of-instances = 1
akka.actor.deployment.service-node2.router = "random"
akka.actor.deployment.service-node2.cluster.preferred-nodes = ["node:node2"]
akka.actor.deployment.service-node2.cluster.replication-factor = 1
akka.actor.deployment.service-node2.nr-of-instances = 1

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-node1.router = "random"
akka.actor.deployment.service-node1.cluster.preferred-nodes = ["node:node1"]
akka.actor.deployment.service-node1.cluster.replication-factor = 1
akka.actor.deployment.service-node1.nr-of-instances = 1
akka.actor.deployment.service-node2.router = "random"
akka.actor.deployment.service-node2.cluster.preferred-nodes = ["node:node2"]
akka.actor.deployment.service-node2.cluster.replication-factor = 1
akka.actor.deployment.service-node2.nr-of-instances = 1

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.replication-factor = 1
akka.actor.deployment.service-hello.nr-of-instances = 1

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.replication-factor = 3
akka.actor.deployment.service-hello.nr-of-instances = 3

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "random"
akka.actor.deployment.service-hello.cluster.replication-factor = 3
akka.actor.deployment.service-hello.nr-of-instances = 3

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1","node:node3"]
akka.cluster.include-ref-node-in-replica-set = on
akka.actor.timeout = 30

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1","node:node3"]
akka.cluster.include-ref-node-in-replica-set = on
akka.actor.timeout = 30

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1","node:node3"]
akka.cluster.include-ref-node-in-replica-set = on
akka.actor.timeout = 30

View file

@ -2,7 +2,7 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-node1.router = "round-robin"
akka.actor.deployment.service-node1.cluster.preferred-nodes = ["node:node1"]
akka.actor.deployment.service-node1.cluster.replication-factor = 1
akka.actor.deployment.service-node1.nr-of-instances = 1
akka.actor.deployment.service-node2.router = "round-robin"
akka.actor.deployment.service-node2.cluster.preferred-nodes = ["node:node2"]
akka.actor.deployment.service-node2.cluster.replication-factor = 1
akka.actor.deployment.service-node2.nr-of-instances = 1

View file

@ -2,4 +2,4 @@ akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.preferred-nodes = ["node:node1"]
akka.actor.deployment.service-hello.cluster.replication-factor = 1
akka.actor.deployment.service-hello.nr-of-instances = 1

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 1
akka.actor.deployment.service-hello.nr-of-instances = 1

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 3
akka.actor.deployment.service-hello.nr-of-instances = 3

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["cluster"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "round-robin"
akka.actor.deployment.service-hello.cluster.replication-factor = 3
akka.actor.deployment.service-hello.nr-of-instances = 3

View file

@ -2,5 +2,5 @@ akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "akka.routing.ScatterGatherFirstCompletedRouter"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.timeout = 30

View file

@ -2,5 +2,5 @@ akka.enabled-modules = ["cluster"]
akka.event-handlers = ["akka.testkit.TestEventListener"]
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.router = "akka.routing.ScatterGatherFirstCompletedRouter"
akka.actor.deployment.service-hello.cluster.replication-factor = 2
akka.actor.deployment.service-hello.nr-of-instances = 2
akka.actor.timeout = 30

View file

@ -296,7 +296,7 @@ are:
- ``remove`` -- removes the actor from the clustered actor registry
The ``store`` method also allows you to specify a replication factor. The
``replicationFactor`` defines the number of (randomly picked) nodes in the cluster that
``nrOfInstances`` defines the number of (randomly picked) nodes in the cluster that
the stored actor should be automatically deployed to and instantiated locally on (using
``use``). If you leave this argument out then a replication factor of ``0`` will be used
which means that the actor will only be stored in the clustered actor registry and not
@ -310,11 +310,11 @@ on your use-case. Default is ``false``
This is the signatures for the ``store`` method (all different permutations of these methods are available for using from Java)::
def store[T <: Actor]
(actorRef: ActorRef, replicationFactor: Int = 0, serializeMailbox: Boolean = false)
(actorRef: ActorRef, nrOfInstances: Int = 0, serializeMailbox: Boolean = false)
(implicit format: Format[T]): ClusterNode
def store[T <: Actor]
(actorClass: Class[T], replicationFactor: Int = 0, serializeMailbox: Boolean = false)
(actorClass: Class[T], nrOfInstances: Int = 0, serializeMailbox: Boolean = false)
(implicit format: Format[T]): ClusterNode
The ``implicit format: Format[T]`` might look scary but this argument is chosen for you and passed in automatically by the compiler as long as you have imported the serialization typeclass for the actor you are storing, e.g. the ``HelloActorFormat`` (defined above and imported in the sample below).
@ -331,9 +331,9 @@ created actor::
val hello = actorOf[HelloActor].start.asInstanceOf[LocalActorRef]
val serializeMailbox = false
val replicationFactor = 5
val nrOfInstances = 5
clusterNode store (hello, serializeMailbox, replicationFactor)
clusterNode store (hello, serializeMailbox, nrOfInstances)
Here is an example of how to use ``store`` to cluster an actor by type::
@ -444,7 +444,7 @@ The workhorse for this is the ``send`` method (in different variations). The
``send`` methods take the following parameters:
- ``f`` -- the function you want to be invoked on the remote nodes in the cluster
- ``arg`` -- the argument to the function (not all of them have this parameter)
- ``replicationFactor`` -- the replication factor defining the number of nodes you want the function to be sent and invoked on
- ``nrOfInstances`` -- the replication factor defining the number of nodes you want the function to be sent and invoked on
You can currently send these function types to the cluster:
- ``Function0[Unit]`` -- takes no arguments and returns nothing

View file

@ -31,7 +31,6 @@ class RemoteActorRefProvider extends ActorRefProvider {
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
// FIXME who evicts this registry, and when? Should it be used instead of ActorRegistry?
private val actors = new ConcurrentHashMap[String, Promise[Option[ActorRef]]]
private val failureDetector = new BannagePeriodFailureDetector(timeToBan = 60 seconds) // FIXME make timeToBan configurable
@ -45,7 +44,7 @@ class RemoteActorRefProvider extends ActorRefProvider {
if (oldFuture eq null) { // we won the race -- create the actor and resolve the future
val actor = try {
Deployer.lookupDeploymentFor(address) match {
case Some(Deploy(_, _, router, _, RemoteScope(host, port)))
case Some(Deploy(_, _, router, nrOfInstances, _, RemoteScope(host, port)))
// FIXME create RoutedActorRef if 'router' is specified
val serverAddress = Remote.address

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["remote"]
akka.event-handler-level = "DEBUG"
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.remote.hostname = "localhost"
akka.actor.deployment.service-hello.remote.port = 9991

View file

@ -1,4 +1,4 @@
akka.enabled-modules = ["remote"]
akka.event-handler-level = "DEBUG"
akka.event-handler-level = "WARNING"
akka.actor.deployment.service-hello.remote.hostname = "localhost"
akka.actor.deployment.service-hello.remote.port = 9991

View file

@ -119,7 +119,7 @@ trait TestKitLight {
* Stop test actor. Should be done at the end of the test unless relying on
* test actor timeout.
*/
def stopTestActor { testActor.stop() }
def stopTestActor() { testActor.stop() }
/**
* Set test actor timeout. By default, the test actor shuts itself down
@ -144,7 +144,7 @@ trait TestKitLight {
/**
* Stop ignoring messages in the test actor.
*/
def ignoreNoMsg { testActor ! TestActor.SetIgnore(None) }
def ignoreNoMsg() { testActor ! TestActor.SetIgnore(None) }
/**
* Obtain current time (`System.nanoTime`) as Duration.
@ -431,7 +431,7 @@ trait TestKitLight {
/**
* Same as `expectNoMsg(remaining)`, but correctly treating the timeFactor.
*/
def expectNoMsg { expectNoMsg_internal(remaining) }
def expectNoMsg() { expectNoMsg_internal(remaining) }
/**
* Assert that no message is received for the specified time.

View file

@ -61,7 +61,7 @@ akka {
deployment {
service-ping { # stateless actor with replication factor 3 and round-robin load-balancer
service-ping { # deployment id pattern
router = "least-cpu" # routing (load-balance) scheme to use
# available: "direct", "round-robin", "random",
@ -70,6 +70,11 @@ akka {
# default is "direct";
# if 'replication' is used then the only available router is "direct"
nr-of-instances = 3 # number of actor instances in the cluster
# available: positive integer (1-N) or the string "auto" for auto-scaling
# default is '1'
# if the "direct" router is used then this element is ignored (always '1')
failure-detector { # failure detection scheme to use
bannage-period { # available: remove-connection-on-first-local-failure {}
time-to-ban = 10 # remove-connection-on-first-failure {}
@ -100,11 +105,6 @@ akka {
# defined as node name
# available: "node:<node name>"
# replication-factor = 3 # number of actor instances in the cluster
# available: positive integer (0-N) or the string "auto" for auto-scaling
# if "auto" is used then 'home' has no meaning
# default is '0', meaning no replicas;
# if the "direct" router is used then this element is ignored (always '1')
# replication { # use replication or not? only makes sense for a stateful actor