Merge remote-tracking branch 'origin/1428-RoutedActorRef-henrikengstrom' into wip-remote-supervision-rk

This commit is contained in:
Roland 2011-12-09 15:13:35 +01:00
commit b84a35456d
35 changed files with 597 additions and 1909 deletions

View file

@ -96,7 +96,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
Deploy(
service,
None,
Direct,
NoRouting,
NrOfInstances(1),
LocalScope)))
}
@ -116,7 +116,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
Deploy(
service,
Some(ActorRecipe(classOf[DeployerSpec.RecipeActor])),
Direct,
NoRouting,
NrOfInstances(1),
LocalScope)))
}
@ -151,11 +151,11 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
}
"be able to parse 'akka.actor.deployment._' with direct router" in {
assertRouting(Direct, "/user/service-direct")
assertRouting(NoRouting, "/user/service-direct")
}
"ignore nr-of-instances with direct router" in {
assertRouting(Direct, "/user/service-direct2")
assertRouting(NoRouting, "/user/service-direct2")
}
"be able to parse 'akka.actor.deployment._' with round-robin router" in {

View file

@ -7,144 +7,109 @@ package akka.actor
import akka.testkit._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class HotSwapSpec extends AkkaSpec {
class HotSwapSpec extends AkkaSpec with ImplicitSender {
"An Actor" must {
"be able to hotswap its behavior with HotSwap(..)" in {
val barrier = TestBarrier(2)
@volatile
var _log = ""
val a = system.actorOf(new Actor {
def receive = { case _ _log += "default" }
})
a ! HotSwap(context {
case _
_log += "swapped"
barrier.await
def receive = { case _ sender ! "default" }
})
a ! HotSwap(context { case _ context.sender ! "swapped" })
a ! "swapped"
barrier.await
_log must be("swapped")
expectMsg("swapped")
}
"be able to hotswap its behavior with become(..)" in {
val barrier = TestBarrier(2)
@volatile
var _log = ""
val a = system.actorOf(new Actor {
def receive = {
case "init"
_log += "init"
barrier.await
case "swap" context.become({
case _
_log += "swapped"
barrier.await
})
case "init" sender ! "init"
case "swap" context.become({ case x: String context.sender ! x })
}
})
a ! "init"
barrier.await
_log must be("init")
barrier.reset
_log = ""
expectMsg("init")
a ! "swap"
a ! "swapped"
barrier.await
_log must be("swapped")
expectMsg("swapped")
}
"be able to revert hotswap its behavior with RevertHotSwap(..)" in {
val barrier = TestBarrier(2)
@volatile
var _log = ""
val a = system.actorOf(new Actor {
def receive = {
case "init"
_log += "init"
barrier.await
case "init" sender ! "init"
}
})
a ! "init"
barrier.await
_log must be("init")
barrier.reset
_log = ""
a ! HotSwap(context {
case "swapped"
_log += "swapped"
barrier.await
})
expectMsg("init")
a ! HotSwap(context { case "swapped" context.sender ! "swapped" })
a ! "swapped"
barrier.await
_log must be("swapped")
expectMsg("swapped")
barrier.reset
_log = ""
a ! RevertHotSwap
a ! "init"
barrier.await
_log must be("init")
expectMsg("init")
// try to revert hotswap below the bottom of the stack
barrier.reset
_log = ""
a ! RevertHotSwap
a ! "init"
barrier.await
_log must be("init")
expectMsg("init")
}
"be able to revert hotswap its behavior with unbecome" in {
val barrier = TestBarrier(2)
@volatile
var _log = ""
val a = system.actorOf(new Actor {
def receive = {
case "init"
_log += "init"
barrier.await
case "init" sender ! "init"
case "swap"
context.become({
case "swapped"
_log += "swapped"
barrier.await
sender ! "swapped"
case "revert"
context.unbecome()
})
barrier.await
}
})
a ! "init"
barrier.await
_log must be("init")
barrier.reset
_log = ""
expectMsg("init")
a ! "swap"
barrier.await
barrier.reset
_log = ""
a ! "swapped"
barrier.await
_log must be("swapped")
expectMsg("swapped")
barrier.reset
_log = ""
a ! "revert"
a ! "init"
barrier.await
_log must be("init")
expectMsg("init")
}
"revert to initial state on restart" in {
val a = system.actorOf(new Actor {
def receive = {
case "state" sender ! "0"
case "swap"
context.become({
case "state" sender ! "1"
case "swapped" sender ! "swapped"
case "crash" throw new Exception("Crash (expected)!")
})
sender ! "swapped"
}
})
a ! "state"
expectMsg("0")
a ! "swap"
expectMsg("swapped")
a ! "state"
expectMsg("1")
EventFilter[Exception](message = "Crash (expected)!", occurrences = 1) intercept { a ! "crash" }
a ! "state"
expectMsg("0")
}
}
}

View file

@ -7,7 +7,6 @@ import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import java.util.Random
import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistics
@ -23,9 +22,6 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
.build
val repeat = 200L * repeatFactor
def clientDelayMicros = {
System.getProperty("benchmark.clientDelayMicros", "250").toInt
}
var stat: DescriptiveStatistics = _
@ -67,19 +63,19 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
val w3 = system.actorOf(new Waypoint(w4))
val w2 = system.actorOf(new Waypoint(w3))
val w1 = system.actorOf(new Waypoint(w2))
Props(new Client(w1, latch, repeatsPerClient, clientDelayMicros, stat)).withDispatcher(clientDispatcher)
Props(new Client(w1, latch, repeatsPerClient, clientDelay.toMicros.intValue, stat)).withDispatcher(clientDispatcher)
}).toList.map(system.actorOf(_))
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, stat)
}
clients.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
}
}

View file

@ -42,13 +42,6 @@ class TellThroughput10000PerformanceSpec extends PerformanceSpec {
val clientDispatcher = createDispatcher("client-dispatcher")
//val destinationDispatcher = createDispatcher("destination-dispatcher")
override def atTermination {
super.atTermination()
System.out.println("Cleaning up after TellThroughputPerformanceSpec")
clientDispatcher.shutdown()
//destinationDispatcher.shutdown()
}
val repeat = 30000L * repeatFactor
"Tell" must {
@ -152,7 +145,7 @@ class TellThroughput10000PerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!ok) {
@ -180,8 +173,8 @@ class TellThroughput10000PerformanceSpec extends PerformanceSpec {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(_ ! PoisonPill)
destinations.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
destinations.foreach(_.stop())
}
}

View file

@ -22,13 +22,6 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
val clientDispatcher = createDispatcher("client-dispatcher")
val destinationDispatcher = createDispatcher("destination-dispatcher")
override def atTermination {
super.atTermination()
System.out.println("Cleaning up after TellThroughputComputationPerformanceSpec")
clientDispatcher.shutdown()
destinationDispatcher.shutdown()
}
val repeat = 500L * repeatFactor
"Tell" must {
@ -126,7 +119,7 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!ok) {
@ -154,8 +147,8 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(_ ! PoisonPill)
destinations.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
destinations.foreach(_.stop())
}
}
@ -211,7 +204,7 @@ object TellThroughputComputationPerformanceSpec {
actor ! Msg
sent += 1
} else if (received >= repeat) {
println("PI: " + pi)
//println("PI: " + pi)
latch.countDown()
}
case Run

View file

@ -22,13 +22,6 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
val clientDispatcher = createDispatcher("client-dispatcher")
val destinationDispatcher = createDispatcher("destination-dispatcher")
override def atTermination {
super.atTermination()
System.out.println("Cleaning up after TellThroughputPerformanceSpec")
clientDispatcher.shutdown()
destinationDispatcher.shutdown()
}
val repeat = 30000L * repeatFactor
"Tell" must {
@ -78,36 +71,15 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!ok) {
System.err.println("Destinations: ")
destinations.foreach {
case l: LocalActorRef
val m = l.underlying.mailbox
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
}
System.err.println("")
System.err.println("Clients: ")
clients.foreach {
case l: LocalActorRef
val m = l.underlying.mailbox
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
}
val e = clientDispatcher.asInstanceOf[Dispatcher].executorService.get().asInstanceOf[ExecutorServiceDelegate].executor.asInstanceOf[ThreadPoolExecutor]
val q = e.getQueue
System.err.println("Client Dispatcher: " + e.getActiveCount + " " + Stream.continually(q.poll()).takeWhile(_ != null).mkString(", "))
}
if (!warmup) {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(_ ! PoisonPill)
destinations.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
destinations.foreach(_.stop())
}
}

View file

@ -26,13 +26,6 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
//val clientDispatcher = createDispatcher("client-dispatcher")
//val destinationDispatcher = createDispatcher("destination-dispatcher")
override def atTermination {
super.atTermination()
System.out.println("Cleaning up after TellThroughputPerformanceSpec")
//clientDispatcher.shutdown()
//destinationDispatcher.shutdown()
}
val repeat = 30000L * repeatFactor
"Tell" must {
@ -138,7 +131,7 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
clients.foreach(_ ! Run)
val ok = latch.await((5000000 + 500 * repeat) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!ok) {
@ -166,8 +159,8 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
ok must be(true)
logMeasurement(numberOfClients, durationNs, repeat)
}
clients.foreach(_ ! PoisonPill)
destinations.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
destinations.foreach(_.stop())
}
}

View file

@ -1,4 +1,5 @@
package akka.performance.trading.domain
import akka.performance.workbench.BenchmarkConfig
abstract class Orderbook(val symbol: String) {
var bidSide: List[Bid] = Nil
@ -49,7 +50,7 @@ abstract class Orderbook(val symbol: String) {
object Orderbook {
val useDummyOrderbook = System.getProperty("benchmark.useDummyOrderbook", "false").toBoolean
val useDummyOrderbook = BenchmarkConfig.config.getBoolean("benchmark.useDummyOrderbook")
def apply(symbol: String, standby: Boolean): Orderbook = (useDummyOrderbook, standby) match {
case (false, false) new Orderbook(symbol) with NopTradeObserver

View file

@ -8,7 +8,6 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.performance.trading.domain.Ask
import akka.performance.trading.domain.Bid
@ -31,10 +30,6 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
var stat: DescriptiveStatistics = _
val random: Random = new Random(0)
def clientDelayMicros = {
System.getProperty("benchmark.clientDelayMicros", "250").toInt
}
override def beforeEach() {
super.beforeEach()
stat = new SynchronizedDescriptiveStatistics
@ -98,12 +93,12 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
val start = System.nanoTime
val clients = (for (i 0 until numberOfClients) yield {
val receiver = receivers(i % receivers.size)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelayMicros)).withDispatcher(clientDispatcher)
val props = Props(new Client(receiver, orders, latch, ordersPerClient, clientDelay.toMicros.toInt)).withDispatcher(clientDispatcher)
system.actorOf(props)
})
clients.foreach(_ ! "run")
val ok = latch.await((5000000L + (clientDelayMicros + 500) * totalNumberOfOrders) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
@ -113,7 +108,7 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
}
logMeasurement(numberOfClients, durationNs, stat)
}
clients.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
}
}

View file

@ -8,7 +8,6 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
import org.junit.runner.RunWith
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.performance.trading.domain.Ask
import akka.performance.trading.domain.Bid
@ -96,7 +95,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
})
clients.foreach(_ ! "run")
val ok = latch.await((5000000L + 500 * totalNumberOfOrders) * timeDilation, TimeUnit.MICROSECONDS)
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
val durationNs = (System.nanoTime - start)
if (!warmup) {
@ -106,7 +105,7 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
}
logMeasurement(numberOfClients, durationNs, totalNumberOfOrders)
}
clients.foreach(_ ! PoisonPill)
clients.foreach(_.stop())
}
}

View file

@ -41,9 +41,10 @@ class FileBenchResultRepository extends BenchResultRepository {
private val statsByName = MutableMap[String, Seq[Stats]]()
private val baselineStats = MutableMap[Key, Stats]()
private val historicalStats = MutableMap[Key, Seq[Stats]]()
private val serDir = System.getProperty("benchmark.resultDir", "target/benchmark") + "/ser"
private def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir")
private val serDir = resultDir + "/ser"
private def serDirExists: Boolean = new File(serDir).exists
private val htmlDir = System.getProperty("benchmark.resultDir", "target/benchmark") + "/html"
private val htmlDir = resultDir + "/html"
private def htmlDirExists: Boolean = new File(htmlDir).exists
protected val maxHistorical = 7

View file

@ -0,0 +1,32 @@
package akka.performance.workbench
import com.typesafe.config.ConfigFactory
object BenchmarkConfig {
private val benchmarkConfig = ConfigFactory.parseString("""
benchmark {
longRunning = false
minClients = 1
maxClients = 4
repeatFactor = 2
timeDilation = 1
maxRunDuration = 10 seconds
clientDelay = 250000 nanoseconds
logResult = true
resultDir = "target/benchmark"
useDummyOrderbook = false
}
""")
private val longRunningBenchmarkConfig = ConfigFactory.parseString("""
benchmark {
longRunning = true
maxClients = 48
repeatFactor = 150
maxRunDuration = 120 seconds
useDummyOrderbook = true
}
""").withFallback(benchmarkConfig)
def config = if (System.getProperty("benchmark.longRunning") == "true")
longRunningBenchmarkConfig else benchmarkConfig
}

View file

@ -1,33 +1,25 @@
package akka.performance.workbench
import scala.collection.immutable.TreeMap
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.scalatest.BeforeAndAfterEach
import akka.actor.simpleName
import akka.testkit.AkkaSpec
import akka.actor.ActorSystem
import akka.util.Duration
import com.typesafe.config.Config
import java.util.concurrent.TimeUnit
trait PerformanceSpec extends AkkaSpec with BeforeAndAfterEach {
abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends AkkaSpec(cfg) with BeforeAndAfterEach {
def isBenchmark() = System.getProperty("benchmark") == "true"
def minClients() = System.getProperty("benchmark.minClients", "1").toInt;
def maxClients() = {
val default = if (isBenchmark) "48" else "4"
System.getProperty("benchmark.maxClients", default).toInt;
}
def repeatFactor() = {
val defaultRepeatFactor = if (isBenchmark) "150" else "2"
System.getProperty("benchmark.repeatFactor", defaultRepeatFactor).toInt
}
def timeDilation() = {
System.getProperty("benchmark.timeDilation", "1").toLong
}
def config = system.settings.config
def isLongRunningBenchmark() = config.getBoolean("benchmark.longRunning")
def minClients() = config.getInt("benchmark.minClients")
def maxClients() = config.getInt("benchmark.maxClients")
def repeatFactor() = config.getInt("benchmark.repeatFactor")
def timeDilation() = config.getLong("benchmark.timeDilation")
def maxRunDuration() = Duration(config.getMilliseconds("benchmark.maxRunDuration"), TimeUnit.MILLISECONDS)
def clientDelay = Duration(config.getNanoseconds("benchmark.clientDelay"), TimeUnit.NANOSECONDS)
val resultRepository = BenchResultRepository()
lazy val report = new Report(system, resultRepository, compareResultWith)

View file

@ -12,7 +12,7 @@ class Report(
resultRepository: BenchResultRepository,
compareResultWith: Option[String] = None) {
private def doLog = System.getProperty("benchmark.logResult", "true").toBoolean
private def doLog = system.settings.config.getBoolean("benchmark.logResult")
val log = Logging(system, "Report")
val dateTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
@ -189,12 +189,8 @@ class Report(
val sb = new StringBuilder
sb.append("Benchmark properties:")
import scala.collection.JavaConversions._
val propNames: Seq[String] = System.getProperties.propertyNames.toSeq.map(_.toString)
for (name propNames if name.startsWith("benchmark")) {
sb.append("\n ").append(name).append("=").append(System.getProperty(name))
}
sb.append("Benchmark properties:\n")
sb.append(system.settings.config.getConfig("benchmark").root.render)
sb.append("\n")
sb.append("Operating system: ").append(os.getName).append(", ").append(os.getArch).append(", ").append(os.getVersion)
@ -215,16 +211,15 @@ class Report(
append(")").append(" MB")
sb.append("\n")
val args = runtime.getInputArguments.filterNot(_.contains("classpath")).mkString("\n ")
import scala.collection.JavaConverters._
val args = runtime.getInputArguments.asScala.filterNot(_.contains("classpath")).mkString("\n ")
sb.append("Args:\n ").append(args)
sb.append("\n")
sb.append("Akka version: ").append(system.settings.ConfigVersion)
sb.append("\n")
sb.append("Akka config:")
for ((key, value) system.settings.config.root) {
sb.append("\n ").append(key).append("=").append(value)
}
sb.append("Akka config:\n")
sb.append(system.settings.toString)
sb.toString
}

View file

@ -26,6 +26,8 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
import akka.routing.RoutingSpec._
// TODO (HE) : Update test with new routing functionality
/*
"direct router" must {
"be started when constructed" in {
val actor1 = system.actorOf[TestActor]
@ -477,4 +479,5 @@ class RoutingSpec extends AkkaSpec with DefaultTimeout {
counter2.get must be(1)
}
}
*/
}

View file

@ -47,6 +47,11 @@ trait ActorContext extends ActorRefFactory {
def self: ActorRef
/**
* Retrieve the Props which were used to create this actor.
*/
def props: Props
/**
* Gets the current receive timeout
* When specified, the receive method should be able to handle a 'ReceiveTimeout' message.
@ -364,6 +369,7 @@ private[akka] final class ActorCell(
}
}
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
hotswap = Props.noHotSwap // Reset the behavior
freshActor.postRestart(cause)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "restarted"))
@ -534,6 +540,7 @@ private[akka] final class ActorCell(
} finally {
currentMessage = null
clearActorFields()
hotswap = Props.noHotSwap
}
}
}

View file

@ -212,6 +212,8 @@ private[akka] class LocalActorRef private[akka] (
private var actorCell = new ActorCell(system, this, _props, _supervisor, _receiveTimeout, _hotswap)
actorCell.start()
protected def actorContext: ActorContext = actorCell
/**
* Is the actor terminated?
* If this method returns true, it will never return false again, but if it

View file

@ -19,6 +19,7 @@ import akka.event._
import akka.event.Logging.Error._
import akka.event.Logging.Warning
import java.io.Closeable
import com.typesafe.config.Config
/**
* Interface for all ActorRef providers to implement.
@ -276,7 +277,7 @@ trait ActorRefFactory {
* ...
* val target = context.actorFor(Seq("..", "myBrother", "myNephew"))
* ...
* }
* }
* }
* }}}
*
@ -299,7 +300,7 @@ trait ActorRefFactory {
* path.add("myNephew");
* final ActorRef target = context().actorFor(path);
* ...
* }
* }
* }
* }}}
*
@ -406,10 +407,14 @@ class LocalActorRefProvider(
private class Guardian extends Actor {
def receive = {
case Terminated(_) context.self.stop()
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e })
case m deadLetters ! DeadLetter(m, sender, self)
case Terminated(_) context.self.stop()
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch {
case e: Exception e
})
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch {
case e: Exception e
})
case m deadLetters ! DeadLetter(m, sender, self)
}
}
@ -418,9 +423,13 @@ class LocalActorRefProvider(
case Terminated(_)
eventStream.stopDefaultLoggers()
context.self.stop()
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e })
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch { case e: Exception e })
case m deadLetters ! DeadLetter(m, sender, self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch {
case e: Exception e
})
case CreateRandomNameChild(child) sender ! (try context.actorOf(child) catch {
case e: Exception e
})
case m deadLetters ! DeadLetter(m, sender, self)
}
}
@ -456,6 +465,7 @@ class LocalActorRefProvider(
def unapply(s: String): Option[InternalActorRef] = extraNames.get(s)
}
override def getParent: InternalActorRef = this
override def getSingleChild(name: String): InternalActorRef = {
name match {
case "temp" tempContainer
@ -505,57 +515,17 @@ class LocalActorRefProvider(
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, name: String, systemService: Boolean): InternalActorRef = {
val path = supervisor.path / name
(if (systemService) None else deployer.lookupDeployment(path.toString)) match {
// create a local actor
case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, DeploymentConfig.LocalScope))
new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
// create a routed actor ref
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope))
implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
implicit val timeout = system.settings.ActorTimeout
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.Broadcast () new BroadcastRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()(
if (props.dispatcher == Props.defaultDispatcher) dispatcher else props.dispatcher, settings.ActorTimeout)
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(implClass) () Routing.createCustomRouter(implClass)
}
val connections: Iterable[ActorRef] = (1 to nrOfInstances.factor) map { i
val routedPath = path.parent / (path.name + ":" + i)
new LocalActorRef(system, props, supervisor, routedPath, systemService)
}
actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), supervisor, path.name)
case unknown throw new Exception("Don't know how to create this actor ref! Why? Got: " + unknown)
props.routerConfig match {
case NoRouting new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
case routedActor new RoutedActorRef(system, props.withRouting(adaptFromDeploy(routedActor, path)), supervisor, path)
}
}
/**
* Creates (or fetches) a routed actor reference, configured by the 'props: RoutedProps' configuration.
*/
def actorOf(system: ActorSystem, props: RoutedProps, supervisor: InternalActorRef, name: String): InternalActorRef = {
// FIXME: this needs to take supervision into account!
//FIXME clustering should be implemented by cluster actor ref provider
//TODO Implement support for configuring by deployment ID etc
//TODO If address matches an already created actor (Ahead-of-time deployed) return that actor
//TODO If address exists in config, it will override the specified Props (should we attempt to merge?)
//TODO If the actor deployed uses a different config, then ignore or throw exception?
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
// val clusteringEnabled = ReflectiveAccess.ClusterModule.isEnabled
// val localOnly = props.localOnly
// if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props)
// else new RoutedActorRef(props, address)
new RoutedActorRef(system, props, supervisor, name)
private def adaptFromDeploy(r: RouterConfig, p: ActorPath): RouterConfig = {
val lookupPath = p.elements.mkString("/", "/", "")
val deploy = deployer.instance.lookupDeployment(lookupPath)
r.adaptFromDeploy(deploy)
}
private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch
@ -607,6 +577,7 @@ class LocalDeathWatch extends DeathWatch with ActorClassification {
* returned from stop().
*/
class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter, dispatcher: MessageDispatcher) extends Scheduler with Closeable {
import org.jboss.netty.akka.util.{ Timeout HWTimeout }
def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable =

View file

@ -109,7 +109,6 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
// akka.actor.deployment.<path>.router
// --------------------------------
val router: Routing = deploymentWithFallback.getString("router") match {
case "direct" Direct
case "round-robin" RoundRobin
case "random" Random
case "scatter-gather" ScatterGather
@ -123,7 +122,7 @@ class Deployer(val settings: ActorSystem.Settings, val eventStream: EventStream,
// akka.actor.deployment.<path>.nr-of-instances
// --------------------------------
val nrOfInstances = {
if (router == Direct) OneNrOfInstances
if (router == NoRouting) OneNrOfInstances
else {
def invalidNrOfInstances(wasValue: Any) = new ConfigurationException(
"Config option [" + deploymentKey +

View file

@ -15,7 +15,7 @@ object DeploymentConfig {
case class Deploy(
path: String,
recipe: Option[ActorRecipe],
routing: Routing = Direct,
routing: Routing = NoRouting,
nrOfInstances: NrOfInstances = ZeroNrOfInstances,
scope: Scope = LocalScope)
@ -31,7 +31,7 @@ object DeploymentConfig {
case class CustomRouter(routerClassName: String) extends Routing
// For Java API
case class Direct() extends Routing
case class NoRouting() extends Routing
case class RoundRobin() extends Routing
case class Random() extends Routing
case class ScatterGather() extends Routing
@ -40,7 +40,7 @@ object DeploymentConfig {
case class LeastMessages() extends Routing
// For Scala API
case object Direct extends Routing
case object NoRouting extends Routing
case object RoundRobin extends Routing
case object Random extends Routing
case object ScatterGather extends Routing
@ -156,7 +156,7 @@ object DeploymentConfig {
}
def routerTypeFor(routing: Routing): RouterType = routing match {
case _: Direct | Direct RouterType.Direct
case _: NoRouting | NoRouting RouterType.NoRouting
case _: RoundRobin | RoundRobin RouterType.RoundRobin
case _: Random | Random RouterType.Random
case _: ScatterGather | ScatterGather RouterType.ScatterGather

View file

@ -8,6 +8,7 @@ import akka.dispatch._
import akka.japi.Creator
import akka.util._
import collection.immutable.Stack
import akka.routing.{ NoRouting, RouterConfig, RoutedProps }
/**
* ActorRef configuration object, this is threadsafe and fully sharable
@ -27,6 +28,9 @@ object Props {
case _: Exception Restart
case _ Escalate
}
final val defaultRoutedProps: RouterConfig = NoRouting
final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(defaultDecider, None, None)
final val noHotSwap: Stack[Actor.Receive] = Stack.empty
@ -80,7 +84,8 @@ object Props {
case class Props(creator: () Actor = Props.defaultCreator,
@transient dispatcher: MessageDispatcher = Props.defaultDispatcher,
timeout: Timeout = Props.defaultTimeout,
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler) {
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler,
routerConfig: RouterConfig = Props.defaultRoutedProps) {
/**
* No-args constructor that sets all the default values
* Java API
@ -89,7 +94,8 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
creator = Props.defaultCreator,
dispatcher = Props.defaultDispatcher,
timeout = Props.defaultTimeout,
faultHandler = Props.defaultFaultHandler)
faultHandler = Props.defaultFaultHandler,
routerConfig = Props.defaultRoutedProps)
/**
* Returns a new Props with the specified creator set
@ -127,4 +133,9 @@ case class Props(creator: () ⇒ Actor = Props.defaultCreator,
*/
def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f)
/**
* Returns a new Props with the specified router config set
* Java API
*/
def withRouting(r: RouterConfig) = copy(routerConfig = r)
}

View file

@ -129,7 +129,7 @@ trait LoggingBus extends ActorEventBus {
val level = _logLevel // volatile access before reading loggers
if (!(loggers contains StandardOutLogger)) {
AllLogLevels filter (level >= _) foreach (l subscribe(StandardOutLogger, classFor(l)))
publish(Info(simpleName(this), "shutting down: StandardOutLogger started"))
publish(Debug(simpleName(this), "shutting down: StandardOutLogger started"))
}
for {
logger loggers
@ -139,7 +139,7 @@ trait LoggingBus extends ActorEventBus {
unsubscribe(logger)
logger.stop()
}
publish(Info(simpleName(this), "all default loggers stopped"))
publish(Debug(simpleName(this), "all default loggers stopped"))
}
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {

View file

@ -4,16 +4,15 @@
package akka.routing
import akka.AkkaException
import akka.actor._
import akka.config.ConfigurationException
import akka.dispatch.{ Future, MessageDispatcher }
import akka.util.{ ReflectiveAccess, Duration }
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger }
import scala.annotation.tailrec
import akka.japi.Creator
import akka.util.ReflectiveAccess
import java.lang.reflect.InvocationTargetException
import akka.config.ConfigurationException
import akka.routing.Routing.Broadcast
import akka.actor.DeploymentConfig.Deploy
import java.util.concurrent.atomic.AtomicInteger
sealed trait RouterType
@ -24,7 +23,10 @@ sealed trait RouterType
*/
object RouterType {
object Direct extends RouterType
/**
* A RouterType that indicates no routing - i.e. direct message.
*/
object NoRouting extends RouterType
/**
* A RouterType that randomly selects a connection to send a message to.
@ -65,6 +67,7 @@ object RouterType {
* A user-defined custom RouterType.
*/
case class Custom(implClass: String) extends RouterType
}
/**
@ -73,20 +76,94 @@ object RouterType {
*/
case class RoutedProps private[akka] (
routerFactory: () Router,
connectionManager: ConnectionManager,
timeout: Timeout = RoutedProps.defaultTimeout,
localOnly: Boolean = RoutedProps.defaultLocalOnly) {
connectionManager: ConnectionManager) {
// Java API
def this(creator: Creator[Router], connectionManager: ConnectionManager, timeout: Timeout, localOnly: Boolean) {
this(() creator.create(), connectionManager, timeout, localOnly)
def this(creator: Creator[Router], connectionManager: ConnectionManager) {
this(() creator.create(), connectionManager)
}
}
object RoutedProps {
final val defaultTimeout = Timeout(Duration.MinusInf)
final val defaultLocalOnly = false
///**
// * The Router is responsible for sending a message to one (or more) of its connections. Connections are stored in the
// * {@link FailureDetector} and each Router should be linked to only one {@link FailureDetector}.
// *
// * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
// */
//trait Router {
//
// /**
// * Initializes this Router with a given set of Connections. The Router can use this datastructure to ask for
// * the current connections, signal that there were problems with one of the connections and see if there have
// * been changes in the connections.
// *
// * This method is not threadsafe, and should only be called once
// *
// * JMM Guarantees:
// * This method guarantees that all changes made in this method, are visible before one of the routing methods is called.
// */
// def init(connectionManager: ConnectionManager)
//
// /**
// * Routes the message to one of the connections.
// *
// * @throws RoutingException if something goes wrong while routing the message
// */
// def route(message: Any)(implicit sender: ActorRef)
//
// /**
// * Routes the message using a timeout to one of the connections and returns a Future to synchronize on the
// * completion of the processing of the message.
// *
// * @throws RoutingExceptionif something goes wrong while routing the message.
// */
// def route[T](message: Any, timeout: Timeout): Future[T]
//}
//
///**
// * An {@link AkkaException} thrown when something goes wrong while routing a message
// */
//class RoutingException(message: String) extends AkkaException(message)
//
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors.
*/
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
extends LocalActorRef(
_system,
_props.copy(creator = _props.routerConfig),
_supervisor,
_path) {
val route: Routing.Route = _props.routerConfig.createRoute(_props.creator, actorContext)
override def !(message: Any)(implicit sender: ActorRef = null) {
route(message) match {
case null super.!(message)(sender)
case ref: ActorRef ref.!(message)(sender)
case refs: Traversable[ActorRef] refs foreach (_.!(message)(sender))
}
}
}
trait RouterConfig extends Function0[Actor] {
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig
def createRoute(creator: () Actor, actorContext: ActorContext): Routing.Route
}
/**
* Routing configuration that indicates no routing.
* Oxymoron style.
*/
case object NoRouting extends RouterConfig {
def adaptFromDeploy(deploy: Option[Deploy]) = null
def createRoute(creator: () Actor, actorContext: ActorContext) = null
def apply(): Actor = null
}
/**
@ -96,40 +173,9 @@ object RoutedProps {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Router {
/**
* Initializes this Router with a given set of Connections. The Router can use this datastructure to ask for
* the current connections, signal that there were problems with one of the connections and see if there have
* been changes in the connections.
*
* This method is not threadsafe, and should only be called once
*
* JMM Guarantees:
* This method guarantees that all changes made in this method, are visible before one of the routing methods is called.
*/
def init(connectionManager: ConnectionManager)
/**
* Routes the message to one of the connections.
*
* @throws RoutingException if something goes wrong while routing the message
*/
def route(message: Any)(implicit sender: ActorRef)
/**
* Routes the message using a timeout to one of the connections and returns a Future to synchronize on the
* completion of the processing of the message.
*
* @throws RoutingExceptionif something goes wrong while routing the message.
*/
def route[T](message: Any, timeout: Timeout): Future[T]
// TODO (HE): implement failure detection
}
/**
* An {@link AkkaException} thrown when something goes wrong while routing a message
*/
class RoutingException(message: String) extends AkkaException(message)
/**
* A Helper class to create actor references that use routing.
*/
@ -144,247 +190,74 @@ object Routing {
case class Broadcast(message: Any) extends RoutingMessage
def createCustomRouter(implClass: String): Router = {
ReflectiveAccess.createInstance(
implClass,
Array[Class[_]](),
Array[AnyRef]()) match {
case Right(router) router.asInstanceOf[Router]
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
throw new ConfigurationException(
"Could not instantiate custom Router of [" +
implClass + "] due to: " +
cause, cause)
}
}
}
/**
* An Abstract convenience implementation for building an ActorReference that uses a Router.
*/
abstract private[akka] class AbstractRoutedActorRef(val system: ActorSystem, val props: RoutedProps) extends MinimalActorRef {
val router = props.routerFactory()
override def !(message: Any)(implicit sender: ActorRef = null): Unit = router.route(message)(sender)
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = router.route(message, timeout)
}
/**
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
* on (or more) of these actors.
*/
private[akka] class RoutedActorRef(system: ActorSystem, val routedProps: RoutedProps, val supervisor: InternalActorRef, name: String) extends AbstractRoutedActorRef(system, routedProps) {
val path = supervisor.path / name
@volatile
private var running: Boolean = true
override def isTerminated: Boolean = !running
override def stop() {
synchronized {
if (running) {
running = false
router.route(Routing.Broadcast(PoisonPill))(this)
supervisor.sendSystemMessage(akka.dispatch.ChildTerminated(this))
}
}
}
router.init(routedProps.connectionManager)
}
/**
* An Abstract Router implementation that already provides the basic infrastructure so that a concrete
* Router only needs to implement the next method.
*/
trait BasicRouter extends Router {
@volatile
protected var connectionManager: ConnectionManager = _
def init(connectionManager: ConnectionManager) = {
this.connectionManager = connectionManager
}
def route(message: Any)(implicit sender: ActorRef) = message match {
case Routing.Broadcast(message)
//it is a broadcast message, we are going to send to message to all connections.
connectionManager.connections.iterable foreach { connection
try {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connectionManager.remove(connection)
throw e
ReflectiveAccess.createInstance(implClass, Array[Class[_]](), Array[AnyRef]()) match {
case Right(router) router.asInstanceOf[Router]
case Left(exception)
val cause = exception match {
case i: InvocationTargetException i.getTargetException
case _ exception
}
}
case _
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
next match {
case Some(connection)
try {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connectionManager.remove(connection)
throw e
}
case None
throwNoConnectionsError
}
}
def route[T](message: Any, timeout: Timeout): Future[T] = message match {
case Routing.Broadcast(message)
throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.")
case _
//it no broadcast message, we are going to select an actor from the connections and send the message to him.
next match {
case Some(connection)
try {
connection.?(message, timeout).asInstanceOf[Future[T]] //FIXME this does not preserve the original sender, shouldn't it??
} catch {
case e: Exception
connectionManager.remove(connection)
throw e
}
case None
throwNoConnectionsError
}
}
protected def next: Option[ActorRef]
private def throwNoConnectionsError = throw new RoutingException("No replica connections for router")
}
/**
* A Router that uses broadcasts a message to all its connections.
*/
class BroadcastRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter with Serializable {
override def route(message: Any)(implicit sender: ActorRef) = {
connectionManager.connections.iterable foreach { connection
try {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connectionManager.remove(connection)
throw e
}
throw new ConfigurationException("Could not instantiate custom Router of [" +
implClass + "] due to: " + cause, cause)
}
}
//protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] =
override def route[T](message: Any, timeout: Timeout): Future[T] = {
import Future._
implicit val t = timeout
val futures = connectionManager.connections.iterable map { connection
connection.?(message, timeout).asInstanceOf[Future[T]]
}
Future.firstCompletedOf(futures)
}
protected def next: Option[ActorRef] = None
}
/**
* A DirectRouter a Router that only has a single connected actorRef and forwards all request to that actorRef.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
private val state = new AtomicReference[DirectRouterState]
lazy val next: Option[ActorRef] = {
val current = currentState
if (current.ref == null) None else Some(current.ref)
}
@tailrec
private def currentState: DirectRouterState = {
val current = state.get
if (current != null && connectionManager.version == current.version) {
//we are lucky since nothing has changed in the connections.
current
} else {
//there has been a change in the connections, or this is the first time this method is called. So we are going to do some updating.
val connections = connectionManager.connections
val connectionCount = connections.iterable.size
if (connectionCount > 1)
throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionCount))
val newState = new DirectRouterState(connections.iterable.head, connections.version)
if (state.compareAndSet(current, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
else //we failed to update the state, lets try again... better luck next time.
currentState // recur
}
}
private case class DirectRouterState(ref: ActorRef, version: Long)
}
/**
* A Router that randomly selects one of the target connections to send a message to.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
import java.security.SecureRandom
private val state = new AtomicReference[RandomRouterState]
private val random = new ThreadLocal[SecureRandom] {
override def initialValue = SecureRandom.getInstance("SHA1PRNG")
}
def next: Option[ActorRef] = currentState.array match {
case a if a.isEmpty None
case a Some(a(random.get.nextInt(a.length)))
}
@tailrec
private def currentState: RandomRouterState = {
val current = state.get
if (current != null && current.version == connectionManager.version) {
//we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
current
} else {
//there has been a change in connections, or it was the first try, so we need to update the internal state
val connections = connectionManager.connections
val newState = new RandomRouterState(connections.iterable.toIndexedSeq, connections.version)
if (state.compareAndSet(current, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
else //we failed to update the state, lets try again... better luck next time.
currentState
}
}
private case class RandomRouterState(array: IndexedSeq[ActorRef], version: Long)
type Route = (Any) AnyRef
}
/**
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
* <br>
* Please note that providing both 'nrOfInstances' and 'targets' does not make logical sense as this means
* that the round robin should both create new actors and use the 'targets' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
case class RoundRobinRouter(nrOfInstances: Int = 0, targets: Iterable[ActorRef] = Nil)
extends Router with RouterConfig {
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig = {
deploy match {
case Some(d) copy(nrOfInstances = d.nrOfInstances.factor)
case _ this
}
}
def apply(): Actor = new Actor {
def receive = {
case _
}
}
def createRoute(creator: () Actor, context: ActorContext): Routing.Route = {
val routees: Vector[ActorRef] = (nrOfInstances, targets) match {
case (0, Nil) throw new IllegalArgumentException("Insufficient information - missing configuration.")
case (x, Nil)
println("----> 0, Nil")
(1 to x).map(_ context.actorOf(context.props.copy(creator = creator, routerConfig = NoRouting)))(scala.collection.breakOut)
case (x, xs)
println("----> x, xs")
Vector.empty[ActorRef] ++ xs
}
val next = new AtomicInteger(0)
def getNext(): ActorRef = {
routees(next.getAndIncrement % routees.size)
}
{
case _: AutoReceivedMessage null //TODO: handle system specific messages
case Broadcast(msg) routees
case msg getNext()
}
}
/*
private val state = new AtomicReference[RoundRobinState]
def next: Option[ActorRef] = currentState.next
@ -402,7 +275,7 @@ class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Time
val connections = connectionManager.connections
val newState = new RoundRobinState(connections.iterable.toIndexedSeq[ActorRef], connections.version)
if (state.compareAndSet(current, newState))
//we are lucky since we just updated the state, so we can send it back as the state to use
//we are lucky since we just updated the state, so we can send it back as the state to use
newState
else //we failed to update the state, lets try again... better luck next time.
currentState
@ -424,8 +297,193 @@ class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Time
else oldIndex
}
}
*/
}
///**
// * An Abstract Router implementation that already provides the basic infrastructure so that a concrete
// * Router only needs to implement the next method.
// */
//trait BasicRouter extends Router {
//
// @volatile
// protected var connectionManager: ConnectionManager = _
//
// def init(connectionManager: ConnectionManager) = {
// this.connectionManager = connectionManager
// }
//
// def route(message: Any)(implicit sender: ActorRef) = message match {
// case Routing.Broadcast(message)
//
// //it is a broadcast message, we are going to send to message to all connections.
// connectionManager.connections.iterable foreach {
// connection
// try {
// connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
// } catch {
// case e: Exception
// connectionManager.remove(connection)
// throw e
// }
// }
// case _
// //it no broadcast message, we are going to select an actor from the connections and send the message to him.
// next match {
// case Some(connection)
// try {
// connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
// } catch {
// case e: Exception
// connectionManager.remove(connection)
// throw e
// }
// case None
// throwNoConnectionsError
// }
// }
//
// def route[T](message: Any, timeout: Timeout): Future[T] = message match {
// case Routing.Broadcast(message)
// throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.")
// case _
// //it no broadcast message, we are going to select an actor from the connections and send the message to him.
// next match {
// case Some(connection)
// try {
// connection.?(message, timeout).asInstanceOf[Future[T]] //FIXME this does not preserve the original sender, shouldn't it??
// } catch {
// case e: Exception
// connectionManager.remove(connection)
// throw e
// }
// case None
// throwNoConnectionsError
// }
// }
//
// protected def next: Option[ActorRef]
//
// private def throwNoConnectionsError = throw new RoutingException("No replica connections for router")
//}
//
///**
// * A Router that uses broadcasts a message to all its connections.
// */
//class BroadcastRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter with Serializable {
// override def route(message: Any)(implicit sender: ActorRef) = {
// connectionManager.connections.iterable foreach {
// connection
// try {
// connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
// } catch {
// case e: Exception
// connectionManager.remove(connection)
// throw e
// }
// }
// }
//
// //protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] =
// override def route[T](message: Any, timeout: Timeout): Future[T] = {
// import Future._
// implicit val t = timeout
// val futures = connectionManager.connections.iterable map {
// connection
// connection.?(message, timeout).asInstanceOf[Future[T]]
// }
// Future.firstCompletedOf(futures)
// }
//
// protected def next: Option[ActorRef] = None
//}
//
///**
// * A DirectRouter a Router that only has a single connected actorRef and forwards all request to that actorRef.
// *
// * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
// */
//class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
//
// private val state = new AtomicReference[DirectRouterState]
//
// lazy val next: Option[ActorRef] = {
// val current = currentState
// if (current.ref == null) None else Some(current.ref)
// }
//
// @tailrec
// private def currentState: DirectRouterState = {
// val current = state.get
//
// if (current != null && connectionManager.version == current.version) {
// //we are lucky since nothing has changed in the connections.
// current
// } else {
// //there has been a change in the connections, or this is the first time this method is called. So we are going to do some updating.
//
// val connections = connectionManager.connections
//
// val connectionCount = connections.iterable.size
// if (connectionCount > 1)
// throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionCount))
//
// val newState = new DirectRouterState(connections.iterable.head, connections.version)
// if (state.compareAndSet(current, newState))
// //we are lucky since we just updated the state, so we can send it back as the state to use
// newState
// else //we failed to update the state, lets try again... better luck next time.
// currentState // recur
// }
// }
//
// private case class DirectRouterState(ref: ActorRef, version: Long)
//
//}
//
///**
// * A Router that randomly selects one of the target connections to send a message to.
// *
// * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
// */
//class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
//
// import java.security.SecureRandom
//
// private val state = new AtomicReference[RandomRouterState]
//
// private val random = new ThreadLocal[SecureRandom] {
// override def initialValue = SecureRandom.getInstance("SHA1PRNG")
// }
//
// def next: Option[ActorRef] = currentState.array match {
// case a if a.isEmpty None
// case a Some(a(random.get.nextInt(a.length)))
// }
//
// @tailrec
// private def currentState: RandomRouterState = {
// val current = state.get
//
// if (current != null && current.version == connectionManager.version) {
// //we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
// current
// } else {
// //there has been a change in connections, or it was the first try, so we need to update the internal state
//
// val connections = connectionManager.connections
// val newState = new RandomRouterState(connections.iterable.toIndexedSeq, connections.version)
// if (state.compareAndSet(current, newState))
// //we are lucky since we just updated the state, so we can send it back as the state to use
// newState
// else //we failed to update the state, lets try again... better luck next time.
// currentState
// }
// }
//
// private case class RandomRouterState(array: IndexedSeq[ActorRef], version: Long)
//}
/**
* ScatterGatherRouter broadcasts the message to all connections and gathers results according to the
* specified strategy (specific router needs to implement `gather` method).
@ -436,6 +494,7 @@ class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Time
* FIXME: This also is the location where a failover is done in the future if an ActorRef fails and a different one needs to be selected.
* FIXME: this is also the location where message buffering should be done in case of failure.
*/
/*
trait ScatterGatherRouter extends BasicRouter with Serializable {
/**
@ -446,15 +505,16 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G]
private def scatterGather[S, G >: S](message: Any, timeout: Timeout): Future[G] = {
val responses = connectionManager.connections.iterable.flatMap { actor
try {
if (actor.isTerminated) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace
Some(actor.?(message, timeout).asInstanceOf[Future[S]])
} catch {
case e: Exception
connectionManager.remove(actor)
None
}
val responses = connectionManager.connections.iterable.flatMap {
actor
try {
if (actor.isTerminated) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace
Some(actor.?(message, timeout).asInstanceOf[Future[S]])
} catch {
case e: Exception
connectionManager.remove(actor)
None
}
}
if (responses.isEmpty)
@ -464,9 +524,10 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
override def route[T](message: Any, timeout: Timeout): Future[T] = message match {
case Routing.Broadcast(message) scatterGather(message, timeout)
case message super.route(message, timeout)
case message super.route(message, timeout)
}
}
*/
/**
* Simple router that broadcasts the message to all connections, and replies with the first response
@ -474,7 +535,8 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
* (wrapped into {@link Routing.Broadcast} and sent with "?" method). For the messages sent in a fire-forget
* mode, the router would behave as {@link RoundRobinRouter}
*/
/*
class ScatterGatherFirstCompletedRouter(implicit dispatcher: MessageDispatcher, timeout: Timeout) extends RoundRobinRouter with ScatterGatherRouter {
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] = Future.firstCompletedOf(results)
}
*/

View file

@ -13,7 +13,6 @@ Java API
dataflow
stm
transactors
remote-actors
fault-tolerance
dispatchers
routing

View file

@ -1,627 +0,0 @@
.. _remote-actors-java:
Remote Actors (Java)
====================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Akka supports starting interacting with UntypedActors and TypedActors on remote nodes using a very efficient and scalable NIO implementation built upon `JBoss Netty <http://jboss.org/netty>`_ and `Google Protocol Buffers <http://code.google.com/p/protobuf/>`_ .
The usage is completely transparent with local actors, both in regards to sending messages and error handling and propagation as well as supervision, linking and restarts. You can send references to other Actors as part of the message.
**WARNING**: For security reasons, do not run an Akka node with a Remote Actor port reachable by untrusted connections unless you have supplied a classloader that restricts access to the JVM.
Managing the Remote Service
---------------------------
Starting remote service in user code as a library
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Here is how to start up the server and specify the hostname and port programmatically:
.. code-block:: java
import static akka.actor.Actors.*;
remote().start("localhost", 2552);
// Specify the classloader to use to load the remote class (actor)
remote().start("localhost", 2552, classLoader);
Here is how to start up the server and specify the hostname and port in the akka.conf configuration file (see the section below for details):
.. code-block:: java
import static akka.actor.Actors.*;
remote();
// Specify the classloader to use to load the remote class (actor)
remote().start(classLoader);
Starting remote service as part of the stand-alone Kernel
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You simply need to make sure that the service is turned on in the external akka.conf configuration file.
.. code-block:: ruby
akka {
remote {
server {
service = on
hostname = "localhost"
port = 2552
connection-timeout = 1000 # in millis
}
}
}
Stopping the server
^^^^^^^^^^^^^^^^^^^
.. code-block:: java
import static akka.actor.Actors.*;
remote().shutdown();
Connecting and shutting down a client connection explicitly
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Normally you should not have to start and stop the client connection explicitly since that is handled by Akka on a demand basis. But if you for some reason want to do that then you can do it like this:
.. code-block:: java
import static akka.actor.Actors.*;
import java.net.InetSocketAddress;
remote().shutdownClientConnection(new InetSocketAddress("localhost", 6666)); //Returns true if successful, else false
remote().restartClientConnection(new InetSocketAddress("localhost", 6666)); //Returns true if successful, else false
Client message frame size configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You can define the max message frame size for the remote messages:
.. code-block:: ruby
akka {
remote {
client {
message-frame-size = 1048576
}
}
}
Client reconnect configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The Client automatically performs reconnection upon connection failure.
You can configure it like this:
.. code-block:: ruby
akka {
remote {
client {
reconnect-delay = 5 # in seconds (5 sec default)
read-timeout = 10 # in seconds (10 sec default)
reconnection-time-window = 600 # the maximum time window that a client should try to reconnect for
}
}
}
The client will automatically trying to reconnect to the server if the connection is broken. By default it has a reconnection window of 10 minutes (600 seconds).
If it has not been able to reconnect during this period of time then it is shut down and further attempts to use it will yield a 'RemoteClientException'. The 'RemoteClientException' contains the message as well as a reference to the address that is not yet connect in order for you to retrieve it an do an explicit connect if needed.
You can also register a listener that will listen for example the 'RemoteClientStopped' event, retrieve the address that got disconnected and reconnect explicitly.
See the section on client listener and events below for details.
Remote Client message buffering and send retry on failure
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The Remote Client implements message buffering on network failure. This feature has zero overhead (even turned on) in the successful scenario and a queue append operation in case of unsuccessful send. So it is really really fast.
The default behavior is that the remote client will maintain a transaction log of all messages that it has failed to send due to network problems (not other problems like serialization errors etc.). The client will try to resend these messages upon first successful reconnect and the message ordering is maintained. This means that the remote client will swallow all exceptions due to network failure and instead queue remote messages in the transaction log. The failures will however be reported through the remote client life-cycle events as well as the regular Akka event handler. You can turn this behavior on and off in the configuration file. It gives 'at-least-once' semantics, use a message id/counter for discarding potential duplicates (or use idempotent messages).
.. code-block:: ruby
akka {
remote {
client {
buffering {
retry-message-send-on-failure = on
capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property
}
}
}
}
If you choose a capacity higher than 0, then a bounded queue will be used and if the limit of the queue is reached then a 'RemoteClientMessageBufferException' will be thrown.
Running Remote Server in untrusted mode
---------------------------------------
You can run the remote server in untrusted mode. This means that the server will not allow any client-managed remote actors or any life-cycle messages and methods. This is useful if you want to let untrusted clients use server-managed actors in a safe way. This can optionally be combined with the secure cookie authentication mechanism described below as well as the SSL support for remote actor communication.
If the client is trying to perform one of these unsafe actions then a 'java.lang.SecurityException' is thrown on the server as well as transferred to the client and thrown there as well.
Here is how you turn it on:
.. code-block:: ruby
akka {
remote {
server {
untrusted-mode = on # the default is 'off'
}
}
}
The messages that it prevents are all that extends 'LifeCycleMessage':
* case class HotSwap(..)
* case object RevertHotSwap
* case class Restart(..)
* case class Exit(..)
* case class Link(..)
* case class Unlink(..)
* case class UnlinkAndStop(..)
* case object ReceiveTimeout
It also prevents the client from invoking any life-cycle and side-effecting methods, such as:
* stop
* etc.
Using secure cookie for remote client authentication
----------------------------------------------------
Akka is using a similar scheme for remote client node authentication as Erlang; using secure cookies. In order to use this authentication mechanism you have to do two things:
* Enable secure cookie authentication in the remote server
* Use the same secure cookie on all the trusted peer nodes
Enabling secure cookie authentication
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The first one is done by enabling the secure cookie authentication in the remote server section in the configuration file:
.. code-block:: ruby
akka {
remote {
server {
require-cookie = on
}
}
Now if you have try to connect to a server from a client then it will first try to authenticate the client by comparing the secure cookie for the two nodes. If they are the same then it allows the client to connect and use the server freely but if they are not the same then it will throw a 'java.lang.SecurityException' and not allow the client to connect.
Generating and using the secure cookie
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The secure cookie can be any string value but in order to ensure that it is secure it is best to randomly generate it. This can be done by invoking the 'generate_config_with_secure_cookie.sh' script which resides in the '$AKKA_HOME/scripts' folder. This script will generate and print out a complete 'akka.conf' configuration file with the generated secure cookie defined that you can either use as-is or cut and paste the 'secure-cookie' snippet. Here is an example of its generated output:
.. code-block:: ruby
# This config imports the Akka reference configuration.
include "akka-reference.conf"
# In this file you can override any option defined in the 'akka-reference.conf' file.
# Copy in all or parts of the 'akka-reference.conf' file and modify as you please.
akka {
remote {
secure-cookie = "000E02050F0300040C050C0D060A040306090B0C"
}
}
The simplest way to use it is to have it create your 'akka.conf' file like this:
.. code-block:: ruby
cd $AKKA_HOME
./scripts/generate_config_with_secure_cookie.sh > ./config/akka.conf
Now it is good to make sure that the configuration file is only accessible by the owner of the file. On Unix-style file system this can be done like this:
.. code-block:: ruby
chmod 400 ./config/akka.conf
Running this script requires having 'scala' on the path (and will take a couple of seconds to run since it is using Scala and has to boot up the JVM to run).
You can also generate the secure cookie by using the 'Crypt' object and its 'generateSecureCookie' method.
.. code-block:: scala
import akka.util.Crypt;
String secureCookie = Crypt.generateSecureCookie();
The secure cookie is a cryptographically secure randomly generated byte array turned into a SHA-1 hash.
Client-managed Remote UntypedActor
----------------------------------
DEPRECATED AS OF 1.1
The client creates the remote actor and "moves it" to the server.
When you define an actors as being remote it is instantiated as on the remote host and your local actor becomes a proxy, it works as a handle to the remote actor. The real execution is always happening on the remote node.
Here is an example:
.. code-block:: java
import akka.actor.UntypedActor;
import static akka.actor.Actors.*;
class MyActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
...
}
}
//How to make it client-managed:
remote().actorOf(MyActor.class,"192.68.23.769", 2552);
An UntypedActor can also start remote child Actors through one of the “spawn/link” methods. These will start, link and make the UntypedActor remote atomically.
.. code-block:: java
...
getContext().spawnRemote(MyActor.class, hostname, port, timeoutInMsForFutures);
getContext().spawnLinkRemote(MyActor.class, hostname, port, timeoutInMsForFutures);
...
Server-managed Remote UntypedActor
----------------------------------
Here it is the server that creates the remote actor and the client can ask for a handle to this actor.
Server side setup
^^^^^^^^^^^^^^^^^
The API for server managed remote actors is really simple. 2 methods only:
.. code-block:: java
import akka.actor.Actors;
import akka.actor.UntypedActor;
class MyActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
...
}
}
Actors.remote().start("localhost", 2552).register("hello-service", Actors.actorOf(HelloWorldActor.class));
Actors created like this are automatically started.
You can also register an actor by its UUID rather than ID or handle. This is done by prefixing the handle with the "uuid:" protocol.
.. code-block:: scala
server.register("uuid:" + actor.uuid, actor);
server.unregister("uuid:" + actor.uuid);
Session bound server side setup
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Session bound server managed remote actors work by creating and starting a new actor for every client that connects. Actors are stopped automatically when the client disconnects. The client side is the same as regular server managed remote actors. Use the function registerPerSession instead of register.
Session bound actors are useful if you need to keep state per session, e.g. username. They are also useful if you need to perform some cleanup when a client disconnects by overriding the postStop method as described `here <actors-scala#Stopping actors>`_
.. code-block:: java
import static akka.actor.Actors.*;
import akka.japi.Creator;
class HelloWorldActor extends Actor {
...
}
remote().start("localhost", 2552);
remote().registerPerSession("hello-service", new Creator<ActorRef>() {
public ActorRef create() {
return actorOf(HelloWorldActor.class);
}
});
Note that the second argument in registerPerSession is a Creator, it means that the create method will create a new ActorRef each invocation.
It will be called to create an actor every time a session is established.
Client side usage
^^^^^^^^^^^^^^^^^
.. code-block:: java
import static akka.actor.Actors.*;
ActorRef actor = remote().actorFor("hello-service", "localhost", 2552);
Object result = actor.ask("Hello").get();
There are many variations on the 'remote()#actorFor' method. Here are some of them:
.. code-block:: java
... = remote().actorFor(className, hostname, port);
... = remote().actorFor(className, timeout, hostname, port);
... = remote().actorFor(uuid, className, hostname, port);
... = remote().actorFor(uuid, className, timeout, hostname, port);
... // etc
All of these also have variations where you can pass in an explicit 'ClassLoader' which can be used when deserializing messages sent from the remote actor.
Automatic remote 'sender' reference management
----------------------------------------------
The sender of a remote message will be reachable with a reply through the remote server on the node that the actor is residing, automatically.
Please note that firewalled clients won't work right now. [2011-01-05]
Identifying remote actors
-------------------------
The 'id' field in the 'Actor' class is of importance since it is used as identifier for the remote actor. If you want to create a brand new actor every time you instantiate a remote actor then you have to set the 'id' field to a unique 'String' for each instance. If you want to reuse the same remote actor instance for each new remote actor (of the same class) you create then you don't have to do anything since the 'id' field by default is equal to the name of the actor class.
Here is an example of overriding the 'id' field:
.. code-block:: java
import akka.actor.UntypedActor;
import com.eaio.uuid.UUID;
class MyActor extends UntypedActor {
public MyActor() {
getContext().setId(new UUID().toString());
}
public void onReceive(Object message) throws Exception {
// ...
}
}
Client-managed Remote Typed Actors
----------------------------------
DEPRECATED AS OF 1.1
Remote Typed Actors are created through the 'TypedActor.newRemoteInstance' factory method.
.. code-block:: java
MyPOJO remoteActor = (MyPOJO) TypedActor.newRemoteInstance(MyPOJO.class, MyPOJOImpl.class, "localhost", 2552);
And if you want to specify the timeout:
.. code-block:: java
MyPOJO remoteActor = (MyPOJO)TypedActor.newRemoteInstance(MyPOJO.class, MyPOJOImpl.class, timeout, "localhost", 2552);
You can also define the Typed Actor to be a client-managed-remote service by adding the RemoteAddress configuration element in the declarative supervisor configuration:
.. code-block:: java
new Component(
Foo.class,
FooImpl.class,
new LifeCycle(new Permanent(), 1000),
1000,
new RemoteAddress("localhost", 2552))
Server-managed Remote Typed Actors
----------------------------------
WARNING: Remote TypedActors do not work with overloaded methods on your TypedActor, refrain from using overloading.
Server side setup
^^^^^^^^^^^^^^^^^
The API for server managed remote typed actors is nearly the same as for untyped actor:
.. code-block:: java
import static akka.actor.Actors.*;
remote().start("localhost", 2552);
RegistrationService typedActor = TypedActor.newInstance(RegistrationService.class, RegistrationServiceImpl.class, 2000);
remote().registerTypedActor("user-service", typedActor);
Client side usage
^^^^^^^^^^^^^^^^^
.. code-block:: java
import static akka.actor.Actors.*;
RegistrationService actor = remote().typedActorFor(RegistrationService.class, "user-service", 5000L, "localhost", 2552);
actor.registerUser(...);
There are variations on the 'remote()#typedActorFor' method. Here are some of them:
.. code-block:: java
... = remote().typedActorFor(interfaceClazz, serviceIdOrClassName, hostname, port);
... = remote().typedActorFor(interfaceClazz, serviceIdOrClassName, timeout, hostname, port);
... = remote().typedActorFor(interfaceClazz, serviceIdOrClassName, timeout, hostname, port, classLoader);
Data Compression Configuration
------------------------------
Akka uses compression to minimize the size of the data sent over the wire. Currently it only supports 'zlib' compression but more will come later.
You can configure it like this:
.. code-block:: ruby
akka {
remote {
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
...
}
}
Code provisioning
-----------------
Akka does currently not support automatic code provisioning but requires you to have the remote actor class files available on both the "client" the "server" nodes.
This is something that will be addressed soon. Until then, sorry for the inconvenience.
Subscribe to Remote Client events
---------------------------------
Akka has a subscription API for remote client events. You can register an Actor as a listener and this actor will have to be able to process these events:
.. code-block:: java
class RemoteClientError { Throwable cause; RemoteClientModule client; InetSocketAddress remoteAddress; }
class RemoteClientDisconnected { RemoteClientModule client; InetSocketAddress remoteAddress; }
class RemoteClientConnected { RemoteClientModule client; InetSocketAddress remoteAddress; }
class RemoteClientStarted { RemoteClientModule client; InetSocketAddress remoteAddress; }
class RemoteClientShutdown { RemoteClientModule client; InetSocketAddress remoteAddress; }
class RemoteClientWriteFailed { Object message; Throwable cause; RemoteClientModule client; InetSocketAddress remoteAddress; }
So a simple listener actor can look like this:
.. code-block:: java
import akka.actor.UntypedActor;
import akka.cluster.*;
class Listener extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (message instanceof RemoteClientError) {
RemoteClientError event = (RemoteClientError) message;
Throwable cause = event.getCause();
// ...
} else if (message instanceof RemoteClientConnected) {
RemoteClientConnected event = (RemoteClientConnected) message;
// ...
} else if (message instanceof RemoteClientDisconnected) {
RemoteClientDisconnected event = (RemoteClientDisconnected) message;
// ...
} else if (message instanceof RemoteClientStarted) {
RemoteClientStarted event = (RemoteClientStarted) message;
// ...
} else if (message instanceof RemoteClientShutdown) {
RemoteClientShutdown event = (RemoteClientShutdown) message;
// ...
} else if (message instanceof RemoteClientWriteFailed) {
RemoteClientWriteFailed event = (RemoteClientWriteFailed) message;
// ...
}
}
}
Registration and de-registration can be done like this:
.. code-block:: java
ActorRef listener = Actors.actorOf(Listener.class);
...
Actors.remote().addListener(listener);
...
Actors.remote().removeListener(listener);
Subscribe to Remote Server events
---------------------------------
Akka has a subscription API for the server events. You can register an Actor as a listener and this actor will have to be able to process these events:
.. code-block:: java
class RemoteServerStarted { RemoteServerModule server; }
class RemoteServerShutdown { RemoteServerModule server; }
class RemoteServerError { Throwable cause; RemoteServerModule server; }
class RemoteServerClientConnected { RemoteServerModule server; Option<InetSocketAddress> clientAddress; }
class RemoteServerClientDisconnected { RemoteServerModule server; Option<InetSocketAddress> clientAddress; }
class RemoteServerClientClosed { RemoteServerModule server; Option<InetSocketAddress> clientAddress; }
class RemoteServerWriteFailed { Object request; Throwable cause; RemoteServerModule server; Option<InetSocketAddress> clientAddress; }
So a simple listener actor can look like this:
.. code-block:: java
import akka.actor.UntypedActor;
import akka.cluster.*;
class Listener extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (message instanceof RemoteClientError) {
RemoteClientError event = (RemoteClientError) message;
Throwable cause = event.getCause();
// ...
} else if (message instanceof RemoteClientConnected) {
RemoteClientConnected event = (RemoteClientConnected) message;
// ...
} else if (message instanceof RemoteClientDisconnected) {
RemoteClientDisconnected event = (RemoteClientDisconnected) message;
// ...
} else if (message instanceof RemoteClientStarted) {
RemoteClientStarted event = (RemoteClientStarted) message;
// ...
} else if (message instanceof RemoteClientShutdown) {
RemoteClientShutdown event = (RemoteClientShutdown) message;
// ...
} else if (message instanceof RemoteClientWriteFailed) {
RemoteClientWriteFailed event = (RemoteClientWriteFailed) message;
// ...
}
}
}
Registration and de-registration can be done like this:
.. code-block:: java
import static akka.actor.Actors.*;
ActorRef listener = actorOf(Listener.class);
...
remote().addListener(listener);
...
remote().removeListener(listener);
Message Serialization
---------------------
All messages that are sent to remote actors needs to be serialized to binary format to be able to travel over the wire to the remote node. This is done by letting your messages extend one of the traits in the 'akka.serialization.Serializable' object. If the messages don't implement any specific serialization trait then the runtime will try to use standard Java serialization.
Protobuf
^^^^^^^^
Protobuf message specification needs to be compiled with 'protoc' compiler.
::
message ProtobufPOJO {
required uint64 id = 1;
required string name = 2;
required bool status = 3;
}
Using the generated message builder to send the message to a remote actor:
.. code-block:: java
actor.tell(ProtobufPOJO.newBuilder()
.setId(11)
.setStatus(true)
.setName("Coltrane")
.build());

View file

@ -487,6 +487,10 @@ Both of these takes a ``ActorRef => PartialFunction[Any, Unit]`` that implements
the new message handler. The hotswapped code is kept in a Stack which can be
pushed and popped.
.. warning::
Please note that the actor will revert to its original behavior when restarted by its Supervisor.
To hotswap the Actor body using the ``HotSwap`` message:
.. code-block:: scala

View file

@ -14,7 +14,6 @@ Scala API
agents
stm
transactors
remote-actors
fault-tolerance
dispatchers
routing

View file

@ -1,696 +0,0 @@
.. _remote-actors-scala:
Remote Actors (Scala)
=====================
.. sidebar:: Contents
.. contents:: :local:
Module stability: **SOLID**
Akka supports starting and interacting with Actors and Typed Actors on remote nodes using a very efficient and scalable NIO implementation built upon `JBoss Netty <http://jboss.org/netty>`_ and `Google Protocol Buffers <http://code.google.com/p/protobuf/>`_ .
The usage is completely transparent with local actors, both in regards to sending messages and error handling and propagation as well as supervision, linking and restarts. You can send references to other Actors as part of the message.
You can find a runnable sample `here <http://github.com/jboner/akka/tree/master/akka-samples/akka-sample-remote/>`__.
Starting up the remote service
------------------------------
Starting remote service in user code as a library
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Here is how to start up the RemoteNode and specify the hostname and port programmatically:
.. code-block:: scala
import akka.actor.Actor._
remote.start("localhost", 2552)
// Specify the classloader to use to load the remote class (actor)
remote.start("localhost", 2552, classLoader)
Here is how to start up the RemoteNode and specify the hostname and port in the 'akka.conf' configuration file (see the section below for details):
.. code-block:: scala
import akka.actor.Actor._
remote
// Specify the classloader to use to load the remote class (actor)
remote.start(classLoader)
Starting remote service as part of the stand-alone Kernel
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You simply need to make sure that the service is turned on in the external 'akka.conf' configuration file.
.. code-block:: ruby
akka {
remote {
server {
service = on
hostname = "localhost"
port = 2552
connection-timeout = 1000 # in millis
}
}
}
Stopping a RemoteNode or RemoteServer
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
If you invoke 'shutdown' on the server then the connection will be closed.
.. code-block:: scala
import akka.actor.Actor._
remote.shutdown()
Connecting and shutting down a client explicitly
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Normally you should not have to start and stop the client connection explicitly since that is handled by Akka on a demand basis. But if you for some reason want to do that then you can do it like this:
.. code-block:: scala
import akka.actor.Actor._
import java.net.InetSocketAddress
remote.shutdownClientConnection(new InetSocketAddress("localhost", 6666)) //Returns true if successful, false otherwise
remote.restartClientConnection(new InetSocketAddress("localhost", 6666)) //Returns true if successful, false otherwise
Remote Client message frame size configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
You can define the max message frame size for the remote messages:
.. code-block:: ruby
akka {
remote {
client {
message-frame-size = 1048576
}
}
}
Remote Client reconnect configuration
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The Remote Client automatically performs reconnection upon connection failure.
You can configure it like this:
.. code-block:: ruby
akka {
remote {
client {
reconnect-delay = 5 # in seconds (5 sec default)
read-timeout = 10 # in seconds (10 sec default)
reconnection-time-window = 600 # the maximum time window that a client should try to reconnect for
}
}
}
The RemoteClient is automatically trying to reconnect to the server if the connection is broken. By default it has a reconnection window of 10 minutes (600 seconds).
If it has not been able to reconnect during this period of time then it is shut down and further attempts to use it will yield a 'RemoteClientException'. The 'RemoteClientException' contains the message as well as a reference to the RemoteClient that is not yet connect in order for you to retrieve it an do an explicit connect if needed.
You can also register a listener that will listen for example the 'RemoteClientStopped' event, retrieve the 'RemoteClient' from it and reconnect explicitly.
See the section on RemoteClient listener and events below for details.
Remote Client message buffering and send retry on failure
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The Remote Client implements message buffering on network failure. This feature has zero overhead (even turned on) in the successful scenario and a queue append operation in case of unsuccessful send. So it is really really fast.
The default behavior is that the remote client will maintain a transaction log of all messages that it has failed to send due to network problems (not other problems like serialization errors etc.). The client will try to resend these messages upon first successful reconnect and the message ordering is maintained. This means that the remote client will swallow all exceptions due to network failure and instead queue remote messages in the transaction log. The failures will however be reported through the remote client life-cycle events as well as the regular Akka event handler. You can turn this behavior on and off in the configuration file. It gives 'at-least-once' semantics, use a message id/counter for discarding potential duplicates (or use idempotent messages).
.. code-block:: ruby
akka {
remote {
client {
buffering {
retry-message-send-on-failure = on
capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property
}
}
}
}
If you choose a capacity higher than 0, then a bounded queue will be used and if the limit of the queue is reached then a 'RemoteClientMessageBufferException' will be thrown.
Running Remote Server in untrusted mode
---------------------------------------
You can run the remote server in untrusted mode. This means that the server will not allow any client-managed remote actors or any life-cycle messages and methods. This is useful if you want to let untrusted clients use server-managed actors in a safe way. This can optionally be combined with the secure cookie authentication mechanism described below as well as the SSL support for remote actor communication.
If the client is trying to perform one of these unsafe actions then a 'java.lang.SecurityException' is thrown on the server as well as transferred to the client and thrown there as well.
Here is how you turn it on:
.. code-block:: ruby
akka {
remote {
server {
untrusted-mode = on # the default is 'off'
}
}
}
The messages that it prevents are all that extends 'LifeCycleMessage':
* class HotSwap(..)
* class RevertHotSwap..)
* class Restart(..)
* class Exit(..)
* class Link(..)
* class Unlink(..)
* class UnlinkAndStop(..)
* class ReceiveTimeout..)
It also prevents the client from invoking any life-cycle and side-effecting methods, such as:
* stop
* etc.
Using secure cookie for remote client authentication
----------------------------------------------------
Akka is using a similar scheme for remote client node authentication as Erlang; using secure cookies. In order to use this authentication mechanism you have to do two things:
* Enable secure cookie authentication in the remote server
* Use the same secure cookie on all the trusted peer nodes
Enabling secure cookie authentication
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The first one is done by enabling the secure cookie authentication in the remote server section in the configuration file:
.. code-block:: ruby
akka {
remote {
server {
require-cookie = on
}
}
}
Now if you have try to connect to a server with a client then it will first try to authenticate the client by comparing the secure cookie for the two nodes. If they are the same then it allows the client to connect and use the server freely but if they are not the same then it will throw a 'java.lang.SecurityException' and not allow the client to connect.
Generating and using the secure cookie
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The secure cookie can be any string value but in order to ensure that it is secure it is best to randomly generate it. This can be done by invoking the 'generate_config_with_secure_cookie.sh' script which resides in the '$AKKA_HOME/scripts' folder. This script will generate and print out a complete 'akka.conf' configuration file with the generated secure cookie defined that you can either use as-is or cut and paste the 'secure-cookie' snippet. Here is an example of its generated output:
.. code-block:: ruby
# This config imports the Akka reference configuration.
include "akka-reference.conf"
# In this file you can override any option defined in the 'akka-reference.conf' file.
# Copy in all or parts of the 'akka-reference.conf' file and modify as you please.
akka {
remote {
secure-cookie = "000E02050F0300040C050C0D060A040306090B0C"
}
}
The simplest way to use it is to have it create your 'akka.conf' file like this:
.. code-block:: ruby
cd $AKKA_HOME
./scripts/generate_config_with_secure_cookie.sh > ./config/akka.conf
Now it is good to make sure that the configuration file is only accessible by the owner of the file. On Unix-style file system this can be done like this:
.. code-block:: ruby
chmod 400 ./config/akka.conf
Running this script requires having 'scala' on the path (and will take a couple of seconds to run since it is using Scala and has to boot up the JVM to run).
You can also generate the secure cookie by using the 'Crypt' object and its 'generateSecureCookie' method.
.. code-block:: scala
import akka.util.Crypt
val secureCookie = Crypt.generateSecureCookie
The secure cookie is a cryptographically secure randomly generated byte array turned into a SHA-1 hash.
Client-managed Remote Actors
----------------------------
DEPRECATED AS OF 1.1
The client creates the remote actor and "moves it" to the server.
When you define an actor as being remote it is instantiated as on the remote host and your local actor becomes a proxy, it works as a handle to the remote actor. The real execution is always happening on the remote node.
Actors can be made remote by calling remote.actorOf[MyActor](host, port)
Here is an example:
.. code-block:: scala
import akka.actor.Actor
class MyActor extends Actor {
def receive = {
case "hello" => self.reply("world")
}
}
val remoteActor = Actor.remote.actorOf[MyActor]("192.68.23.769", 2552)
An Actor can also start remote child Actors through one of the 'spawn/link' methods. These will start, link and make the Actor remote atomically.
.. code-block:: scala
...
self.spawnRemote[MyActor](hostname, port, timeout)
self.spawnLinkRemote[MyActor](hostname, port, timeout)
...
Server-managed Remote Actors
----------------------------
Here it is the server that creates the remote actor and the client can ask for a handle to this actor.
Server side setup
^^^^^^^^^^^^^^^^^
The API for server managed remote actors is really simple. 2 methods only:
.. code-block:: scala
class HelloWorldActor extends Actor {
def receive = {
case "Hello" => self.reply("World")
}
}
remote.start("localhost", 2552) //Start the server
remote.register("hello-service", actorOf[HelloWorldActor]) //Register the actor with the specified service id
Actors created like this are automatically started.
You can also register an actor by its UUID rather than ID or handle. This is done by prefixing the handle with the "uuid:" protocol.
.. code-block:: scala
remote.register("uuid:" + actor.uuid, actor)
remote.unregister("uuid:" + actor.uuid)
Session bound server side setup
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Session bound server managed remote actors work by creating and starting a new actor for every client that connects. Actors are stopped automatically when the client disconnects. The client side is the same as regular server managed remote actors. Use the function registerPerSession instead of register.
Session bound actors are useful if you need to keep state per session, e.g. username.
They are also useful if you need to perform some cleanup when a client disconnects by overriding the postStop method as described `here <actors-scala#Stopping actors>`__
.. code-block:: scala
class HelloWorldActor extends Actor {
def receive = {
case "Hello" => self.reply("World")
}
}
remote.start("localhost", 2552)
remote.registerPerSession("hello-service", actorOf[HelloWorldActor])
Note that the second argument in registerPerSession is an implicit function. It will be called to create an actor every time a session is established.
Client side usage
^^^^^^^^^^^^^^^^^
.. code-block:: scala
val actor = remote.actorFor("hello-service", "localhost", 2552)
val result = (actor ? "Hello").as[String]
There are many variations on the 'remote#actorFor' method. Here are some of them:
.. code-block:: scala
... = remote.actorFor(className, hostname, port)
... = remote.actorFor(className, timeout, hostname, port)
... = remote.actorFor(uuid, className, hostname, port)
... = remote.actorFor(uuid, className, timeout, hostname, port)
... // etc
All of these also have variations where you can pass in an explicit 'ClassLoader' which can be used when deserializing messages sent from the remote actor.
Running sample
^^^^^^^^^^^^^^
Here is a complete running sample (also available `here <http://github.com/jboner/akka/blob/master/akka-core/src/test/scala/ServerInitiatedRemoteActorSample.scala>`_):
Paste in the code below into two sbt concole shells. Then run:
- ServerInitiatedRemoteActorServer.run() in one shell
- ServerInitiatedRemoteActorClient.run() in the other shell
.. code-block:: scala
import akka.actor.Actor
import Actor._
import akka.event.EventHandler
class HelloWorldActor extends Actor {
def receive = {
case "Hello" => self.reply("World")
}
}
object ServerInitiatedRemoteActorServer {
def run() {
remote.start("localhost", 2552)
remote.register("hello-service", actorOf[HelloWorldActor])
}
def main(args: Array[String]) { run() }
}
object ServerInitiatedRemoteActorClient {
def run() {
val actor = remote.actorFor("hello-service", "localhost", 2552)
val result = (actor ? "Hello").as[AnyRef]
EventHandler.info("Result from Remote Actor: %s", result)
}
def main(args: Array[String]) { run() }
}
Automatic remote 'sender' reference management
----------------------------------------------
The sender of a remote message will be reachable with a reply through the remote server on the node that the actor is residing, automatically.
Please note that firewalled clients won't work right now. [2011-01-05]
Identifying remote actors
-------------------------
The 'id' field in the 'Actor' class is of importance since it is used as identifier for the remote actor. If you want to create a brand new actor every time you instantiate a remote actor then you have to set the 'id' field to a unique 'String' for each instance. If you want to reuse the same remote actor instance for each new remote actor (of the same class) you create then you don't have to do anything since the 'id' field by default is equal to the name of the actor class.
Here is an example of overriding the 'id' field:
.. code-block:: scala
import akka.actor.newUuid
class MyActor extends Actor {
self.id = newUuid.toString
def receive = {
case "hello" => self.reply("world")
}
}
val actor = remote.actorOf[MyActor]("192.68.23.769", 2552)
Client-managed Remote Typed Actors
----------------------------------
DEPRECATED AS OF 1.1
You can define the Typed Actor to be a remote service by adding the 'RemoteAddress' configuration element in the declarative supervisor configuration:
.. code-block:: java
new Component(
Foo.class,
new LifeCycle(new Permanent(), 1000),
1000,
new RemoteAddress("localhost", 2552))
You can also define an Typed Actor to be remote programmatically when creating it explicitly:
.. code-block:: java
TypedActorFactory factory = new TypedActorFactory();
POJO pojo = (POJO) factory.newRemoteInstance(POJO.class, 1000, "localhost", 2552)
... // use pojo as usual
Server-managed Remote Typed Actors
----------------------------------
WARNING: Remote TypedActors do not work with overloaded methods on your TypedActor, refrain from using overloading.
Server side setup
^^^^^^^^^^^^^^^^^
The API for server managed remote typed actors is nearly the same as for untyped actor
.. code-block:: scala
class RegistrationServiceImpl extends TypedActor with RegistrationService {
def registerUser(user: User) {
... // register user
}
}
remote.start("localhost", 2552)
val typedActor = TypedActor.newInstance(classOf[RegistrationService], classOf[RegistrationServiceImpl], 2000)
remote.registerTypedActor("user-service", typedActor)
Actors created like this are automatically started.
Session bound server side setup
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Session bound server managed remote actors work by creating and starting a new actor for every client that connects. Actors are stopped automatically when the client disconnects. The client side is the same as regular server managed remote actors. Use the function registerTypedPerSessionActor instead of registerTypedActor.
Session bound actors are useful if you need to keep state per session, e.g. username.
They are also useful if you need to perform some cleanup when a client disconnects.
.. code-block:: scala
class RegistrationServiceImpl extends TypedActor with RegistrationService {
def registerUser(user: User) {
... // register user
}
}
remote.start("localhost", 2552)
remote.registerTypedPerSessionActor("user-service",
TypedActor.newInstance(classOf[RegistrationService],
classOf[RegistrationServiceImpl], 2000))
Note that the second argument in registerTypedPerSessionActor is an implicit function. It will be called to create an actor every time a session is established.
Client side usage
^^^^^^^^^^^^^^^^^
.. code-block:: scala
val actor = remote.typedActorFor(classOf[RegistrationService], "user-service", 5000L, "localhost", 2552)
actor.registerUser(…)
There are variations on the 'remote#typedActorFor' method. Here are some of them:
.. code-block:: scala
... = remote.typedActorFor(interfaceClazz, serviceIdOrClassName, hostname, port)
... = remote.typedActorFor(interfaceClazz, serviceIdOrClassName, timeout, hostname, port)
... = remote.typedActorFor(interfaceClazz, serviceIdOrClassName, timeout, hostname, port, classLoader)
Data Compression Configuration
------------------------------
Akka uses compression to minimize the size of the data sent over the wire. Currently it only supports 'zlib' compression but more will come later.
You can configure it like this:
.. code-block:: ruby
akka {
remote {
compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
}
}
Code provisioning
-----------------
Akka does currently not support automatic code provisioning but requires you to have the remote actor class files available on both the "client" the "server" nodes.
This is something that will be addressed soon. Until then, sorry for the inconvenience.
Subscribe to Remote Client events
---------------------------------
Akka has a subscription API for the client event. You can register an Actor as a listener and this actor will have to be able to process these events:
.. code-block:: scala
sealed trait RemoteClientLifeCycleEvent
case class RemoteClientError(
@BeanProperty cause: Throwable,
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
case class RemoteClientDisconnected(
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
case class RemoteClientConnected(
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
case class RemoteClientStarted(
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
case class RemoteClientShutdown(
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
case class RemoteClientWriteFailed(
@BeanProperty request: AnyRef,
@BeanProperty cause: Throwable,
@BeanProperty client: RemoteClientModule,
@BeanProperty remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
So a simple listener actor can look like this:
.. code-block:: scala
import akka.actor.Actor
import akka.actor.Actor._
import akka.cluster._
val listener = actorOf(new Actor {
def receive = {
case RemoteClientError(cause, client, address) => //... act upon error
case RemoteClientDisconnected(client, address) => //... act upon disconnection
case RemoteClientConnected(client, address) => //... act upon connection
case RemoteClientStarted(client, address) => //... act upon client shutdown
case RemoteClientShutdown(client, address) => //... act upon client shutdown
case RemoteClientWriteFailed(request, cause, client, address) => //... act upon write failure
case _ => // ignore other
}
})
Registration and de-registration can be done like this:
.. code-block:: scala
remote.addListener(listener)
...
remote.removeListener(listener)
Subscribe to Remote Server events
---------------------------------
Akka has a subscription API for the 'RemoteServer'. You can register an Actor as a listener and this actor will have to be able to process these events:
.. code-block:: scala
sealed trait RemoteServerLifeCycleEvent
case class RemoteServerStarted(
@BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
case class RemoteServerShutdown(
@BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
case class RemoteServerError(
@BeanProperty val cause: Throwable,
@BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
case class RemoteServerClientConnected(
@BeanProperty val server: RemoteServerModule,
@BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
case class RemoteServerClientDisconnected(
@BeanProperty val server: RemoteServerModule,
@BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
case class RemoteServerClientClosed(
@BeanProperty val server: RemoteServerModule,
@BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
case class RemoteServerWriteFailed(
@BeanProperty request: AnyRef,
@BeanProperty cause: Throwable,
@BeanProperty server: RemoteServerModule,
@BeanProperty clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
So a simple listener actor can look like this:
.. code-block:: scala
import akka.actor.Actor
import akka.actor.Actor._
import akka.cluster._
val listener = actorOf(new Actor {
def receive = {
case RemoteServerStarted(server) => //... act upon server start
case RemoteServerShutdown(server) => //... act upon server shutdown
case RemoteServerError(cause, server) => //... act upon server error
case RemoteServerClientConnected(server, clientAddress) => //... act upon client connection
case RemoteServerClientDisconnected(server, clientAddress) => //... act upon client disconnection
case RemoteServerClientClosed(server, clientAddress) => //... act upon client connection close
case RemoteServerWriteFailed(request, cause, server, clientAddress) => //... act upon server write failure
}
})
Registration and de-registration can be done like this:
.. code-block:: scala
remote.addListener(listener)
...
remote.removeListener(listener)
Message Serialization
---------------------
All messages that are sent to remote actors needs to be serialized to binary format to be able to travel over the wire to the remote node. This is done by letting your messages extend one of the traits in the 'akka.serialization.Serializable' object. If the messages don't implement any specific serialization trait then the runtime will try to use standard Java serialization.
Scala JSON
^^^^^^^^^^
.. code-block:: scala
case class MyMessage(id: String, value: Tuple2[String, Int]) extends Serializable.ScalaJSON[MyMessage]
Protobuf
^^^^^^^^
Protobuf message specification needs to be compiled with 'protoc' compiler.
::
message ProtobufPOJO {
required uint64 id = 1;
required string name = 2;
required bool status = 3;
}
Using the generated message builder to send the message to a remote actor:
.. code-block:: scala
val resultFuture = actor ? ProtobufPOJO.newBuilder
.setId(11)
.setStatus(true)
.setName("Coltrane")
.build

View file

@ -463,19 +463,22 @@ internally scaled by a factor taken from ``akka.conf``,
Resolving Conflicts with Implicit ActorRef
------------------------------------------
The :class:`TestKit` trait contains an implicit value of type :class:`ActorRef`
to enable the magic reply handling. This value is named ``self`` so that e.g.
anonymous actors may be declared within a test class without having to care
about the ambiguous implicit issues which would otherwise arise. If you find
yourself in a situation where the implicit you need comes from a different
trait than :class:`TestKit` and is not named ``self``, then use
:class:`TestKitLight`, which differs only in not having any implicit members.
You would then need to make an implicit available in locally confined scopes
which need it, e.g. different test cases. If this cannot be done, you will need
to resort to explicitly specifying the sender reference::
If you want the sender of messages inside your TestKit-based tests to be the `testActor```
simply mix in `ÌmplicitSender`` into your test.
val actor = actorOf[MyWorker]
actor.!(msg)(testActor)
.. code-block:: scala
class SomeSpec extends WordSpec with MustMatchers with TestKit with ImplicitSender {
"A Worker" must {
"send timely replies" in {
val worker = actorOf(...)
within (500 millis) {
worker ! "some work" // testActor is the "sender" for this message
expectMsg("some result")
}
}
}
}
Using Multiple Probe Actors
---------------------------

View file

@ -155,10 +155,10 @@ class RemoteActorRefProvider(
* Copied from LocalActorRefProvider...
*/
// FIXME: implement supervision, ticket #1408
def actorOf(system: ActorSystem, props: RoutedProps, supervisor: InternalActorRef, name: String): InternalActorRef = {
if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
new RoutedActorRef(system, props, supervisor, name)
}
// def actorOf(system: ActorSystem, props: RoutedProps, supervisor: InternalActorRef, name: String): InternalActorRef = {
// if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + name + "] has zero connections configured; can't create a router")
// new RoutedActorRef(system, props, supervisor, name)
// }
def actorFor(path: ActorPath): InternalActorRef = path.root match {
case `rootPath` actorFor(rootGuardian, path.elements)

View file

@ -64,7 +64,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec with De
barrier("start")
val actor = system.actorOf[SomeActor]("service-hello")
actor.isInstanceOf[RoutedActorRef] must be(true)
actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)
//actor.asInstanceOf[RoutedActorRef].router.isInstanceOf[ScatterGatherFirstCompletedRouter] must be(true)
val connectionCount = NrOfNodes - 1
val iterationCount = 10

View file

@ -111,7 +111,7 @@ akka {
expectMsg("postStop")
}
"look-up actors across node boundaries" in {
"look-up actors across node boundaries" ignore {
val l = system.actorOf(Props(new Actor {
def receive = {
case (p: Props, n: String) sender ! context.actorOf(p, n)

View file

@ -107,7 +107,9 @@ public class Pi {
this.latch = latch;
Creator<Router> routerCreator = new Creator<Router>() {
public Router create() {
return new RoundRobinRouter(getContext().dispatcher(), new akka.actor.Timeout(-1));
// TODO (HE) : implement
//return new RoundRobinRouter(getContext().dispatcher(), new akka.actor.Timeout(-1));
return null;
}
};
LinkedList<ActorRef> actors = new LinkedList<ActorRef>() {
@ -115,9 +117,11 @@ public class Pi {
for (int i = 0; i < nrOfWorkers; i++) add(getContext().actorOf(Worker.class));
}
};
// FIXME routers are intended to be used like this
RoutedProps props = new RoutedProps(routerCreator, new LocalConnectionManager(actors), new akka.actor.Timeout(-1), true);
router = new RoutedActorRef(getContext().system(), props, (InternalActorRef) getSelf(), "pi");
// TODO (HE): implement
//RoutedProps props = new RoutedProps(routerCreator, new LocalConnectionManager(actors), new akka.actor.Timeout(-1), true);
//router = new RoutedActorRef(getContext().system(), props, (InternalActorRef) getSelf(), "pi");
}
// message handler

View file

@ -0,0 +1,6 @@
akka.actor.deployment {
/user/pi2 {
router = round-robin
nr-of-instances = 4
}
}

View file

@ -4,14 +4,14 @@
package akka.tutorial.first.scala
import java.util.concurrent.CountDownLatch
import akka.routing.{ RoutedActorRef, LocalConnectionManager, RoundRobinRouter, RoutedProps }
import akka.actor.{ ActorSystemImpl, Actor, ActorSystem }
import akka.actor.InternalActorRef
import akka.actor._
import akka.routing._
import com.typesafe.config.ConfigFactory
object Pi extends App {
// Initiate the calculation
calculate(nrOfWorkers = 4, nrOfElements = 10000, nrOfMessages = 10000)
calculate(nrOfWorkers = 4, nrOfElements = 10, nrOfMessages = 10)
// ====================
// ===== Messages =====
@ -38,7 +38,9 @@ object Pi extends App {
}
def receive = {
case Work(start, nrOfElements) sender ! Result(calculatePiFor(start, nrOfElements)) // perform the work
case Work(start, nrOfElements)
println("*** RECEIVED MESSAGE IN: " + self.path)
sender ! Result(calculatePiFor(start, nrOfElements)) // perform the work
}
}
@ -53,14 +55,26 @@ object Pi extends App {
var start: Long = _
// create the workers
val workers = Vector.fill(nrOfWorkers)(context.actorOf[Worker])
var workers = Vector.empty[ActorRef]
for (i 1 to 2) {
workers = context.actorOf[Worker] +: workers
}
// TODO (HE) : use this way of creating actors
//val workers = Vector.fill(nrOfWorkers)(context.actorOf[Worker])
/*
// wrap them with a load-balancing router
// FIXME routers are intended to be used like this
implicit val timout = context.system.settings.ActorTimeout
implicit val dispatcher = context.dispatcher
val props = RoutedProps(routerFactory = () new RoundRobinRouter, connectionManager = new LocalConnectionManager(workers))
val router = new RoutedActorRef(context.system, props, self.asInstanceOf[InternalActorRef], "pi")
*/
//val router = context.actorOf(Props(new Worker).withRouting(RoundRobinRouter(nrOfInstances = 5)), "pi")
val router = context.actorOf(Props(new Worker).withRouting(RoundRobinRouter(nrOfInstances = 3, targets = Seq(workers.head, workers.tail.head))), "pi")
// message handler
def receive = {
@ -93,13 +107,13 @@ object Pi extends App {
// ===== Run it =====
// ==================
def calculate(nrOfWorkers: Int, nrOfElements: Int, nrOfMessages: Int) {
val system = ActorSystem()
val system = ActorSystem("x", ConfigFactory.parseString("akka.actor.debug.lifecycle=true\nakka.loglevel=DEBUG"))
// this latch is only plumbing to know when the calculation is completed
val latch = new CountDownLatch(1)
// create the master
val master = system.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch))
val master = system.actorOf(Props(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)), "master")
// start the calculation
master ! Calculate