Merge commit
This commit is contained in:
commit
a12ee36151
57 changed files with 596 additions and 275 deletions
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)))))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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 ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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 ⇒
|
||||
|
|
|
|||
|
|
@ -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é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))
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -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
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -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";
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue