Making the ExecutionContext protected and adding tryRecover
This commit is contained in:
commit
5d2669769f
83 changed files with 2900 additions and 2071 deletions
|
|
@ -268,7 +268,7 @@ public class JavaFutureTests {
|
|||
}
|
||||
}, system.dispatcher());
|
||||
|
||||
assertEquals(expect, Await.result(f, timeout));
|
||||
assertEquals(expect, Await.result(f, timeout).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -302,7 +302,21 @@ public class JavaFutureTests {
|
|||
});
|
||||
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
||||
p.failure(fail);
|
||||
Await.ready(p, d);
|
||||
assertEquals(Await.result(p, d), "foo");
|
||||
assertEquals(Await.result(f, d), "foo");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void tryRecoverToMustBeCallable() {
|
||||
final IllegalStateException fail = new IllegalStateException("OHNOES");
|
||||
Promise<Object> p = Futures.promise(system.dispatcher());
|
||||
Future<Object> f = p.future().tryRecover(new Recover<Future<Object>>() {
|
||||
public Future<Object> recover(Throwable t) throws Throwable {
|
||||
if (t == fail) return Futures.<Object>successful("foo", system.dispatcher()).future();
|
||||
else throw t;
|
||||
}
|
||||
});
|
||||
Duration d = Duration.create(1, TimeUnit.SECONDS);
|
||||
p.failure(fail);
|
||||
assertEquals(Await.result(f, d), "foo");
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ import akka.testkit._
|
|||
import akka.util.duration._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
import java.net.MalformedURLException
|
||||
|
||||
object ActorLookupSpec {
|
||||
|
||||
|
|
@ -46,9 +47,10 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
|
|||
val syst = sysImpl.systemGuardian
|
||||
val root = sysImpl.lookupRoot
|
||||
|
||||
def empty(path: String) = new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, path match {
|
||||
def empty(path: String) =
|
||||
new EmptyLocalActorRef(sysImpl.provider, path match {
|
||||
case RelativeActorPath(elems) ⇒ system.actorFor("/").path / elems
|
||||
})
|
||||
}, system.eventStream)
|
||||
|
||||
"An ActorSystem" must {
|
||||
|
||||
|
|
@ -286,4 +288,25 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
}
|
||||
|
||||
"An ActorPath" must {
|
||||
|
||||
"support parsing its String rep" in {
|
||||
val path = system.actorFor("user").path
|
||||
ActorPath.fromString(path.toString) must be(path)
|
||||
}
|
||||
|
||||
"support parsing remote paths" in {
|
||||
val remote = "akka://sys@host:1234/some/ref"
|
||||
ActorPath.fromString(remote).toString must be(remote)
|
||||
}
|
||||
|
||||
"throw exception upon malformed paths" in {
|
||||
intercept[MalformedURLException] { ActorPath.fromString("") }
|
||||
intercept[MalformedURLException] { ActorPath.fromString("://hallo") }
|
||||
intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") }
|
||||
intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") }
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -290,7 +290,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val sysImpl = system.asInstanceOf[ActorSystemImpl]
|
||||
val addr = sysImpl.provider.rootPath.address
|
||||
val serialized = SerializedActorRef(addr + "/non-existing")
|
||||
val serialized = SerializedActorRef(RootActorPath(addr, "/non-existing"))
|
||||
|
||||
out.writeObject(serialized)
|
||||
|
||||
|
|
@ -299,7 +299,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
Serialization.currentSystem.withValue(sysImpl) {
|
||||
val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray))
|
||||
in.readObject must be === new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, system.actorFor("/").path / "non-existing")
|
||||
in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -39,7 +39,6 @@ object FutureSpec {
|
|||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class JavaFutureSpec extends JavaFutureTests with JUnitSuite
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
|
|
|
|||
|
|
@ -56,11 +56,12 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
"decorate a Receive" in {
|
||||
new TestKit(appLogging) {
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
val r: Actor.Receive = {
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
def receive = new LoggingReceive(Some("funky"), {
|
||||
case null ⇒
|
||||
}
|
||||
val log = LoggingReceive("funky")(r)
|
||||
log.isDefinedAt("hallo")
|
||||
})
|
||||
}))
|
||||
a ! "hallo"
|
||||
expectMsg(1 second, Logging.Debug("funky", classOf[DummyClassForStringSources], "received unhandled message hallo"))
|
||||
}
|
||||
}
|
||||
|
|
@ -77,7 +78,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
|
||||
val actor = TestActorRef(new Actor {
|
||||
def switch: Actor.Receive = { case "becomenull" ⇒ context.become(r, false) }
|
||||
def receive = switch orElse LoggingReceive(this) {
|
||||
def receive = switch orElse LoggingReceive {
|
||||
case x ⇒ sender ! "x"
|
||||
}
|
||||
})
|
||||
|
|
@ -85,7 +86,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
val name = actor.path.toString
|
||||
actor ! "buh"
|
||||
within(1 second) {
|
||||
expectMsg(Logging.Debug(name, actor.underlyingActor.getClass, "received handled message buh"))
|
||||
expectMsg(Logging.Debug(actor.path.toString, actor.underlyingActor.getClass, "received handled message buh"))
|
||||
expectMsg("x")
|
||||
}
|
||||
|
||||
|
|
@ -105,7 +106,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
|
|||
new TestKit(appLogging) with ImplicitSender {
|
||||
system.eventStream.subscribe(testActor, classOf[Logging.Debug])
|
||||
val actor = TestActorRef(new Actor {
|
||||
def receive = LoggingReceive(this)(LoggingReceive(this) {
|
||||
def receive = LoggingReceive(LoggingReceive {
|
||||
case _ ⇒ sender ! "x"
|
||||
})
|
||||
})
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package akka.actor
|
||||
import scala.annotation.tailrec
|
||||
import java.net.MalformedURLException
|
||||
|
||||
object ActorPath {
|
||||
def split(s: String): List[String] = {
|
||||
|
|
@ -16,6 +17,11 @@ object ActorPath {
|
|||
rec(s.length, Nil)
|
||||
}
|
||||
|
||||
def fromString(s: String): ActorPath = s match {
|
||||
case ActorPathExtractor(addr, elems) ⇒ RootActorPath(addr) / elems
|
||||
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
||||
}
|
||||
|
||||
val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r
|
||||
}
|
||||
|
||||
|
|
@ -87,6 +93,12 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
|
|||
*/
|
||||
def root: RootActorPath
|
||||
|
||||
/**
|
||||
* Generate String representation, replacing the Address in the RootActor
|
||||
* Path with the given one unless this path’s address includes host and port
|
||||
* information.
|
||||
*/
|
||||
def toStringWithAddress(address: Address): String
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -105,6 +117,10 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
|||
|
||||
override val toString = address + name
|
||||
|
||||
def toStringWithAddress(addr: Address): String =
|
||||
if (address.host.isDefined) address + name
|
||||
else addr + name
|
||||
|
||||
def compareTo(other: ActorPath) = other match {
|
||||
case r: RootActorPath ⇒ toString compareTo r.toString
|
||||
case c: ChildActorPath ⇒ 1
|
||||
|
|
@ -151,6 +167,15 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
|||
rec(parent, new StringBuilder(32).append(name)).toString
|
||||
}
|
||||
|
||||
override def toStringWithAddress(addr: Address) = {
|
||||
@tailrec
|
||||
def rec(p: ActorPath, s: StringBuilder): StringBuilder = p match {
|
||||
case r: RootActorPath ⇒ s.insert(0, r.toStringWithAddress(addr))
|
||||
case _ ⇒ rec(p.parent, s.insert(0, '/').insert(0, p.name))
|
||||
}
|
||||
rec(parent, new StringBuilder(32).append(name)).toString
|
||||
}
|
||||
|
||||
override def equals(other: Any): Boolean = {
|
||||
@tailrec
|
||||
def rec(left: ActorPath, right: ActorPath): Boolean =
|
||||
|
|
|
|||
|
|
@ -211,7 +211,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe
|
|||
* This is an internal look-up failure token, not useful for anything else.
|
||||
*/
|
||||
private[akka] case object Nobody extends MinimalActorRef {
|
||||
val path = new RootActorPath(new LocalAddress("all-systems"), "/Nobody")
|
||||
val path = new RootActorPath(Address("akka", "all-systems"), "/Nobody")
|
||||
def provider = throw new UnsupportedOperationException("Nobody does not provide")
|
||||
}
|
||||
|
||||
|
|
@ -329,13 +329,13 @@ private[akka] class LocalActorRef private[akka] (
|
|||
def restart(cause: Throwable): Unit = actorCell.restart(cause)
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path.toString)
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
/**
|
||||
* Memento pattern for serializing ActorRefs transparently
|
||||
*/
|
||||
case class SerializedActorRef(path: String) {
|
||||
case class SerializedActorRef private (path: String) {
|
||||
import akka.serialization.Serialization.currentSystem
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
|
|
@ -349,6 +349,15 @@ case class SerializedActorRef(path: String) {
|
|||
}
|
||||
}
|
||||
|
||||
object SerializedActorRef {
|
||||
def apply(path: ActorPath): SerializedActorRef = {
|
||||
Serialization.currentTransportAddress.value match {
|
||||
case null ⇒ new SerializedActorRef(path.toString)
|
||||
case addr ⇒ new SerializedActorRef(path.toStringWithAddress(addr))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Trait for ActorRef implementations where all methods contain default stubs.
|
||||
*/
|
||||
|
|
@ -375,7 +384,7 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
|
|||
def restart(cause: Throwable): Unit = ()
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path.toString)
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
private[akka] object MinimalActorRef {
|
||||
|
|
@ -398,57 +407,39 @@ private[akka] object DeadLetterActorRef {
|
|||
val serialized = new SerializedDeadLetterActorRef
|
||||
}
|
||||
|
||||
private[akka] trait DeadLetterActorRefLike extends MinimalActorRef {
|
||||
|
||||
def eventStream: EventStream
|
||||
|
||||
@volatile
|
||||
private var _path: ActorPath = _
|
||||
def path: ActorPath = {
|
||||
assert(_path != null)
|
||||
_path
|
||||
}
|
||||
|
||||
@volatile
|
||||
private var _provider: ActorRefProvider = _
|
||||
def provider = _provider
|
||||
|
||||
private[akka] def init(provider: ActorRefProvider, path: ActorPath) {
|
||||
_path = path
|
||||
_provider = provider
|
||||
}
|
||||
|
||||
override def isTerminated(): Boolean = true
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRefLike {
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
/**
|
||||
* This special dead letter reference has a name: it is that which is returned
|
||||
* by a local look-up which is unsuccessful.
|
||||
*/
|
||||
private[akka] class EmptyLocalActorRef(
|
||||
val eventStream: EventStream,
|
||||
_provider: ActorRefProvider,
|
||||
_dispatcher: MessageDispatcher,
|
||||
_path: ActorPath) extends DeadLetterActorRefLike {
|
||||
val provider: ActorRefProvider,
|
||||
val path: ActorPath,
|
||||
val eventStream: EventStream) extends MinimalActorRef {
|
||||
|
||||
init(_provider, _path)
|
||||
override def isTerminated(): Boolean = true
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
|
||||
case d: DeadLetter ⇒ // do NOT form endless loops
|
||||
case d: DeadLetter ⇒ // do NOT form endless loops, since deadLetters will resend!
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal implementation of the dead letter destination: will publish any
|
||||
* received message to the eventStream, wrapped as [[akka.actor.DeadLetter]].
|
||||
*/
|
||||
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
|
||||
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal implementation detail used for paths like “/temp”
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -33,11 +33,22 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def systemGuardian: InternalActorRef
|
||||
|
||||
/**
|
||||
* Dead letter destination for this provider.
|
||||
*/
|
||||
def deadLetters: ActorRef
|
||||
|
||||
/**
|
||||
* Reference to the death watch service.
|
||||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* Care-taker of actor refs which await final termination but cannot be kept
|
||||
* in their parent’s children list because the name shall be freed.
|
||||
*/
|
||||
def locker: Locker
|
||||
|
||||
/**
|
||||
* The root path for all actors within this actor system, including remote
|
||||
* address if enabled.
|
||||
|
|
@ -281,25 +292,30 @@ class LocalActorRefProvider(
|
|||
val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
val deadLetters: InternalActorRef,
|
||||
val rootPath: ActorPath,
|
||||
val deployer: Deployer) extends ActorRefProvider {
|
||||
|
||||
// this is the constructor needed for reflectively instantiating the provider
|
||||
def this(_systemName: String,
|
||||
settings: ActorSystem.Settings,
|
||||
eventStream: EventStream,
|
||||
scheduler: Scheduler,
|
||||
deadLetters: InternalActorRef) =
|
||||
classloader: ClassLoader) =
|
||||
this(_systemName,
|
||||
settings,
|
||||
eventStream,
|
||||
scheduler,
|
||||
deadLetters,
|
||||
new RootActorPath(LocalAddress(_systemName)),
|
||||
new Deployer(settings))
|
||||
new Deployer(settings, classloader))
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
|
||||
val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")")
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
|
||||
|
||||
val deathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
val locker: Locker = new Locker(scheduler, settings.ReaperInterval, this, rootPath / "locker", deathWatch)
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
|
|
@ -455,8 +471,6 @@ class LocalActorRefProvider(
|
|||
tempContainer.removeChild(path.name)
|
||||
}
|
||||
|
||||
val deathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
def init(_system: ActorSystemImpl) {
|
||||
system = _system
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
|
|
@ -472,7 +486,7 @@ class LocalActorRefProvider(
|
|||
deadLetters
|
||||
} else if (elems.head.isEmpty) actorFor(rootGuardian, elems.tail)
|
||||
else actorFor(ref, elems)
|
||||
case LocalActorPath(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems)
|
||||
case ActorPathExtractor(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems)
|
||||
case _ ⇒
|
||||
log.debug("look-up of unknown path '{}' failed", path)
|
||||
deadLetters
|
||||
|
|
@ -492,7 +506,7 @@ class LocalActorRefProvider(
|
|||
} else ref.getChild(path.iterator) match {
|
||||
case Nobody ⇒
|
||||
log.debug("look-up of path sequence '{}' failed", path)
|
||||
new EmptyLocalActorRef(eventStream, system.provider, dispatcher, ref.path / path)
|
||||
new EmptyLocalActorRef(system.provider, ref.path / path, eventStream)
|
||||
case x ⇒ x
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -267,7 +267,7 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
* (below which the logging actors reside) and the execute all registered
|
||||
* termination handlers (see [[ActorSystem.registerOnTermination]]).
|
||||
*/
|
||||
def shutdown()
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Registers the provided extension and creates its payload, if this extension isn't already registered
|
||||
|
|
@ -322,6 +322,14 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* ClassLoader which is used for reflective accesses internally. This is set
|
||||
* to the context class loader, if one is set, or the class loader which
|
||||
* loaded the ActorSystem implementation. The context class loader is also
|
||||
* set on all threads created by the ActorSystem, if one was set during
|
||||
* creation.
|
||||
*/
|
||||
def internalClassLoader: ClassLoader
|
||||
}
|
||||
|
||||
class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem {
|
||||
|
|
@ -331,8 +339,10 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
|
||||
import ActorSystem._
|
||||
|
||||
final val settings = new Settings(applicationConfig, name)
|
||||
final val threadFactory = new MonitorableThreadFactory(name, settings.Daemonicity)
|
||||
final val settings: Settings = new Settings(applicationConfig, name)
|
||||
|
||||
final val threadFactory: MonitorableThreadFactory =
|
||||
MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader))
|
||||
|
||||
def logConfiguration(): Unit = log.info(settings.toString)
|
||||
|
||||
|
|
@ -377,18 +387,32 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
import settings._
|
||||
|
||||
// this provides basic logging (to stdout) until .start() is called below
|
||||
val eventStream = new EventStream(DebugEventStream)
|
||||
val eventStream: EventStream = new EventStream(DebugEventStream)
|
||||
eventStream.startStdoutLogger(settings)
|
||||
|
||||
// unfortunately we need logging before we know the rootpath address, which wants to be inserted here
|
||||
@volatile
|
||||
private var _log = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass)
|
||||
def log = _log
|
||||
val log: LoggingAdapter = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass)
|
||||
|
||||
val scheduler = createScheduler()
|
||||
val scheduler: Scheduler = createScheduler()
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(eventStream)
|
||||
val deadLetterMailbox = new Mailbox(null) {
|
||||
val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader
|
||||
|
||||
val provider: ActorRefProvider = {
|
||||
val arguments = Seq(
|
||||
classOf[String] -> name,
|
||||
classOf[Settings] -> settings,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[Scheduler] -> scheduler,
|
||||
classOf[ClassLoader] -> internalClassLoader)
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
}
|
||||
|
||||
def deadLetters: ActorRef = provider.deadLetters
|
||||
|
||||
val deadLetterMailbox: Mailbox = new Mailbox(null) {
|
||||
becomeClosed()
|
||||
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||
override def dequeue() = null
|
||||
|
|
@ -399,28 +423,12 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
override def numberOfMessages = 0
|
||||
}
|
||||
|
||||
val provider: ActorRefProvider = {
|
||||
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(b) ⇒ b
|
||||
}
|
||||
val arguments = Seq(
|
||||
classOf[String] -> name,
|
||||
classOf[Settings] -> settings,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[Scheduler] -> scheduler,
|
||||
classOf[InternalActorRef] -> deadLetters)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
def locker: Locker = provider.locker
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
}
|
||||
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
||||
threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader))
|
||||
|
||||
val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler))
|
||||
val dispatcher = dispatchers.defaultGlobalDispatcher
|
||||
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
||||
|
||||
def terminationFuture: Future[Unit] = provider.terminationFuture
|
||||
def lookupRoot: InternalActorRef = provider.rootGuardian
|
||||
|
|
@ -434,21 +442,13 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
private lazy val _start: this.type = {
|
||||
// the provider is expected to start default loggers, LocalActorRefProvider does this
|
||||
provider.init(this)
|
||||
_log = new BusLogging(eventStream, "ActorSystem(" + lookupRoot.path.address + ")", this.getClass)
|
||||
deadLetters.init(provider, lookupRoot.path / "deadLetters")
|
||||
registerOnTermination(stopScheduler())
|
||||
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
|
||||
_locker = new Locker(scheduler, ReaperInterval, provider, lookupRoot.path / "locker", deathWatch)
|
||||
loadExtensions()
|
||||
if (LogConfigOnStart) logConfiguration()
|
||||
this
|
||||
}
|
||||
|
||||
@volatile
|
||||
private var _locker: Locker = _ // initialized in start()
|
||||
def locker = _locker
|
||||
|
||||
def start() = _start
|
||||
def start(): this.type = _start
|
||||
|
||||
private lazy val terminationCallbacks = {
|
||||
val callbacks = new TerminationCallbacks
|
||||
|
|
@ -460,9 +460,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) }
|
||||
def awaitTermination() = awaitTermination(Duration.Inf)
|
||||
|
||||
def shutdown() {
|
||||
stop(guardian)
|
||||
}
|
||||
def shutdown(): Unit = stop(guardian)
|
||||
|
||||
/**
|
||||
* Create the scheduler service. This one needs one special behavior: if
|
||||
|
|
@ -547,8 +545,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
private def loadExtensions() {
|
||||
import scala.collection.JavaConversions._
|
||||
settings.config.getStringList("akka.extensions") foreach { fqcn ⇒
|
||||
import ReflectiveAccess._
|
||||
getObjectFor[AnyRef](fqcn).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
|
||||
import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs }
|
||||
getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
|
||||
case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup());
|
||||
case Right(p: ExtensionId[_]) ⇒ registerExtension(p);
|
||||
case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||
|
|
@ -558,7 +556,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
}
|
||||
}
|
||||
|
||||
override def toString = lookupRoot.path.root.address.toString
|
||||
override def toString: String = lookupRoot.path.root.address.toString
|
||||
|
||||
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
|
||||
private val lock = new ReentrantGuard
|
||||
|
|
|
|||
|
|
@ -9,17 +9,41 @@ import java.net.URISyntaxException
|
|||
* The address specifies the physical location under which an Actor can be
|
||||
* reached. Examples are local addresses, identified by the ActorSystem’s
|
||||
* name, and remote addresses, identified by protocol, host and port.
|
||||
*
|
||||
* This class is final to allow use as a case class (copy method etc.); if
|
||||
* for example a remote transport would want to associate additional
|
||||
* information with an address, then this must be done externally.
|
||||
*/
|
||||
abstract class Address {
|
||||
def protocol: String
|
||||
def hostPort: String
|
||||
final case class Address(protocol: String, system: String, host: Option[String], port: Option[Int]) {
|
||||
|
||||
def this(protocol: String, system: String) = this(protocol, system, None, None)
|
||||
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
|
||||
|
||||
@transient
|
||||
override lazy val toString = protocol + "://" + hostPort
|
||||
override lazy val toString = {
|
||||
val sb = new StringBuilder(protocol)
|
||||
sb.append("://")
|
||||
sb.append(hostPort)
|
||||
sb.toString
|
||||
}
|
||||
|
||||
@transient
|
||||
lazy val hostPort = {
|
||||
val sb = new StringBuilder(system)
|
||||
if (host.isDefined) {
|
||||
sb.append('@')
|
||||
sb.append(host.get)
|
||||
}
|
||||
if (port.isDefined) {
|
||||
sb.append(':')
|
||||
sb.append(port.get)
|
||||
}
|
||||
sb.toString
|
||||
}
|
||||
}
|
||||
|
||||
case class LocalAddress(systemName: String) extends Address {
|
||||
def protocol = "akka"
|
||||
def hostPort = systemName
|
||||
object Address {
|
||||
def apply(protocol: String, system: String) = new Address(protocol, system)
|
||||
}
|
||||
|
||||
object RelativeActorPath {
|
||||
|
|
@ -32,12 +56,34 @@ object RelativeActorPath {
|
|||
}
|
||||
}
|
||||
|
||||
object LocalActorPath {
|
||||
def unapply(addr: String): Option[(LocalAddress, Iterable[String])] = {
|
||||
object AddressExtractor {
|
||||
def unapply(addr: String): Option[Address] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme != "akka" || uri.getUserInfo != null || uri.getHost == null || uri.getPath == null) None
|
||||
else Some(LocalAddress(uri.getHost), ActorPath.split(uri.getPath).drop(1))
|
||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None
|
||||
else {
|
||||
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
|
||||
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
|
||||
if (uri.getPort < 0) None else Some(uri.getPort))
|
||||
Some(addr)
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ActorPathExtractor {
|
||||
def unapply(addr: String): Option[(Address, Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None
|
||||
else {
|
||||
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
|
||||
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
|
||||
if (uri.getPort < 0) None else Some(uri.getPort))
|
||||
Some((addr, ActorPath.split(uri.getPath).drop(1)))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ case object LocalScope extends Scope
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Deployer(val settings: ActorSystem.Settings) {
|
||||
class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
|
@ -41,7 +41,6 @@ class Deployer(val settings: ActorSystem.Settings) {
|
|||
def deploy(d: Deploy): Unit = deployments.put(d.path, d)
|
||||
|
||||
protected def parseConfig(key: String, config: Config): Option[Deploy] = {
|
||||
import akka.util.ReflectiveAccess.getClassFor
|
||||
|
||||
val deployment = config.withFallback(default)
|
||||
|
||||
|
|
@ -65,8 +64,8 @@ class Deployer(val settings: ActorSystem.Settings) {
|
|||
case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer)
|
||||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
||||
case fqn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config])
|
||||
ReflectiveAccess.createInstance[RouterConfig](fqn, constructorSignature, Array[AnyRef](deployment)) match {
|
||||
val args = Seq(classOf[Config] -> deployment)
|
||||
ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match {
|
||||
case Right(router) ⇒ router
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -38,6 +38,12 @@ trait ExtensionId[T <: Extension] {
|
|||
/**
|
||||
* Returns an instance of the extension identified by this ExtensionId instance.
|
||||
* Java API
|
||||
* For extensions written in Scala that are to be used used from Java also,
|
||||
* this method should be overridden to get correct return type.
|
||||
* {{{
|
||||
* override def get(system: ActorSystem): TheExtension = super.get(system)
|
||||
* }}}
|
||||
*
|
||||
*/
|
||||
def get(system: ActorSystem): T = apply(system)
|
||||
|
||||
|
|
|
|||
|
|
@ -227,8 +227,8 @@ abstract class SupervisorStrategy {
|
|||
}
|
||||
|
||||
/**
|
||||
* Restart all actors linked to the same supervisor when one fails,
|
||||
* @param maxNrOfRetries the number of times an actor is allowed to be restarted
|
||||
* Restart all child actors when one fails
|
||||
* @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit
|
||||
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
|
||||
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a
|
||||
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
|
||||
|
|
@ -270,8 +270,8 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
|
|||
}
|
||||
|
||||
/**
|
||||
* Restart an actor when it fails
|
||||
* @param maxNrOfRetries the number of times an actor is allowed to be restarted
|
||||
* Restart a child actor when it fails
|
||||
* @param maxNrOfRetries the number of times an actor is allowed to be restarted, negative value means no limit
|
||||
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
|
||||
* @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a
|
||||
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
|
||||
|
|
|
|||
|
|
@ -321,8 +321,8 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
BoundedMailbox(capacity, duration)
|
||||
}
|
||||
case fqcn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config])
|
||||
ReflectiveAccess.createInstance[MailboxType](fqcn, constructorSignature, Array[AnyRef](config)) match {
|
||||
val args = Seq(classOf[Config] -> config)
|
||||
ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -19,13 +19,15 @@ trait DispatcherPrerequisites {
|
|||
def eventStream: EventStream
|
||||
def deadLetterMailbox: Mailbox
|
||||
def scheduler: Scheduler
|
||||
def classloader: ClassLoader
|
||||
}
|
||||
|
||||
case class DefaultDispatcherPrerequisites(
|
||||
val threadFactory: ThreadFactory,
|
||||
val eventStream: EventStream,
|
||||
val deadLetterMailbox: Mailbox,
|
||||
val scheduler: Scheduler) extends DispatcherPrerequisites
|
||||
val scheduler: Scheduler,
|
||||
val classloader: ClassLoader) extends DispatcherPrerequisites
|
||||
|
||||
object Dispatchers {
|
||||
/**
|
||||
|
|
@ -134,8 +136,8 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||
case fqn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, constructorSignature, Array[AnyRef](cfg, prerequisites)) match {
|
||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match {
|
||||
case Right(configurator) ⇒ configurator
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -342,7 +342,7 @@ object Future {
|
|||
|
||||
sealed trait Future[+T] extends Await.Awaitable[T] {
|
||||
|
||||
implicit def executor: ExecutionContext
|
||||
protected implicit def executor: ExecutionContext
|
||||
|
||||
protected final def resolve[X](source: Either[Throwable, X]): Either[Throwable, X] = source match {
|
||||
case Left(t: scala.runtime.NonLocalReturnControl[_]) ⇒ Right(t.value.asInstanceOf[X])
|
||||
|
|
@ -471,6 +471,32 @@ sealed trait Future[+T] extends Await.Awaitable[T] {
|
|||
future
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new future that will handle any matching throwable that this
|
||||
* future might contain by assigning it a value of another future.
|
||||
*
|
||||
* If there is no match, or if this future contains
|
||||
* a valid result then the new future will contain the same result.
|
||||
*
|
||||
* Example:
|
||||
*
|
||||
* {{{
|
||||
* val f = Future { Int.MaxValue }
|
||||
* future (6 / 0) tryRecover { case e: ArithmeticException => f } // result: Int.MaxValue
|
||||
* }}}
|
||||
*/
|
||||
def tryRecover[U >: T](pf: PartialFunction[Throwable, Future[U]]): Future[U] = {
|
||||
val p = Promise[U]()
|
||||
|
||||
onComplete {
|
||||
case Left(t) if pf isDefinedAt t ⇒
|
||||
try { p completeWith pf(t) } catch { case t: Throwable ⇒ p complete resolve(Left(t)) }
|
||||
case otherwise ⇒ p complete otherwise
|
||||
}
|
||||
|
||||
p.future
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Future by applying a function to the successful result of
|
||||
* this Future. If this Future is completed with an exception then the new
|
||||
|
|
|
|||
|
|
@ -159,6 +159,7 @@ object MonitorableThreadFactory {
|
|||
|
||||
case class MonitorableThreadFactory(name: String,
|
||||
daemonic: Boolean,
|
||||
contextClassLoader: Option[ClassLoader],
|
||||
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing)
|
||||
extends ThreadFactory {
|
||||
protected val counter = new AtomicLong
|
||||
|
|
@ -167,6 +168,7 @@ case class MonitorableThreadFactory(name: String,
|
|||
val t = new Thread(runnable, name + counter.incrementAndGet())
|
||||
t.setUncaughtExceptionHandler(exceptionHandler)
|
||||
t.setDaemon(daemonic)
|
||||
contextClassLoader foreach (t.setContextClassLoader(_))
|
||||
t
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -101,7 +101,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
if loggerName != StandardOutLoggerName
|
||||
} yield {
|
||||
try {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName) match {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match {
|
||||
case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName)
|
||||
case Left(exception) ⇒ throw exception
|
||||
}
|
||||
|
|
@ -648,7 +648,7 @@ object Logging {
|
|||
* <code>akka.stdout-loglevel</code> in <code>akka.conf</code>.
|
||||
*/
|
||||
class StandardOutLogger extends MinimalActorRef with StdOutLogger {
|
||||
val path: ActorPath = new RootActorPath(LocalAddress("all-systems"), "/StandardOutLogger")
|
||||
val path: ActorPath = new RootActorPath(Address("akka", "all-systems"), "/StandardOutLogger")
|
||||
def provider: ActorRefProvider = throw new UnsupportedOperationException("StandardOutLogger does not provide")
|
||||
override val toString = "StandardOutLogger"
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message)
|
||||
|
|
|
|||
|
|
@ -4,7 +4,8 @@
|
|||
package akka.event
|
||||
|
||||
import akka.actor.Actor.Receive
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.ActorCell
|
||||
import akka.event.Logging.Debug
|
||||
|
||||
object LoggingReceive {
|
||||
|
|
@ -15,7 +16,7 @@ object LoggingReceive {
|
|||
* This includes messages which are not handled.
|
||||
*
|
||||
* <pre><code>
|
||||
* def receive = LoggingReceive(this) {
|
||||
* def receive = LoggingReceive {
|
||||
* case x => ...
|
||||
* }
|
||||
* </code></pre>
|
||||
|
|
@ -23,21 +24,23 @@ object LoggingReceive {
|
|||
* This method does NOT modify the given Receive unless
|
||||
* akka.actor.debug.receive is set within akka.conf.
|
||||
*/
|
||||
def apply(source: AnyRef)(r: Receive)(implicit system: ActorSystem): Receive = r match {
|
||||
def apply(r: Receive)(implicit context: ActorContext): Receive = r match {
|
||||
case _: LoggingReceive ⇒ r
|
||||
case _ if !system.settings.AddLoggingReceive ⇒ r
|
||||
case _ ⇒ new LoggingReceive(source, r)
|
||||
case _ ⇒
|
||||
if (context.system.settings.AddLoggingReceive) new LoggingReceive(None, r)
|
||||
else r
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This decorator adds invocation logging to a Receive function.
|
||||
* @param source the log source, if not defined the actor of the context will be used
|
||||
*/
|
||||
class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive {
|
||||
class LoggingReceive(source: Option[AnyRef], r: Receive)(implicit context: ActorContext) extends Receive {
|
||||
def isDefinedAt(o: Any) = {
|
||||
val handled = r.isDefinedAt(o)
|
||||
val (str, clazz) = LogSource.fromAnyRef(source)
|
||||
system.eventStream.publish(Debug(str, clazz, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
|
||||
val (str, clazz) = LogSource.fromAnyRef(source getOrElse context.asInstanceOf[ActorCell].actor)
|
||||
context.system.eventStream.publish(Debug(str, clazz, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
|
||||
handled
|
||||
}
|
||||
def apply(o: Any): Unit = r(o)
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.util.ReflectiveAccess
|
|||
import scala.util.DynamicVariable
|
||||
import com.typesafe.config.Config
|
||||
import akka.config.ConfigurationException
|
||||
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem }
|
||||
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address }
|
||||
|
||||
case class NoSerializerFoundException(m: String) extends AkkaException(m)
|
||||
|
||||
|
|
@ -27,6 +27,12 @@ object Serialization {
|
|||
*/
|
||||
val currentSystem = new DynamicVariable[ActorSystem](null)
|
||||
|
||||
/**
|
||||
* This holds a reference to the current transport address to be inserted
|
||||
* into local actor refs during serialization.
|
||||
*/
|
||||
val currentTransportAddress = new DynamicVariable[Address](null)
|
||||
|
||||
class Settings(val config: Config) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -75,10 +81,10 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
def deserialize(bytes: Array[Byte],
|
||||
serializerId: Int,
|
||||
clazz: Option[Class[_]],
|
||||
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
|
||||
classLoader: ClassLoader): Either[Exception, AnyRef] =
|
||||
try {
|
||||
currentSystem.withValue(system) {
|
||||
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, classLoader))
|
||||
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader)))
|
||||
}
|
||||
} catch { case e: Exception ⇒ Left(e) }
|
||||
|
||||
|
|
|
|||
|
|
@ -38,6 +38,15 @@ object ReflectiveAccess {
|
|||
}
|
||||
}
|
||||
|
||||
def createInstance[T](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
|
||||
createInstance(clazz, args.map(_._1).toArray, args.map(_._2).toArray)
|
||||
|
||||
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)], classloader: ClassLoader): Either[Exception, T] =
|
||||
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, classloader)
|
||||
|
||||
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
|
||||
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, loader)
|
||||
|
||||
//Obtains a reference to fqn.MODULE$
|
||||
def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, T] = try {
|
||||
getClassFor(fqn, classloader) match {
|
||||
|
|
|
|||
|
|
@ -217,6 +217,20 @@ and parsed by the actor system can be displayed like this:
|
|||
println(system.settings());
|
||||
// this is a shortcut for system.settings().config().root().render()
|
||||
|
||||
A Word About ClassLoaders
|
||||
-------------------------
|
||||
|
||||
In several places of the configuration file it is possible to specify the
|
||||
fully-qualified class name of something to be instantiated by Akka. This is
|
||||
done using Java reflection, which in turn uses a :class:`ClassLoader`. Getting
|
||||
the right one in challenging environments like application containers or OSGi
|
||||
bundles is not always trivial, the current approach of Akka is that each
|
||||
:class:`ActorSystem` implementation stores the current thread’s context class
|
||||
loader (if available, otherwise just its own loader as in
|
||||
``this.getClass.getClassLoader``) and uses that for all reflective accesses.
|
||||
This implies that putting Akka on the boot class path will yield
|
||||
:class:`NullPointerException` from strange places: this is simply not
|
||||
supported.
|
||||
|
||||
Application specific settings
|
||||
-----------------------------
|
||||
|
|
|
|||
|
|
@ -237,13 +237,24 @@ e.g. in ``$AKKA_HOME/tutorial/akka/tutorial/first/java/Pi.java``.
|
|||
Creating the messages
|
||||
---------------------
|
||||
|
||||
The design we are aiming for is to have one ``Master`` actor initiating the computation, creating a set of ``Worker`` actors. Then it splits up the work into discrete chunks, and sends these chunks to the different workers in a round-robin fashion. The master waits until all the workers have completed their work and sent back results for aggregation. When computation is completed the master prints out the result, shuts down all workers and then itself.
|
||||
The design we are aiming for is to have one ``Master`` actor initiating the
|
||||
computation, creating a set of ``Worker`` actors. Then it splits up the work
|
||||
into discrete chunks, and sends these chunks to the different workers in a
|
||||
round-robin fashion. The master waits until all the workers have completed their
|
||||
work and sent back results for aggregation. When computation is completed the
|
||||
master sends the result to the ``Listener``, which prints out the result.
|
||||
|
||||
With this in mind, let's now create the messages that we want to have flowing in the system. We need three different messages:
|
||||
With this in mind, let's now create the messages that we want to have flowing in
|
||||
the system. We need four different messages:
|
||||
|
||||
- ``Calculate`` -- sent to the ``Master`` actor to start the calculation
|
||||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor containing the result from the worker's calculation
|
||||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing
|
||||
the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor
|
||||
containing the result from the worker's calculation
|
||||
- ``PiApproximation`` -- sent from the ``Master`` actor to the
|
||||
``Listener`` actor containing the the final pi result and how long time
|
||||
the calculation took
|
||||
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state. So let's start by creating three messages as immutable POJOs. We also create a wrapper ``Pi`` class to hold our implementation:
|
||||
|
||||
|
|
@ -285,19 +296,8 @@ Here is the master actor:
|
|||
|
||||
A couple of things are worth explaining further.
|
||||
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``ask()`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
Note that we are passing in a ``ActorRef`` to the ``Master`` actor. This is used to
|
||||
report the the final result to the outside world.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master`` actor.
|
||||
This message handler needs to be able to react to two different messages:
|
||||
|
|
@ -310,15 +310,25 @@ The ``Calculate`` handler is sending out work to all the ``Worker`` via its rout
|
|||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
sends the final result to the ``listener``. When done it also invokes the ``getContext().stop(getSelf())``
|
||||
method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
||||
|
||||
Let's capture this in code:
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#master-receive
|
||||
|
||||
Creating the result listener
|
||||
----------------------------
|
||||
|
||||
The listener is straightforward. When it receives the ``PiApproximation`` from the ``Master`` it
|
||||
prints the result and shuts down the ``ActorSystem``.
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#result-listener
|
||||
|
||||
Bootstrap the calculation
|
||||
-------------------------
|
||||
|
||||
|
|
@ -329,11 +339,11 @@ invoke method ``calculate`` in which we start up the ``Master`` actor and wait f
|
|||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
As you can see the *calculate* method above it creates an ActorSystem and this is the Akka container which
|
||||
As you can see the *calculate* method above it creates an ``ActorSystem`` and this is the Akka container which
|
||||
will contain all actors created in that "context". An example of how to create actors in the container
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create a top level actor.
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create two top level actors.
|
||||
If you instead where in an actor context, i.e. inside an actor creating other actors, you should use
|
||||
*this.getContext.actorOf(...)*. This is illustrated in the Master code above.
|
||||
*getContext().actorOf(...)*. This is illustrated in the Master code above.
|
||||
|
||||
That's it. Now we are done.
|
||||
|
||||
|
|
@ -365,8 +375,8 @@ we compiled ourselves::
|
|||
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:. \
|
||||
akka.tutorial.java.first.Pi
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 609 millis
|
||||
Pi approximation: 3.1435501812459323
|
||||
Calculation time: 359 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
@ -382,8 +392,8 @@ When this in done we can run our application directly inside Maven::
|
|||
|
||||
$ mvn exec:java -Dexec.mainClass="akka.tutorial.first.java.Pi"
|
||||
...
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 597 millis
|
||||
Pi approximation: 3.1435501812459323
|
||||
Calculation time: 359 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
|
|||
|
|
@ -268,11 +268,16 @@ We start by creating case classes for each type of message in our application, s
|
|||
call it ``PiMessage``. Right click on the package and choose ``New Scala Class``, and enter ``PiMessage`` as
|
||||
the name of the class.
|
||||
|
||||
We need three different messages:
|
||||
We need four different messages:
|
||||
|
||||
- ``Calculate`` -- sent to the ``Master`` actor to start the calculation
|
||||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor containing the result from the worker's calculation
|
||||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing
|
||||
the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor
|
||||
containing the result from the worker's calculation
|
||||
- ``PiApproximation`` -- sent from the ``Master`` actor to the
|
||||
``Listener`` actor containing the the final pi result and how long time
|
||||
the calculation took
|
||||
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable state.
|
||||
In Scala we have 'case classes' which make excellent messages. So let's start by creating three messages as case classes.
|
||||
|
|
@ -343,19 +348,8 @@ Here is the master actor:
|
|||
|
||||
A couple of things are worth explaining further.
|
||||
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``?`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
Note that we are passing in a ``ActorRef`` to the ``Master`` actor. This is used to
|
||||
report the the final result to the outside world.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master``
|
||||
actor. This message handler needs to be able to react to two different messages:
|
||||
|
|
@ -368,7 +362,8 @@ The ``Calculate`` handler is sending out work to all the ``Worker`` via its rout
|
|||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
sends the final result to the ``listener``. When done it also invokes the ``context.stop(self)``
|
||||
method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
|
@ -377,6 +372,13 @@ Let's capture this in code:
|
|||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive
|
||||
|
||||
Creating the result listener
|
||||
----------------------------
|
||||
|
||||
The listener is straightforward. When it receives the ``PiApproximation`` from the ``Master`` it
|
||||
prints the result and shuts down the ``ActorSystem``.
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#result-listener
|
||||
|
||||
Bootstrap the calculation
|
||||
-------------------------
|
||||
|
|
@ -391,6 +393,12 @@ We also create a method ``calculate`` in which we start up the ``Master`` actor
|
|||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
As you can see the *calculate* method above it creates an ``ActorSystem`` and this is the Akka container which
|
||||
will contain all actors created in that "context". An example of how to create actors in the container
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create two top level actors.
|
||||
If you instead where in an actor context, i.e. inside an actor creating other actors, you should use
|
||||
*context.actorOf(...)*. This is illustrated in the Master code above.
|
||||
|
||||
That's it. Now we are done.
|
||||
|
||||
Run it from Eclipse
|
||||
|
|
@ -401,8 +409,8 @@ If not, bring you project up to date by clicking ``Project/Build Project``. If t
|
|||
you can right-click in the editor where ``Pi`` is defined, and choose ``Run as.. /Scala application``.
|
||||
If everything works fine, you should see::
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 632 millis
|
||||
Pi approximation: 3.1435501812459323
|
||||
Calculation time: 359 millis
|
||||
|
||||
You can also define a new Run configuration, by going to ``Run/Run Configurations``. Create a new ``Scala application``
|
||||
and choose the tutorial project and the main class to be ``akkatutorial.Pi``. You can pass additional command line
|
||||
|
|
|
|||
|
|
@ -262,16 +262,19 @@ computation, creating a set of ``Worker`` actors. Then it splits up the work
|
|||
into discrete chunks, and sends these chunks to the different workers in a
|
||||
round-robin fashion. The master waits until all the workers have completed their
|
||||
work and sent back results for aggregation. When computation is completed the
|
||||
master prints out the result, shuts down all workers and then itself.
|
||||
master sends the result to the ``Listener``, which prints out the result.
|
||||
|
||||
With this in mind, let's now create the messages that we want to have flowing in
|
||||
the system. We need three different messages:
|
||||
the system. We need four different messages:
|
||||
|
||||
- ``Calculate`` -- sent to the ``Master`` actor to start the calculation
|
||||
- ``Work`` -- sent from the ``Master`` actor to the ``Worker`` actors containing
|
||||
the work assignment
|
||||
- ``Result`` -- sent from the ``Worker`` actors to the ``Master`` actor
|
||||
containing the result from the worker's calculation
|
||||
- ``PiApproximation`` -- sent from the ``Master`` actor to the
|
||||
``Listener`` actor containing the the final pi result and how long time
|
||||
the calculation took
|
||||
|
||||
Messages sent to actors should always be immutable to avoid sharing mutable
|
||||
state. In scala we have 'case classes' which make excellent messages. So let's
|
||||
|
|
@ -330,19 +333,8 @@ Here is the master actor:
|
|||
|
||||
A couple of things are worth explaining further.
|
||||
|
||||
First, we are passing in a ``java.util.concurrent.CountDownLatch`` to the
|
||||
``Master`` actor. This latch is only used for plumbing (in this specific
|
||||
tutorial), to have a simple way of letting the outside world knowing when the
|
||||
master can deliver the result and shut down. In more idiomatic Akka code
|
||||
we would not use a latch but other abstractions and functions like ``Future``
|
||||
and ``?`` to achieve the same thing in a non-blocking way.
|
||||
But for simplicity let's stick to a ``CountDownLatch`` for now.
|
||||
|
||||
Second, we are adding a couple of life-cycle callback methods; ``preStart`` and
|
||||
``postStop``. In the ``preStart`` callback we are recording the time when the
|
||||
actor is started and in the ``postStop`` callback we are printing out the result
|
||||
(the approximation of Pi) and the time it took to calculate it. In this call we
|
||||
also invoke ``latch.countDown()`` to tell the outside world that we are done.
|
||||
Note that we are passing in a ``ActorRef`` to the ``Master`` actor. This is used to
|
||||
report the the final result to the outside world.
|
||||
|
||||
But we are not done yet. We are missing the message handler for the ``Master``
|
||||
actor. This message handler needs to be able to react to two different messages:
|
||||
|
|
@ -355,7 +347,8 @@ The ``Calculate`` handler is sending out work to all the ``Worker`` via its rout
|
|||
The ``Result`` handler gets the value from the ``Result`` message and aggregates it to
|
||||
our ``pi`` member variable. We also keep track of how many results we have received back,
|
||||
and if that matches the number of tasks sent out, the ``Master`` actor considers itself done and
|
||||
invokes the ``self.stop()`` method to stop itself *and* all its supervised actors.
|
||||
sends the final result to the ``listener``. When done it also invokes the ``context.stop(self)``
|
||||
method to stop itself *and* all its supervised actors.
|
||||
In this case it has one supervised actor, the router, and this in turn has ``nrOfWorkers`` supervised actors.
|
||||
All of them will be stopped automatically as the invocation of any supervisor's ``stop`` method
|
||||
will propagate down to all its supervised 'children'.
|
||||
|
|
@ -365,6 +358,14 @@ Let's capture this in code:
|
|||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#master-receive
|
||||
|
||||
|
||||
Creating the result listener
|
||||
============================
|
||||
|
||||
The listener is straightforward. When it receives the ``PiApproximation`` from the ``Master`` it
|
||||
prints the result and shuts down the ``ActorSystem``.
|
||||
|
||||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#result-listener
|
||||
|
||||
Bootstrap the calculation
|
||||
=========================
|
||||
|
||||
|
|
@ -380,9 +381,9 @@ start up the ``Master`` actor and wait for it to finish:
|
|||
.. includecode:: ../../akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala#app
|
||||
:exclude: actors-and-messages
|
||||
|
||||
As you can see the *calculate* method above it creates an ActorSystem and this is the Akka container which
|
||||
As you can see the *calculate* method above it creates an ``ActorSystem`` and this is the Akka container which
|
||||
will contain all actors created in that "context". An example of how to create actors in the container
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create a top level actor.
|
||||
is the *'system.actorOf(...)'* line in the calculate method. In this case we create two top level actors.
|
||||
If you instead where in an actor context, i.e. inside an actor creating other actors, you should use
|
||||
*context.actorOf(...)*. This is illustrated in the Master code above.
|
||||
|
||||
|
|
@ -417,8 +418,8 @@ compiled ourselves::
|
|||
-cp lib/scala-library.jar:lib/akka/akka-actor-2.0-SNAPSHOT.jar:. \
|
||||
akka.tutorial.first.scala.Pi
|
||||
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 553 millis
|
||||
Pi approximation: 3.1435501812459323
|
||||
Calculation time: 359 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
@ -436,8 +437,8 @@ When this in done we can run our application directly inside SBT::
|
|||
|
||||
> run
|
||||
...
|
||||
Pi estimate: 3.1435501812459323
|
||||
Calculation time: 531 millis
|
||||
Pi approximation: 3.1435501812459323
|
||||
Calculation time: 359 millis
|
||||
|
||||
Yippee! It is working.
|
||||
|
||||
|
|
|
|||
|
|
@ -82,6 +82,20 @@ Akka can be used in two different ways
|
|||
|
||||
See the :ref:`deployment-scenarios` for details.
|
||||
|
||||
What happened to Cloudy Akka?
|
||||
=============================
|
||||
|
||||
The commercial offering was earlier referred to as Cloudy Akka. This offering
|
||||
consisted of two things:
|
||||
|
||||
- Cluster support for Akka
|
||||
- Monitoring & Management (formerly called Atmos)
|
||||
|
||||
Cloudy Akka have been discontinued and the Cluster support is now being moved into the
|
||||
Open Source version of Akka (the upcoming Akka 2.1), while the Monitoring & Management
|
||||
(Atmos) is now rebranded into Typesafe Console and is part of the commercial subscription
|
||||
for the Typesafe Stack (see below for details).
|
||||
|
||||
Typesafe Stack
|
||||
==============
|
||||
|
||||
|
|
@ -100,6 +114,7 @@ Typesafe Console
|
|||
On top of the Typesafe Stack we have also have commercial product called Typesafe
|
||||
Console which provides the following features:
|
||||
|
||||
#. Slick Web UI with real-time view into the system
|
||||
#. Management through Dashboard, JMX and REST
|
||||
#. Dapper-style tracing of messages across components and remote nodes
|
||||
#. Real-time statistics
|
||||
|
|
|
|||
|
|
@ -18,12 +18,14 @@ import akka.actor.UntypedActor;
|
|||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.testkit.TestProbe;
|
||||
import akka.testkit.AkkaSpec;
|
||||
|
||||
public class FSMDocTestBase {
|
||||
|
||||
//#data
|
||||
public static final class SetTarget {
|
||||
final ActorRef ref;
|
||||
|
||||
public SetTarget(ActorRef ref) {
|
||||
this.ref = ref;
|
||||
}
|
||||
|
|
@ -31,6 +33,7 @@ public class FSMDocTestBase {
|
|||
|
||||
public static final class Queue {
|
||||
final Object o;
|
||||
|
||||
public Queue(Object o) {
|
||||
this.o = o;
|
||||
}
|
||||
|
|
@ -40,10 +43,12 @@ public class FSMDocTestBase {
|
|||
|
||||
public static final class Batch {
|
||||
final List<Object> objects;
|
||||
|
||||
public Batch(List<Object> objects) {
|
||||
this.objects = objects;
|
||||
}
|
||||
}
|
||||
|
||||
//#data
|
||||
|
||||
//#base
|
||||
|
|
@ -52,7 +57,10 @@ public class FSMDocTestBase {
|
|||
/*
|
||||
* This is the mutable state of this state machine.
|
||||
*/
|
||||
protected enum State { IDLE, ACTIVE; }
|
||||
protected enum State {
|
||||
IDLE, ACTIVE;
|
||||
}
|
||||
|
||||
private State state = State.IDLE;
|
||||
private ActorRef target;
|
||||
private List<Object> queue;
|
||||
|
|
@ -73,12 +81,14 @@ public class FSMDocTestBase {
|
|||
}
|
||||
|
||||
protected void enqueue(Object o) {
|
||||
if (queue != null) queue.add(o);
|
||||
if (queue != null)
|
||||
queue.add(o);
|
||||
}
|
||||
|
||||
protected List<Object> drainQueue() {
|
||||
final List<Object> q = queue;
|
||||
if (q == null) throw new IllegalStateException("drainQueue(): not yet initialized");
|
||||
if (q == null)
|
||||
throw new IllegalStateException("drainQueue(): not yet initialized");
|
||||
queue = new ArrayList<Object>();
|
||||
return q;
|
||||
}
|
||||
|
|
@ -95,7 +105,8 @@ public class FSMDocTestBase {
|
|||
}
|
||||
|
||||
protected ActorRef getTarget() {
|
||||
if (target == null) throw new IllegalStateException("getTarget(): not yet initialized");
|
||||
if (target == null)
|
||||
throw new IllegalStateException("getTarget(): not yet initialized");
|
||||
return target;
|
||||
}
|
||||
|
||||
|
|
@ -104,6 +115,7 @@ public class FSMDocTestBase {
|
|||
*/
|
||||
abstract protected void transition(State old, State next);
|
||||
}
|
||||
|
||||
//#base
|
||||
|
||||
//#actor
|
||||
|
|
@ -119,14 +131,16 @@ public class FSMDocTestBase {
|
|||
if (o instanceof SetTarget)
|
||||
init(((SetTarget) o).ref);
|
||||
|
||||
else whenUnhandled(o);
|
||||
else
|
||||
whenUnhandled(o);
|
||||
|
||||
} else if (getState() == State.ACTIVE) {
|
||||
|
||||
if (o == flush)
|
||||
setState(State.IDLE);
|
||||
|
||||
else whenUnhandled(o);
|
||||
else
|
||||
whenUnhandled(o);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -147,9 +161,15 @@ public class FSMDocTestBase {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#actor
|
||||
|
||||
ActorSystem system = ActorSystem.create();
|
||||
ActorSystem system;
|
||||
|
||||
@org.junit.Before
|
||||
public void setUp() {
|
||||
system = ActorSystem.create("FSMSystem", AkkaSpec.testConf());
|
||||
}
|
||||
|
||||
@org.junit.Test
|
||||
public void mustBunch() {
|
||||
|
|
|
|||
|
|
@ -216,6 +216,7 @@ public class UntypedActorDocTestBase {
|
|||
class Result {
|
||||
final int x;
|
||||
final String s;
|
||||
|
||||
public Result(int x, String s) {
|
||||
this.x = x;
|
||||
this.s = s;
|
||||
|
|
@ -248,6 +249,7 @@ public class UntypedActorDocTestBase {
|
|||
|
||||
pipeTo(transformed, actorC);
|
||||
//#ask-pipeTo
|
||||
system.shutdown();
|
||||
}
|
||||
|
||||
public static class MyActor extends UntypedActor {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,474 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.docs.actor.japi;
|
||||
|
||||
//#all
|
||||
//#imports
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import akka.actor.*;
|
||||
import akka.dispatch.Mapper;
|
||||
import akka.japi.Function;
|
||||
import akka.util.Duration;
|
||||
import akka.util.Timeout;
|
||||
import akka.event.Logging;
|
||||
import akka.event.LoggingAdapter;
|
||||
import com.typesafe.config.Config;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
|
||||
import static akka.japi.Util.manifest;
|
||||
|
||||
import static akka.actor.SupervisorStrategy.*;
|
||||
import static akka.pattern.Patterns.ask;
|
||||
import static akka.pattern.Patterns.pipeTo;
|
||||
|
||||
import static akka.docs.actor.japi.FaultHandlingDocSample.WorkerApi.*;
|
||||
import static akka.docs.actor.japi.FaultHandlingDocSample.CounterServiceApi.*;
|
||||
import static akka.docs.actor.japi.FaultHandlingDocSample.CounterApi.*;
|
||||
import static akka.docs.actor.japi.FaultHandlingDocSample.StorageApi.*;
|
||||
|
||||
//#imports
|
||||
|
||||
public class FaultHandlingDocSample {
|
||||
|
||||
/**
|
||||
* Runs the sample
|
||||
*/
|
||||
public static void main(String[] args) {
|
||||
Config config = ConfigFactory.parseString("akka.loglevel = DEBUG \n" + "akka.actor.debug.lifecycle = on");
|
||||
|
||||
ActorSystem system = ActorSystem.create("FaultToleranceSample", config);
|
||||
ActorRef worker = system.actorOf(new Props(Worker.class), "worker");
|
||||
ActorRef listener = system.actorOf(new Props(Listener.class), "listener");
|
||||
// start the work and listen on progress
|
||||
// note that the listener is used as sender of the tell,
|
||||
// i.e. it will receive replies from the worker
|
||||
worker.tell(Start, listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Listens on progress from the worker and shuts down the system when enough
|
||||
* work has been done.
|
||||
*/
|
||||
public static class Listener extends UntypedActor {
|
||||
final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
// If we don't get any progress within 15 seconds then the service is unavailable
|
||||
getContext().setReceiveTimeout(Duration.parse("15 seconds"));
|
||||
}
|
||||
|
||||
public void onReceive(Object msg) {
|
||||
log.debug("received message {}", msg);
|
||||
if (msg instanceof Progress) {
|
||||
Progress progress = (Progress) msg;
|
||||
log.info("Current progress: {} %", progress.percent);
|
||||
if (progress.percent >= 100.0) {
|
||||
log.info("That's all, shutting down");
|
||||
getContext().system().shutdown();
|
||||
}
|
||||
} else if (msg == Actors.receiveTimeout()) {
|
||||
// No progress within 15 seconds, ServiceUnavailable
|
||||
log.error("Shutting down due to unavailable service");
|
||||
getContext().system().shutdown();
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
public interface WorkerApi {
|
||||
public static final Object Start = "Start";
|
||||
public static final Object Do = "Do";
|
||||
|
||||
public static class Progress {
|
||||
public final double percent;
|
||||
|
||||
public Progress(double percent) {
|
||||
this.percent = percent;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s)", getClass().getSimpleName(), percent);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Worker performs some work when it receives the Start message. It will
|
||||
* continuously notify the sender of the Start message of current Progress.
|
||||
* The Worker supervise the CounterService.
|
||||
*/
|
||||
public static class Worker extends UntypedActor {
|
||||
final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
final Timeout askTimeout = new Timeout(Duration.parse("5 seconds"));
|
||||
|
||||
// The sender of the initial Start message will continuously be notified about progress
|
||||
ActorRef progressListener;
|
||||
final ActorRef counterService = getContext().actorOf(new Props(CounterService.class), "counter");
|
||||
final int totalCount = 51;
|
||||
|
||||
// Stop the CounterService child if it throws ServiceUnavailable
|
||||
private static SupervisorStrategy strategy = new OneForOneStrategy(-1, Duration.Inf(),
|
||||
new Function<Throwable, Action>() {
|
||||
@Override
|
||||
public Action apply(Throwable t) {
|
||||
if (t instanceof ServiceUnavailable) {
|
||||
return stop();
|
||||
} else {
|
||||
return escalate();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
@Override
|
||||
public SupervisorStrategy supervisorStrategy() {
|
||||
return strategy;
|
||||
}
|
||||
|
||||
public void onReceive(Object msg) {
|
||||
log.debug("received message {}", msg);
|
||||
if (msg.equals(Start) && progressListener == null) {
|
||||
progressListener = getSender();
|
||||
getContext().system().scheduler().schedule(Duration.Zero(), Duration.parse("1 second"), getSelf(), Do);
|
||||
} else if (msg.equals(Do)) {
|
||||
counterService.tell(new Increment(1), getSelf());
|
||||
counterService.tell(new Increment(1), getSelf());
|
||||
counterService.tell(new Increment(1), getSelf());
|
||||
|
||||
// Send current progress to the initial sender
|
||||
pipeTo(ask(counterService, GetCurrentCount, askTimeout)
|
||||
.mapTo(manifest(CurrentCount.class))
|
||||
.map(new Mapper<CurrentCount, Progress>() {
|
||||
public Progress apply(CurrentCount c) {
|
||||
return new Progress(100.0 * c.count / totalCount);
|
||||
}
|
||||
}), progressListener);
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
public interface CounterServiceApi {
|
||||
|
||||
public static final Object GetCurrentCount = "GetCurrentCount";
|
||||
|
||||
public static class CurrentCount {
|
||||
public final String key;
|
||||
public final long count;
|
||||
|
||||
public CurrentCount(String key, long count) {
|
||||
this.key = key;
|
||||
this.count = count;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s, %s)", getClass().getSimpleName(), key, count);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Increment {
|
||||
public final long n;
|
||||
|
||||
public Increment(long n) {
|
||||
this.n = n;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s)", getClass().getSimpleName(), n);
|
||||
}
|
||||
}
|
||||
|
||||
public static class ServiceUnavailable extends RuntimeException {
|
||||
public ServiceUnavailable(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Adds the value received in Increment message to a persistent counter.
|
||||
* Replies with CurrentCount when it is asked for CurrentCount. CounterService
|
||||
* supervise Storage and Counter.
|
||||
*/
|
||||
public static class CounterService extends UntypedActor {
|
||||
|
||||
// Reconnect message
|
||||
static final Object Reconnect = "Reconnect";
|
||||
|
||||
private static class SenderMsgPair {
|
||||
final ActorRef sender;
|
||||
final Object msg;
|
||||
|
||||
SenderMsgPair(ActorRef sender, Object msg) {
|
||||
this.msg = msg;
|
||||
this.sender = sender;
|
||||
}
|
||||
}
|
||||
|
||||
final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
final String key = getSelf().path().name();
|
||||
ActorRef storage;
|
||||
ActorRef counter;
|
||||
final List<SenderMsgPair> backlog = new ArrayList<SenderMsgPair>();
|
||||
final int MAX_BACKLOG = 10000;
|
||||
|
||||
// Restart the storage child when StorageException is thrown.
|
||||
// After 3 restarts within 5 seconds it will be stopped.
|
||||
private static SupervisorStrategy strategy = new OneForOneStrategy(3, Duration.parse("5 seconds"),
|
||||
new Function<Throwable, Action>() {
|
||||
@Override
|
||||
public Action apply(Throwable t) {
|
||||
if (t instanceof StorageException) {
|
||||
return restart();
|
||||
} else {
|
||||
return escalate();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
@Override
|
||||
public SupervisorStrategy supervisorStrategy() {
|
||||
return strategy;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
initStorage();
|
||||
}
|
||||
|
||||
/**
|
||||
* The child storage is restarted in case of failure, but after 3 restarts,
|
||||
* and still failing it will be stopped. Better to back-off than
|
||||
* continuously failing. When it has been stopped we will schedule a
|
||||
* Reconnect after a delay. Watch the child so we receive Terminated message
|
||||
* when it has been terminated.
|
||||
*/
|
||||
void initStorage() {
|
||||
storage = getContext().watch(getContext().actorOf(new Props(Storage.class), "storage"));
|
||||
// Tell the counter, if any, to use the new storage
|
||||
if (counter != null)
|
||||
counter.tell(new UseStorage(storage), getSelf());
|
||||
// We need the initial value to be able to operate
|
||||
storage.tell(new Get(key), getSelf());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceive(Object msg) {
|
||||
log.debug("received message {}", msg);
|
||||
if (msg instanceof Entry && ((Entry) msg).key.equals(key) && counter == null) {
|
||||
// Reply from Storage of the initial value, now we can create the Counter
|
||||
final long value = ((Entry) msg).value;
|
||||
counter = getContext().actorOf(new Props().withCreator(new UntypedActorFactory() {
|
||||
public Actor create() {
|
||||
return new Counter(key, value);
|
||||
}
|
||||
}));
|
||||
// Tell the counter to use current storage
|
||||
counter.tell(new UseStorage(storage), getSelf());
|
||||
// and send the buffered backlog to the counter
|
||||
for (SenderMsgPair each : backlog) {
|
||||
counter.tell(each.msg, each.sender);
|
||||
}
|
||||
backlog.clear();
|
||||
} else if (msg instanceof Increment) {
|
||||
forwardOrPlaceInBacklog(msg);
|
||||
} else if (msg.equals(GetCurrentCount)) {
|
||||
forwardOrPlaceInBacklog(msg);
|
||||
} else if (msg instanceof Terminated) {
|
||||
// After 3 restarts the storage child is stopped.
|
||||
// We receive Terminated because we watch the child, see initStorage.
|
||||
storage = null;
|
||||
// Tell the counter that there is no storage for the moment
|
||||
counter.tell(new UseStorage(null), getSelf());
|
||||
// Try to re-establish storage after while
|
||||
getContext().system().scheduler().scheduleOnce(Duration.parse("10 seconds"), getSelf(), Reconnect);
|
||||
} else if (msg.equals(Reconnect)) {
|
||||
// Re-establish storage after the scheduled delay
|
||||
initStorage();
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
|
||||
void forwardOrPlaceInBacklog(Object msg) {
|
||||
// We need the initial value from storage before we can start delegate to the counter.
|
||||
// Before that we place the messages in a backlog, to be sent to the counter when
|
||||
// it is initialized.
|
||||
if (counter == null) {
|
||||
if (backlog.size() >= MAX_BACKLOG)
|
||||
throw new ServiceUnavailable("CounterService not available, lack of initial value");
|
||||
backlog.add(new SenderMsgPair(getSender(), msg));
|
||||
} else {
|
||||
counter.forward(msg, getContext());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
public interface CounterApi {
|
||||
public static class UseStorage {
|
||||
public final ActorRef storage;
|
||||
|
||||
public UseStorage(ActorRef storage) {
|
||||
this.storage = storage;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s)", getClass().getSimpleName(), storage);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* The in memory count variable that will send current value to the Storage,
|
||||
* if there is any storage available at the moment.
|
||||
*/
|
||||
public static class Counter extends UntypedActor {
|
||||
final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
final String key;
|
||||
long count;
|
||||
ActorRef storage;
|
||||
|
||||
public Counter(String key, long initialValue) {
|
||||
this.key = key;
|
||||
this.count = initialValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReceive(Object msg) {
|
||||
log.debug("received message {}", msg);
|
||||
if (msg instanceof UseStorage) {
|
||||
storage = ((UseStorage) msg).storage;
|
||||
storeCount();
|
||||
} else if (msg instanceof Increment) {
|
||||
count += ((Increment) msg).n;
|
||||
storeCount();
|
||||
} else if (msg.equals(GetCurrentCount)) {
|
||||
getSender().tell(new CurrentCount(key, count), getSelf());
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
|
||||
void storeCount() {
|
||||
// Delegate dangerous work, to protect our valuable state.
|
||||
// We can continue without storage.
|
||||
if (storage != null) {
|
||||
storage.tell(new Store(new Entry(key, count)), getSelf());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
public interface StorageApi {
|
||||
|
||||
public static class Store {
|
||||
public final Entry entry;
|
||||
|
||||
public Store(Entry entry) {
|
||||
this.entry = entry;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s)", getClass().getSimpleName(), entry);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Entry {
|
||||
public final String key;
|
||||
public final long value;
|
||||
|
||||
public Entry(String key, long value) {
|
||||
this.key = key;
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s, %s)", getClass().getSimpleName(), key, value);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Get {
|
||||
public final String key;
|
||||
|
||||
public Get(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return String.format("%s(%s)", getClass().getSimpleName(), key);
|
||||
}
|
||||
}
|
||||
|
||||
public static class StorageException extends RuntimeException {
|
||||
public StorageException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Saves key/value pairs to persistent storage when receiving Store message.
|
||||
* Replies with current value when receiving Get message. Will throw
|
||||
* StorageException if the underlying data store is out of order.
|
||||
*/
|
||||
public static class Storage extends UntypedActor {
|
||||
|
||||
final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
final DummyDB db = DummyDB.instance;
|
||||
|
||||
@Override
|
||||
public void onReceive(Object msg) {
|
||||
log.debug("received message {}", msg);
|
||||
if (msg instanceof Store) {
|
||||
Store store = (Store) msg;
|
||||
db.save(store.entry.key, store.entry.value);
|
||||
} else if (msg instanceof Get) {
|
||||
Get get = (Get) msg;
|
||||
Long value = db.load(get.key);
|
||||
getSender().tell(new Entry(get.key, value == null ? Long.valueOf(0L) : value), getSelf());
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#dummydb
|
||||
public static class DummyDB {
|
||||
public static final DummyDB instance = new DummyDB();
|
||||
private final Map<String, Long> db = new HashMap<String, Long>();
|
||||
|
||||
private DummyDB() {
|
||||
}
|
||||
|
||||
public synchronized void save(String key, Long value) throws StorageException {
|
||||
if (11 <= value && value <= 14)
|
||||
throw new StorageException("Simulated store failure " + value);
|
||||
db.put(key, value);
|
||||
}
|
||||
|
||||
public synchronized Long load(String key) throws StorageException {
|
||||
return db.get(key);
|
||||
}
|
||||
}
|
||||
//#dummydb
|
||||
}
|
||||
//#all
|
||||
7
akka-docs/java/fault-tolerance-sample.rst
Normal file
7
akka-docs/java/fault-tolerance-sample.rst
Normal file
|
|
@ -0,0 +1,7 @@
|
|||
.. _fault-tolerance-sample-java:
|
||||
|
||||
Full Source Code of the Fault Tolerance Sample (Java)
|
||||
------------------------------------------------------
|
||||
|
||||
.. includecode:: code/akka/docs/actor/japi/FaultHandlingDocSample.java#all
|
||||
|
||||
|
|
@ -12,9 +12,31 @@ children, and as such each actor defines fault handling supervisor strategy.
|
|||
This strategy cannot be changed afterwards as it is an integral part of the
|
||||
actor system’s structure.
|
||||
|
||||
Fault Handling in Practice
|
||||
--------------------------
|
||||
|
||||
First, let us look at a sample that illustrates one way to handle data store errors,
|
||||
which is a typical source of failure in real world applications. Of course it depends
|
||||
on the actual application what is possible to do when the data store is unavailable,
|
||||
but in this sample we use a best effort re-connect approach.
|
||||
|
||||
Read the following source code. The inlined comments explain the different pieces of
|
||||
the fault handling and why they are added. It is also highly recommended to run this
|
||||
sample as it is easy to follow the log output to understand what is happening in runtime.
|
||||
|
||||
.. toctree::
|
||||
|
||||
fault-tolerance-sample
|
||||
|
||||
.. includecode:: code/akka/docs/actor/japi/FaultHandlingDocSample.java#all
|
||||
:exclude: imports,messages,dummydb
|
||||
|
||||
Creating a Supervisor Strategy
|
||||
------------------------------
|
||||
|
||||
The following sections explain the fault handling mechanism and alternatives
|
||||
in more depth.
|
||||
|
||||
For the sake of demonstration let us consider the following strategy:
|
||||
|
||||
.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java
|
||||
|
|
@ -26,12 +48,28 @@ First off, it is a one-for-one strategy, meaning that each child is treated
|
|||
separately (an all-for-one strategy works very similarly, the only difference
|
||||
is that any decision is applied to all children of the supervisor, not only the
|
||||
failing one). There are limits set on the restart frequency, namely maximum 10
|
||||
restarts per minute; each of these settings could be left out, which means
|
||||
that the respective limit does not apply, leaving the possibility to specify an
|
||||
absolute upper limit on the restarts or to make the restarts work infinitely.
|
||||
restarts per minute. ``-1`` and ``Duration.Inf()`` means that the respective limit
|
||||
does not apply, leaving the possibility to specify an absolute upper limit on the
|
||||
restarts or to make the restarts work infinitely.
|
||||
|
||||
Practical Application
|
||||
---------------------
|
||||
Default Supervisor Strategy
|
||||
---------------------------
|
||||
|
||||
``Escalate`` is used if the defined strategy doesn't cover the exception that was thrown.
|
||||
|
||||
When the supervisor strategy is not defined for an actor the following
|
||||
exceptions are handled by default:
|
||||
|
||||
* ``ActorInitializationException`` will stop the failing child actor
|
||||
* ``ActorKilledException`` will stop the failing child actor
|
||||
* ``Exception`` will restart the failing child actor
|
||||
* Other types of ``Throwable`` will be escalated to parent actor
|
||||
|
||||
If the exception escalate all the way up to the root guardian it will handle it
|
||||
in the same way as the default strategy defined above.
|
||||
|
||||
Test Application
|
||||
----------------
|
||||
|
||||
The following section shows the effects of the different actions in practice,
|
||||
wherefor a test setup is needed. First off, we need a suitable supervisor:
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ command (on a unix-based system):
|
|||
|
||||
.. code-block:: none
|
||||
|
||||
bin/akka sample.kernel.hello.HelloKernel
|
||||
bin/start sample.kernel.hello.HelloKernel
|
||||
|
||||
Use ``Ctrl-C`` to interrupt and exit the microkernel.
|
||||
|
||||
|
|
@ -34,3 +34,34 @@ The code for the Hello Kernel example (see the ``HelloKernel`` class for an exam
|
|||
of creating a Bootable):
|
||||
|
||||
.. includecode:: ../../akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala
|
||||
|
||||
|
||||
Distribution of microkernel application
|
||||
---------------------------------------
|
||||
|
||||
To make a distribution package of the microkernel and your application the ``akka-sbt-plugin`` provides
|
||||
``AkkaKernelPlugin``. It creates the directory structure, with jar files, configuration files and
|
||||
start scripts.
|
||||
|
||||
To use the sbt plugin you define it in your ``project/plugins.sbt``:
|
||||
|
||||
.. includecode:: ../../akka-sbt-plugin/sample/project/plugins.sbt
|
||||
|
||||
Then you add it to the settings of your ``project/Build.scala``. It is also important that you add the ``akka-kernel`` dependency.
|
||||
This is an example of a complete sbt build file:
|
||||
|
||||
.. includecode:: ../../akka-sbt-plugin/sample/project/Build.scala
|
||||
|
||||
Run the plugin with sbt::
|
||||
|
||||
> dist
|
||||
> dist:clean
|
||||
|
||||
There are several settings that can be defined:
|
||||
|
||||
* ``outputDirectory`` - destination directory of the package, default ``target/dist``
|
||||
* ``distJvmOptions`` - JVM parameters to be used in the start script
|
||||
* ``configSourceDirs`` - Configuration files are copied from these directories, default ``src/config``, ``src/main/config``, ``src/main/resources``
|
||||
* ``distMainClass`` - Kernel main class to use in start script
|
||||
* ``libFilter`` - Filter of dependency jar files
|
||||
* ``additionalLibs`` - Additional dependency jar files
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package akka.docs.actor
|
||||
|
||||
//#all
|
||||
//#imports
|
||||
import akka.actor._
|
||||
import akka.actor.SupervisorStrategy._
|
||||
import akka.util.duration._
|
||||
|
|
@ -12,13 +13,13 @@ import akka.util.Timeout
|
|||
import akka.event.LoggingReceive
|
||||
import akka.pattern.ask
|
||||
import com.typesafe.config.ConfigFactory
|
||||
//#imports
|
||||
|
||||
/**
|
||||
* Runs the sample
|
||||
*/
|
||||
object FaultHandlingDocSample extends App {
|
||||
import Worker._
|
||||
import CounterService._
|
||||
|
||||
val config = ConfigFactory.parseString("""
|
||||
akka.loglevel = DEBUG
|
||||
|
|
@ -30,45 +31,53 @@ object FaultHandlingDocSample extends App {
|
|||
|
||||
val system = ActorSystem("FaultToleranceSample", config)
|
||||
val worker = system.actorOf(Props[Worker], name = "worker")
|
||||
val listener = system.actorOf(Props[Listener], name = "listener")
|
||||
// start the work and listen on progress
|
||||
// note that the listener is used as sender of the tell,
|
||||
// i.e. it will receive replies from the worker
|
||||
worker.tell(Start, sender = listener)
|
||||
}
|
||||
|
||||
// Create an Actor that start the work and listens to progress
|
||||
system.actorOf(Props(new Actor with ActorLogging {
|
||||
/**
|
||||
* Listens on progress from the worker and shuts down the system when enough
|
||||
* work has been done.
|
||||
*/
|
||||
class Listener extends Actor with ActorLogging {
|
||||
import Worker._
|
||||
// If we don't get any progress within 15 seconds then the service is unavailable
|
||||
context.setReceiveTimeout(15 seconds)
|
||||
worker ! Start
|
||||
|
||||
def receive = {
|
||||
case CurrentCount(key, count) ⇒
|
||||
log.info("Current count for [{}] is [{}]", key, count)
|
||||
if (count > 50) {
|
||||
log.info("That's enough, shutting down")
|
||||
system.shutdown()
|
||||
case Progress(percent) ⇒
|
||||
log.info("Current progress: {} %", percent)
|
||||
if (percent >= 100.0) {
|
||||
log.info("That's all, shutting down")
|
||||
context.system.shutdown()
|
||||
}
|
||||
|
||||
case ReceiveTimeout ⇒
|
||||
// No progress within 15 seconds, ServiceUnavailable
|
||||
log.error("Shutting down due to unavailable service")
|
||||
system.shutdown()
|
||||
context.system.shutdown()
|
||||
}
|
||||
}))
|
||||
|
||||
}
|
||||
|
||||
//#messages
|
||||
object Worker {
|
||||
// Messages
|
||||
case object Start
|
||||
case object Do
|
||||
case class Progress(percent: Double)
|
||||
}
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Worker performs some work when it receives the `Start` message.
|
||||
* It will continuously notify the sender of the `Start` message
|
||||
* of current progress. The `Worker` supervise the `CounterService`.
|
||||
* of current ``Progress``. The `Worker` supervise the `CounterService`.
|
||||
*/
|
||||
class Worker extends Actor with ActorLogging {
|
||||
import Worker._
|
||||
import CounterService._
|
||||
implicit def system = context.system
|
||||
implicit val askTimeout = Timeout(5 seconds)
|
||||
|
||||
// Stop the CounterService child if it throws ServiceUnavailable
|
||||
|
|
@ -79,8 +88,9 @@ class Worker extends Actor with ActorLogging {
|
|||
// The sender of the initial Start message will continuously be notified about progress
|
||||
var progressListener: Option[ActorRef] = None
|
||||
val counterService = context.actorOf(Props[CounterService], name = "counter")
|
||||
val totalCount = 51
|
||||
|
||||
def receive = LoggingReceive(this) {
|
||||
def receive = LoggingReceive {
|
||||
case Start if progressListener.isEmpty ⇒
|
||||
progressListener = Some(sender)
|
||||
context.system.scheduler.schedule(Duration.Zero, 1 second, self, Do)
|
||||
|
|
@ -90,19 +100,23 @@ class Worker extends Actor with ActorLogging {
|
|||
counterService ! Increment(1)
|
||||
counterService ! Increment(1)
|
||||
|
||||
// Send current count to the initial sender
|
||||
counterService ? GetCurrentCount pipeTo progressListener.get
|
||||
// Send current progress to the initial sender
|
||||
counterService ? GetCurrentCount map {
|
||||
case CurrentCount(_, count) ⇒ Progress(100.0 * count / totalCount)
|
||||
} pipeTo progressListener.get
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
object CounterService {
|
||||
// Messages
|
||||
case class Increment(n: Int)
|
||||
case object GetCurrentCount
|
||||
case class CurrentCount(key: String, count: Long)
|
||||
case object Reconnect
|
||||
class ServiceUnavailable(msg: String) extends RuntimeException(msg)
|
||||
|
||||
private case object Reconnect
|
||||
}
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Adds the value received in `Increment` message to a persistent
|
||||
|
|
@ -113,7 +127,6 @@ class CounterService extends Actor {
|
|||
import CounterService._
|
||||
import Counter._
|
||||
import Storage._
|
||||
implicit def system = context.system
|
||||
|
||||
// Restart the storage child when StorageException is thrown.
|
||||
// After 3 restarts within 5 seconds it will be stopped.
|
||||
|
|
@ -121,10 +134,10 @@ class CounterService extends Actor {
|
|||
case _: Storage.StorageException ⇒ Restart
|
||||
}
|
||||
|
||||
val key = context.self.path.name
|
||||
val key = self.path.name
|
||||
var storage: Option[ActorRef] = None
|
||||
var counter: Option[ActorRef] = None
|
||||
var backlog = IndexedSeq.empty[Any]
|
||||
var backlog = IndexedSeq.empty[(ActorRef, Any)]
|
||||
val MaxBacklog = 10000
|
||||
|
||||
override def preStart() {
|
||||
|
|
@ -145,7 +158,7 @@ class CounterService extends Actor {
|
|||
storage.get ! Get(key)
|
||||
}
|
||||
|
||||
def receive = LoggingReceive(this) {
|
||||
def receive = LoggingReceive {
|
||||
|
||||
case Entry(k, v) if k == key && counter == None ⇒
|
||||
// Reply from Storage of the initial value, now we can create the Counter
|
||||
|
|
@ -154,7 +167,7 @@ class CounterService extends Actor {
|
|||
// Tell the counter to use current storage
|
||||
c ! UseStorage(storage)
|
||||
// and send the buffered backlog to the counter
|
||||
backlog foreach { c ! _ }
|
||||
for ((replyTo, msg) ← backlog) c.tell(msg, sender = replyTo)
|
||||
backlog = IndexedSeq.empty
|
||||
|
||||
case msg @ Increment(n) ⇒ forwardOrPlaceInBacklog(msg)
|
||||
|
|
@ -184,16 +197,17 @@ class CounterService extends Actor {
|
|||
case None ⇒
|
||||
if (backlog.size >= MaxBacklog)
|
||||
throw new ServiceUnavailable("CounterService not available, lack of initial value")
|
||||
backlog = backlog :+ msg
|
||||
backlog = backlog :+ (sender, msg)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
//#messages
|
||||
object Counter {
|
||||
// Messages
|
||||
case class UseStorage(storage: Option[ActorRef])
|
||||
}
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* The in memory count variable that will send current
|
||||
|
|
@ -204,12 +218,11 @@ class Counter(key: String, initialValue: Long) extends Actor {
|
|||
import Counter._
|
||||
import CounterService._
|
||||
import Storage._
|
||||
implicit def system = context.system
|
||||
|
||||
var count = initialValue
|
||||
var storage: Option[ActorRef] = None
|
||||
|
||||
def receive = LoggingReceive(this) {
|
||||
def receive = LoggingReceive {
|
||||
case UseStorage(s) ⇒
|
||||
storage = s
|
||||
storeCount()
|
||||
|
|
@ -231,13 +244,14 @@ class Counter(key: String, initialValue: Long) extends Actor {
|
|||
|
||||
}
|
||||
|
||||
//#messages
|
||||
object Storage {
|
||||
// Messages
|
||||
case class Store(entry: Entry)
|
||||
case class Get(key: String)
|
||||
case class Entry(key: String, value: Long)
|
||||
class StorageException(msg: String) extends RuntimeException(msg)
|
||||
}
|
||||
//#messages
|
||||
|
||||
/**
|
||||
* Saves key/value pairs to persistent storage when receiving `Store` message.
|
||||
|
|
@ -246,19 +260,19 @@ object Storage {
|
|||
*/
|
||||
class Storage extends Actor {
|
||||
import Storage._
|
||||
implicit def system = context.system
|
||||
|
||||
val db = DummyDB
|
||||
|
||||
def receive = LoggingReceive(this) {
|
||||
def receive = LoggingReceive {
|
||||
case Store(Entry(key, count)) ⇒ db.save(key, count)
|
||||
case Get(key) ⇒ sender ! Entry(key, db.load(key).getOrElse(0L))
|
||||
}
|
||||
}
|
||||
|
||||
//#dummydb
|
||||
object DummyDB {
|
||||
import Storage.StorageException
|
||||
var db = Map[String, Long]()
|
||||
private var db = Map[String, Long]()
|
||||
|
||||
@throws(classOf[StorageException])
|
||||
def save(key: String, value: Long): Unit = synchronized {
|
||||
|
|
@ -271,4 +285,5 @@ object DummyDB {
|
|||
db.get(key)
|
||||
}
|
||||
}
|
||||
//#dummydb
|
||||
//#all
|
||||
|
|
|
|||
|
|
@ -61,8 +61,7 @@ object TestkitDocSpec {
|
|||
class LoggingActor extends Actor {
|
||||
//#logging-receive
|
||||
import akka.event.LoggingReceive
|
||||
implicit def system = context.system
|
||||
def receive = LoggingReceive(this) {
|
||||
def receive = LoggingReceive {
|
||||
case msg ⇒ // Do something...
|
||||
}
|
||||
//#logging-receive
|
||||
|
|
|
|||
7
akka-docs/scala/fault-tolerance-sample.rst
Normal file
7
akka-docs/scala/fault-tolerance-sample.rst
Normal file
|
|
@ -0,0 +1,7 @@
|
|||
.. _fault-tolerance-sample-scala:
|
||||
|
||||
Full Source Code of the Fault Tolerance Sample (Scala)
|
||||
------------------------------------------------------
|
||||
|
||||
.. includecode:: code/akka/docs/actor/FaultHandlingDocSample.scala#all
|
||||
|
||||
|
|
@ -24,7 +24,12 @@ Read the following source code. The inlined comments explain the different piece
|
|||
the fault handling and why they are added. It is also highly recommended to run this
|
||||
sample as it is easy to follow the log output to understand what is happening in runtime.
|
||||
|
||||
.. toctree::
|
||||
|
||||
fault-tolerance-sample
|
||||
|
||||
.. includecode:: code/akka/docs/actor/FaultHandlingDocSample.scala#all
|
||||
:exclude: imports,messages,dummydb
|
||||
|
||||
Creating a Supervisor Strategy
|
||||
------------------------------
|
||||
|
|
@ -43,7 +48,7 @@ First off, it is a one-for-one strategy, meaning that each child is treated
|
|||
separately (an all-for-one strategy works very similarly, the only difference
|
||||
is that any decision is applied to all children of the supervisor, not only the
|
||||
failing one). There are limits set on the restart frequency, namely maximum 10
|
||||
restarts per minute; each of these settings defaults to ``None`` which means
|
||||
restarts per minute; each of these settings could be left out, which means
|
||||
that the respective limit does not apply, leaving the possibility to specify an
|
||||
absolute upper limit on the restarts or to make the restarts work infinitely.
|
||||
|
||||
|
|
@ -57,14 +62,12 @@ Default Supervisor Strategy
|
|||
``Escalate`` is used if the defined strategy doesn't cover the exception that was thrown.
|
||||
|
||||
When the supervisor strategy is not defined for an actor the following
|
||||
exceptions are handled by default::
|
||||
exceptions are handled by default:
|
||||
|
||||
OneForOneStrategy() {
|
||||
case _: ActorInitializationException ⇒ Stop
|
||||
case _: ActorKilledException ⇒ Stop
|
||||
case _: Exception ⇒ Restart
|
||||
case _ ⇒ Escalate
|
||||
}
|
||||
* ``ActorInitializationException`` will stop the failing child actor
|
||||
* ``ActorKilledException`` will stop the failing child actor
|
||||
* ``Exception`` will restart the failing child actor
|
||||
* Other types of ``Throwable`` will be escalated to parent actor
|
||||
|
||||
If the exception escalate all the way up to the root guardian it will handle it
|
||||
in the same way as the default strategy defined above.
|
||||
|
|
|
|||
|
|
@ -600,9 +600,6 @@ options:
|
|||
.. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#logging-receive
|
||||
|
||||
.
|
||||
The first argument to :meth:`LoggingReceive` defines the source to be used in the
|
||||
logging events, which should be the current actor.
|
||||
|
||||
If the abovementioned setting is not given in the :ref:`configuration`, this method will
|
||||
pass through the given :class:`Receive` function unmodified, meaning that
|
||||
there is no runtime cost unless actually enabled.
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.actor.mailbox
|
||||
|
||||
import akka.actor.{ ActorContext, ActorRef }
|
||||
import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem }
|
||||
import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox }
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol }
|
||||
|
|
@ -15,7 +15,7 @@ private[akka] object DurableExecutableMailboxConfig {
|
|||
abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue {
|
||||
import DurableExecutableMailboxConfig._
|
||||
|
||||
def system = owner.system
|
||||
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
|
||||
def ownerPath = owner.self.path
|
||||
val ownerPathString = ownerPath.elements.mkString("/")
|
||||
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
|
||||
|
|
@ -44,7 +44,7 @@ trait DurableMessageSerialization {
|
|||
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = owner.system.actorFor(refProtocol.getPath)
|
||||
|
||||
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
|
||||
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage)
|
||||
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage, getClass.getClassLoader)
|
||||
val sender = deserializeActorRef(durableMessage.getSender)
|
||||
|
||||
new Envelope(message, sender)(owner.system)
|
||||
|
|
|
|||
|
|
@ -16,11 +16,9 @@ import org.bson.DefaultBSONSerializer
|
|||
|
||||
import akka.remote.RemoteProtocol.MessageProtocol
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
||||
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
val systemImpl = system.asInstanceOf[ActorSystemImpl]
|
||||
class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = {
|
||||
|
||||
|
|
@ -67,10 +65,10 @@ class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObjec
|
|||
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
|
||||
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
||||
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
|
||||
val msg = MessageSerializer.deserialize(system, msgData)
|
||||
val msg = MessageSerializer.deserialize(system, msgData, system.internalClassLoader)
|
||||
val ownerPath = doc.as[String]("ownerPath")
|
||||
val senderPath = doc.as[String]("senderPath")
|
||||
val sender = systemImpl.actorFor(senderPath)
|
||||
val sender = system.actorFor(senderPath)
|
||||
|
||||
MongoDurableMessage(ownerPath, msg, sender)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,33 +38,86 @@ akka {
|
|||
|
||||
remote {
|
||||
|
||||
# Which implementation of akka.remote.RemoteSupport to use
|
||||
# Which implementation of akka.remote.RemoteTransport to use
|
||||
# default is a TCP-based remote transport based on Netty
|
||||
transport = "akka.remote.netty.NettyRemoteSupport"
|
||||
transport = "akka.remote.netty.NettyRemoteTransport"
|
||||
|
||||
# In case of increased latency / overflow how long
|
||||
# should we wait (blocking the sender) until we deem the send to be cancelled?
|
||||
# 0 means "never backoff", any positive number will indicate time to block at most.
|
||||
backoff-timeout = 0ms
|
||||
|
||||
use-compression = off
|
||||
|
||||
# Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
|
||||
# or using 'akka.util.Crypt.generateSecureCookie'
|
||||
secure-cookie = ""
|
||||
# Enable untrusted mode for full security of server managed actors, allows
|
||||
# untrusted clients to connect.
|
||||
untrusted-mode = off
|
||||
|
||||
# Timeout for ACK of cluster operations, lik checking actor out etc.
|
||||
remote-daemon-ack-timeout = 30s
|
||||
|
||||
# Reuse inbound connections for outbound messages
|
||||
use-passive-connections = on
|
||||
|
||||
# If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged
|
||||
log-received-messages = off
|
||||
|
||||
# If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged
|
||||
log-sent-messages = off
|
||||
|
||||
# Each property is annotated with (I) or (O) or (I&O), where I stands for “inbound” and O for “outbound” connections.
|
||||
# The NettyRemoteTransport always starts the server role to allow inbound connections, and it starts
|
||||
# active client connections whenever sending to a destination which is not yet connected; if configured
|
||||
# it reuses inbound connections for replies, which is called a passive client connection (i.e. from server
|
||||
# to client).
|
||||
netty {
|
||||
|
||||
# (O) In case of increased latency / overflow how long
|
||||
# should we wait (blocking the sender) until we deem the send to be cancelled?
|
||||
# 0 means "never backoff", any positive number will indicate time to block at most.
|
||||
backoff-timeout = 0ms
|
||||
|
||||
# (I&O) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
|
||||
# or using 'akka.util.Crypt.generateSecureCookie'
|
||||
secure-cookie = ""
|
||||
|
||||
# (I) Should the remote server require that it peers share the same secure-cookie
|
||||
# (defined in the 'remote' section)?
|
||||
require-cookie = off
|
||||
|
||||
# (I) Reuse inbound connections for outbound messages
|
||||
use-passive-connections = on
|
||||
|
||||
# (I) The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# (I) The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA), use 0 if you want a random available port
|
||||
port = 2552
|
||||
|
||||
# (I&O) Increase this if you want to be able to send messages with large payloads
|
||||
message-frame-size = 1 MiB
|
||||
|
||||
# (O) Timeout duration
|
||||
connection-timeout = 120s
|
||||
|
||||
# (I) Sets the size of the connection backlog
|
||||
backlog = 4096
|
||||
|
||||
# (I) Length in akka.time-unit how long core threads will be kept alive if idling
|
||||
execution-pool-keepalive = 60s
|
||||
|
||||
# (I) Size of the core pool of the remote execution unit
|
||||
execution-pool-size = 4
|
||||
|
||||
# (I) Maximum channel size, 0 for off
|
||||
max-channel-memory-size = 0b
|
||||
|
||||
# (I) Maximum total size of all channels, 0 for off
|
||||
max-total-memory-size = 0b
|
||||
|
||||
# (O) Time between reconnect attempts for active clients
|
||||
reconnect-delay = 5s
|
||||
|
||||
# (O) Inactivity period after which active client connection is shutdown; will be
|
||||
# re-established in case of new communication requests
|
||||
read-timeout = 3600s
|
||||
|
||||
# (O) Maximum time window that a client should try to reconnect for
|
||||
reconnection-time-window = 600s
|
||||
}
|
||||
|
||||
# accrual failure detection config
|
||||
failure-detector {
|
||||
|
||||
|
|
@ -94,58 +147,6 @@ akka {
|
|||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA), use 0 if you want a random available port
|
||||
port = 2552
|
||||
|
||||
# Increase this if you want to be able to send messages with large payloads
|
||||
message-frame-size = 1 MiB
|
||||
|
||||
# Should the remote server require that it peers share the same secure-cookie
|
||||
# (defined in the 'remote' section)?
|
||||
require-cookie = off
|
||||
|
||||
# Enable untrusted mode for full security of server managed actors, allows
|
||||
# untrusted clients to connect.
|
||||
untrusted-mode = off
|
||||
|
||||
# Sets the size of the connection backlog
|
||||
backlog = 4096
|
||||
|
||||
# Length in akka.time-unit how long core threads will be kept alive if idling
|
||||
execution-pool-keepalive = 60s
|
||||
|
||||
# Size of the core pool of the remote execution unit
|
||||
execution-pool-size = 4
|
||||
|
||||
# Maximum channel size, 0 for off
|
||||
max-channel-memory-size = 0b
|
||||
|
||||
# Maximum total size of all channels, 0 for off
|
||||
max-total-memory-size = 0b
|
||||
}
|
||||
|
||||
client {
|
||||
# Time before an attempted connection is considered failed
|
||||
connection-timeout = 10s
|
||||
|
||||
#Time between each reconnection attempt
|
||||
reconnect-delay = 5s
|
||||
|
||||
# Maximum time window that a client should try to reconnect for
|
||||
reconnection-time-window = 600s
|
||||
|
||||
#Period of time of connection inactivity to be tolerated before hanging up
|
||||
read-timeout = 3600s
|
||||
|
||||
#Max size per message
|
||||
message-frame-size = 1 MiB
|
||||
}
|
||||
}
|
||||
|
||||
cluster {
|
||||
|
|
|
|||
|
|
@ -5,11 +5,10 @@
|
|||
package akka.remote
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.collection.immutable.Map
|
||||
import scala.annotation.tailrec
|
||||
|
||||
import System.{ currentTimeMillis ⇒ newTimestamp }
|
||||
import akka.actor.{ ActorSystem, Address }
|
||||
|
||||
/**
|
||||
* Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper:
|
||||
|
|
@ -33,9 +32,9 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
*/
|
||||
private case class State(
|
||||
version: Long = 0L,
|
||||
failureStats: Map[ParsedTransportAddress, FailureStats] = Map.empty[ParsedTransportAddress, FailureStats],
|
||||
intervalHistory: Map[ParsedTransportAddress, Vector[Long]] = Map.empty[ParsedTransportAddress, Vector[Long]],
|
||||
timestamps: Map[ParsedTransportAddress, Long] = Map.empty[ParsedTransportAddress, Long])
|
||||
failureStats: Map[Address, FailureStats] = Map.empty[Address, FailureStats],
|
||||
intervalHistory: Map[Address, Vector[Long]] = Map.empty[Address, Vector[Long]],
|
||||
timestamps: Map[Address, Long] = Map.empty[Address, Long])
|
||||
|
||||
private val state = new AtomicReference[State](State())
|
||||
|
||||
|
|
@ -43,13 +42,13 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Returns true if the connection is considered to be up and healthy
|
||||
* and returns false otherwise.
|
||||
*/
|
||||
def isAvailable(connection: ParsedTransportAddress): Boolean = phi(connection) < threshold
|
||||
def isAvailable(connection: Address): Boolean = phi(connection) < threshold
|
||||
|
||||
/**
|
||||
* Records a heartbeat for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def heartbeat(connection: ParsedTransportAddress) {
|
||||
final def heartbeat(connection: Address) {
|
||||
val oldState = state.get
|
||||
|
||||
val latestTimestamp = oldState.timestamps.get(connection)
|
||||
|
|
@ -130,7 +129,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Implementations of 'Cumulative Distribution Function' for Exponential Distribution.
|
||||
* For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597].
|
||||
*/
|
||||
def phi(connection: ParsedTransportAddress): Double = {
|
||||
def phi(connection: Address): Double = {
|
||||
val oldState = state.get
|
||||
val oldTimestamp = oldState.timestamps.get(connection)
|
||||
if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
|
||||
|
|
@ -145,7 +144,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Removes the heartbeat management for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def remove(connection: ParsedTransportAddress) {
|
||||
final def remove(connection: Address) {
|
||||
val oldState = state.get
|
||||
|
||||
if (oldState.failureStats.contains(connection)) {
|
||||
|
|
|
|||
|
|
@ -26,8 +26,8 @@ import akka.pattern.ask
|
|||
* Interface for node membership change listener.
|
||||
*/
|
||||
trait NodeMembershipChangeListener {
|
||||
def nodeConnected(node: ParsedTransportAddress)
|
||||
def nodeDisconnected(node: ParsedTransportAddress)
|
||||
def nodeConnected(node: Address)
|
||||
def nodeDisconnected(node: Address)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -35,9 +35,9 @@ trait NodeMembershipChangeListener {
|
|||
*/
|
||||
case class Gossip(
|
||||
version: VectorClock,
|
||||
node: ParsedTransportAddress,
|
||||
availableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress],
|
||||
unavailableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress])
|
||||
node: Address,
|
||||
availableNodes: Set[Address] = Set.empty[Address],
|
||||
unavailableNodes: Set[Address] = Set.empty[Address])
|
||||
|
||||
// ====== START - NEW GOSSIP IMPLEMENTATION ======
|
||||
/*
|
||||
|
|
@ -93,7 +93,7 @@ case class Gossip(
|
|||
* gossip to random seed with certain probability depending on number of unreachable, seed and live nodes.
|
||||
* </pre>
|
||||
*/
|
||||
class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
||||
class Gossiper(remote: RemoteActorRefProvider, system: ActorSystemImpl) {
|
||||
|
||||
/**
|
||||
* Represents the state for this Gossiper. Implemented using optimistic lockless concurrency,
|
||||
|
|
@ -107,27 +107,19 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
private val serialization = remote.serialization
|
||||
private val log = Logging(system, "Gossiper")
|
||||
private val failureDetector = remote.failureDetector
|
||||
private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[ParsedTransportAddress, ActorRef])
|
||||
private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[Address, ActorRef])
|
||||
|
||||
private val seeds = {
|
||||
val seeds = remoteSettings.SeedNodes flatMap {
|
||||
case x: UnparsedTransportAddress ⇒
|
||||
x.parse(remote.transports) match {
|
||||
case y: ParsedTransportAddress ⇒ Some(y)
|
||||
case _ ⇒ None
|
||||
}
|
||||
case _ ⇒ None
|
||||
}
|
||||
if (seeds.isEmpty) throw new ConfigurationException(
|
||||
if (remoteSettings.SeedNodes.isEmpty) throw new ConfigurationException(
|
||||
"At least one seed node must be defined in the configuration [akka.cluster.seed-nodes]")
|
||||
else seeds
|
||||
else remoteSettings.SeedNodes
|
||||
}
|
||||
|
||||
private val address = remote.remoteAddress
|
||||
private val address = remote.transport.address
|
||||
private val nodeFingerprint = address.##
|
||||
|
||||
private val random = SecureRandom.getInstance("SHA1PRNG")
|
||||
private val initalDelayForGossip = remoteSettings.InitalDelayForGossip
|
||||
private val initalDelayForGossip = remoteSettings.InitialDelayForGossip
|
||||
private val gossipFrequency = remoteSettings.GossipFrequency
|
||||
|
||||
private val state = new AtomicReference[State](State(currentGossip = newGossip()))
|
||||
|
|
@ -165,7 +157,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
node ← oldAvailableNodes
|
||||
if connectionManager.connectionFor(node).isEmpty
|
||||
} {
|
||||
val connectionFactory = () ⇒ system.actorFor(RootActorPath(RemoteSystemAddress(system.name, gossipingNode)) / "remote")
|
||||
val connectionFactory = () ⇒ system.actorFor(RootActorPath(gossipingNode) / "remote")
|
||||
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
|
||||
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
|
||||
}
|
||||
|
|
@ -239,7 +231,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
/**
|
||||
* Gossips set of nodes passed in as argument. Returns 'true' if it gossiped to a "seed" node.
|
||||
*/
|
||||
private def gossipTo(nodes: Set[ParsedTransportAddress]): Boolean = {
|
||||
private def gossipTo(nodes: Set[Address]): Boolean = {
|
||||
val peers = nodes filter (_ != address) // filter out myself
|
||||
val peer = selectRandomNode(peers)
|
||||
val oldState = state.get
|
||||
|
|
@ -297,8 +289,8 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
|
||||
private def newGossip(): Gossip = Gossip(
|
||||
version = VectorClock(),
|
||||
node = address.transport,
|
||||
availableNodes = Set(address.transport))
|
||||
node = address,
|
||||
availableNodes = Set(address))
|
||||
|
||||
private def incrementVersionForGossip(from: Gossip): Gossip = {
|
||||
val newVersion = from.version.increment(nodeFingerprint, newTimestamp)
|
||||
|
|
@ -313,7 +305,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
}
|
||||
}
|
||||
|
||||
private def selectRandomNode(nodes: Set[ParsedTransportAddress]): ParsedTransportAddress = {
|
||||
private def selectRandomNode(nodes: Set[Address]): Address = {
|
||||
nodes.toList(random.nextInt(nodes.size))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,11 +12,11 @@ import akka.util.ReflectiveAccess
|
|||
|
||||
object MessageSerializer {
|
||||
|
||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
|
||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = {
|
||||
val clazz = if (messageProtocol.hasMessageManifest) {
|
||||
Option(ReflectiveAccess.getClassFor[AnyRef](
|
||||
messageProtocol.getMessageManifest.toStringUtf8,
|
||||
classLoader.getOrElse(ReflectiveAccess.loader)) match {
|
||||
classLoader) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(r) ⇒ r
|
||||
})
|
||||
|
|
|
|||
|
|
@ -5,7 +5,8 @@
|
|||
package akka.remote
|
||||
|
||||
import scala.collection.mutable
|
||||
import akka.actor.{ Actor, Props, ActorSystemImpl }
|
||||
|
||||
import akka.actor.{ Props, Address, ActorSystemImpl, Actor }
|
||||
|
||||
/**
|
||||
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
|
||||
|
|
@ -15,10 +16,10 @@ object NetworkEventStream {
|
|||
|
||||
private sealed trait NetworkEventStreamEvent
|
||||
|
||||
private case class Register(listener: Listener, connectionAddress: ParsedTransportAddress)
|
||||
private case class Register(listener: Listener, connectionAddress: Address)
|
||||
extends NetworkEventStreamEvent
|
||||
|
||||
private case class Unregister(listener: Listener, connectionAddress: ParsedTransportAddress)
|
||||
private case class Unregister(listener: Listener, connectionAddress: Address)
|
||||
extends NetworkEventStreamEvent
|
||||
|
||||
/**
|
||||
|
|
@ -33,8 +34,8 @@ object NetworkEventStream {
|
|||
*/
|
||||
private class Channel extends Actor {
|
||||
|
||||
val listeners = new mutable.HashMap[ParsedTransportAddress, mutable.Set[Listener]]() {
|
||||
override def default(k: ParsedTransportAddress) = mutable.Set.empty[Listener]
|
||||
val listeners = new mutable.HashMap[Address, mutable.Set[Listener]]() {
|
||||
override def default(k: Address) = mutable.Set.empty[Listener]
|
||||
}
|
||||
|
||||
def receive = {
|
||||
|
|
@ -65,12 +66,12 @@ class NetworkEventStream(system: ActorSystemImpl) {
|
|||
/**
|
||||
* Registers a network event stream listener (asyncronously).
|
||||
*/
|
||||
def register(listener: Listener, connectionAddress: ParsedTransportAddress) =
|
||||
def register(listener: Listener, connectionAddress: Address) =
|
||||
sender ! Register(listener, connectionAddress)
|
||||
|
||||
/**
|
||||
* Unregisters a network event stream listener (asyncronously) .
|
||||
*/
|
||||
def unregister(listener: Listener, connectionAddress: ParsedTransportAddress) =
|
||||
def unregister(listener: Listener, connectionAddress: Address) =
|
||||
sender ! Unregister(listener, connectionAddress)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,274 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import akka.event._
|
||||
import akka.util._
|
||||
import akka.serialization.{ Serialization, SerializationExtension }
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.SystemMessage
|
||||
import scala.annotation.tailrec
|
||||
import akka.remote.RemoteProtocol.{ ActorRefProtocol, AkkaRemoteProtocol, RemoteControlProtocol, RemoteMessageProtocol }
|
||||
|
||||
/**
|
||||
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
||||
*/
|
||||
class Remote(val settings: ActorSystem.Settings, val remoteSettings: RemoteSettings) {
|
||||
|
||||
import settings._
|
||||
|
||||
// TODO make this really pluggable
|
||||
val transports: TransportsMap = Map("akka" -> ((h, p) ⇒ Right(RemoteNettyAddress(h, p))))
|
||||
val remoteAddress: RemoteSystemAddress[ParsedTransportAddress] = {
|
||||
val unparsedAddress = remoteSettings.serverSettings.URI match {
|
||||
case RemoteAddressExtractor(a) ⇒ a
|
||||
case x ⇒ throw new IllegalArgumentException("cannot parse URI " + x)
|
||||
}
|
||||
val parsed = unparsedAddress.parse(transports) match {
|
||||
case Left(x) ⇒ throw new IllegalArgumentException(x.transport.error)
|
||||
case Right(x) ⇒ x
|
||||
}
|
||||
parsed.copy(system = settings.name)
|
||||
}
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
|
||||
@volatile
|
||||
private var _computeGridDispatcher: MessageDispatcher = _
|
||||
def computeGridDispatcher = _computeGridDispatcher
|
||||
|
||||
@volatile
|
||||
private var _remoteDaemon: InternalActorRef = _
|
||||
def remoteDaemon = _remoteDaemon
|
||||
|
||||
@volatile
|
||||
private var _eventStream: NetworkEventStream = _
|
||||
def eventStream = _eventStream
|
||||
|
||||
@volatile
|
||||
private var _transport: RemoteSupport[ParsedTransportAddress] = _
|
||||
def transport = _transport
|
||||
|
||||
@volatile
|
||||
private var _provider: RemoteActorRefProvider = _
|
||||
def provider = _provider
|
||||
|
||||
def init(system: ActorSystemImpl, provider: RemoteActorRefProvider) = {
|
||||
|
||||
val log = Logging(system, "Remote")
|
||||
|
||||
_provider = provider
|
||||
_serialization = SerializationExtension(system)
|
||||
_computeGridDispatcher = system.dispatchers.lookup("akka.remote.compute-grid-dispatcher")
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log)
|
||||
_eventStream = new NetworkEventStream(system)
|
||||
_transport = {
|
||||
val arguments = Seq(
|
||||
classOf[ActorSystemImpl] -> system,
|
||||
classOf[Remote] -> this,
|
||||
classOf[RemoteSystemAddress[_ <: ParsedTransportAddress]] -> remoteAddress)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
|
||||
ReflectiveAccess.createInstance[RemoteSupport[ParsedTransportAddress]](remoteSettings.RemoteTransport, types, values) match {
|
||||
case Left(problem) ⇒
|
||||
|
||||
log.error(problem, "Could not load remote transport layer")
|
||||
throw problem
|
||||
|
||||
case Right(remote) ⇒
|
||||
|
||||
remote.start(Option(Thread.currentThread().getContextClassLoader)) //TODO Any application loader here?
|
||||
|
||||
val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case _ ⇒ //ignore other
|
||||
}
|
||||
}), "RemoteClientLifeCycleListener")
|
||||
|
||||
system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
remote
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Starting remote server on [{}@{}]", system.name, remoteAddress)
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait DaemonMsg
|
||||
case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||
case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
|
||||
|
||||
/**
|
||||
* Internal system "daemon" actor for remote internal communication.
|
||||
*
|
||||
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
|
||||
*/
|
||||
class RemoteSystemDaemon(system: ActorSystemImpl, remote: Remote, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter)
|
||||
extends VirtualPathContainer(system.provider, _path, _parent, _log) {
|
||||
|
||||
/**
|
||||
* Find the longest matching path which we know about and return that ref
|
||||
* (or ask that ref to continue searching if elements are left).
|
||||
*/
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
|
||||
@tailrec
|
||||
def rec(s: String, n: Int): (InternalActorRef, Int) = {
|
||||
getChild(s) match {
|
||||
case null ⇒
|
||||
val last = s.lastIndexOf('/')
|
||||
if (last == -1) (Nobody, n)
|
||||
else rec(s.substring(0, last), n + 1)
|
||||
case ref ⇒ (ref, n)
|
||||
}
|
||||
}
|
||||
|
||||
val full = Vector() ++ names
|
||||
rec(full.mkString("/"), 0) match {
|
||||
case (Nobody, _) ⇒ Nobody
|
||||
case (ref, 0) ⇒ ref
|
||||
case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator)
|
||||
}
|
||||
}
|
||||
|
||||
override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match {
|
||||
case message: DaemonMsg ⇒
|
||||
log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address.hostPort)
|
||||
message match {
|
||||
case DaemonMsgCreate(factory, path, supervisor) ⇒
|
||||
import remote.remoteAddress
|
||||
implicit val t = remote.transports
|
||||
|
||||
path match {
|
||||
case ParsedActorPath(`remoteAddress`, elems) if elems.nonEmpty && elems.head == "remote" ⇒
|
||||
// TODO RK canonicalize path so as not to duplicate it always #1446
|
||||
val subpath = elems.drop(1)
|
||||
val path = remote.remoteDaemon.path / subpath
|
||||
val actor = system.provider.actorOf(system,
|
||||
Props(creator = factory),
|
||||
supervisor.asInstanceOf[InternalActorRef],
|
||||
path, true, None)
|
||||
addChild(subpath.mkString("/"), actor)
|
||||
system.deathWatch.subscribe(this, actor)
|
||||
case _ ⇒
|
||||
log.error("remote path does not match path from message [{}]", message)
|
||||
}
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
val other = system.actorFor(watcher.path.root / "remote")
|
||||
system.deathWatch.subscribe(other, watched)
|
||||
}
|
||||
|
||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
||||
|
||||
case t: Terminated ⇒ system.deathWatch.publish(t)
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, classLoader: Option[ClassLoader] = None) {
|
||||
|
||||
def originalReceiver = input.getRecipient.getPath
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath)
|
||||
else system.deadLetters
|
||||
|
||||
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
|
||||
|
||||
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, classLoader)
|
||||
|
||||
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
|
||||
}
|
||||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def log: LoggingAdapter
|
||||
|
||||
def system: ActorSystem
|
||||
|
||||
def remote: Remote
|
||||
|
||||
protected def useUntrustedMode: Boolean
|
||||
|
||||
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setMessage(rmp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setInstruction(rcp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
/**
|
||||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
ActorRefProtocol.newBuilder.setPath(actor.path.toString).build
|
||||
}
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
recipient: ActorRef,
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
||||
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
|
||||
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
||||
|
||||
messageBuilder
|
||||
}
|
||||
|
||||
def receiveMessage(remoteMessage: RemoteMessage) {
|
||||
if (remote.remoteSettings.LogReceivedMessages)
|
||||
log.debug("received message [{}]", remoteMessage)
|
||||
|
||||
val remoteDaemon = remote.remoteDaemon
|
||||
|
||||
remoteMessage.recipient match {
|
||||
case `remoteDaemon` ⇒
|
||||
remoteMessage.payload match {
|
||||
case m @ (_: DaemonMsg | _: Terminated) ⇒
|
||||
try remoteDaemon ! m catch {
|
||||
case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender)
|
||||
}
|
||||
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
||||
}
|
||||
case l: LocalRef ⇒
|
||||
remoteMessage.payload match {
|
||||
case msg: SystemMessage ⇒
|
||||
if (useUntrustedMode)
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message")
|
||||
else l.sendSystemMessage(msg)
|
||||
case _: AutoReceivedMessage if (useUntrustedMode) ⇒
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor")
|
||||
case m ⇒ l.!(m)(remoteMessage.sender)
|
||||
}
|
||||
case r: RemoteActorRef ⇒
|
||||
implicit val t = remote.transports
|
||||
remoteMessage.originalReceiver match {
|
||||
case ParsedActorPath(address, _) if address == remote.remoteDaemon.path.address ⇒
|
||||
r.!(remoteMessage.payload)(remoteMessage.sender)
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -6,9 +6,14 @@ package akka.remote
|
|||
|
||||
import akka.actor._
|
||||
import akka.dispatch._
|
||||
import akka.event.{ DeathWatch, Logging }
|
||||
import akka.event.{ DeathWatch, Logging, LoggingAdapter }
|
||||
import akka.event.EventStream
|
||||
import akka.config.ConfigurationException
|
||||
import java.util.concurrent.{ TimeoutException }
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.serialization.Serialization
|
||||
import akka.serialization.SerializationExtension
|
||||
|
||||
/**
|
||||
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
|
||||
|
|
@ -18,39 +23,93 @@ class RemoteActorRefProvider(
|
|||
val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
_deadLetters: InternalActorRef) extends ActorRefProvider {
|
||||
val classloader: ClassLoader) extends ActorRefProvider {
|
||||
|
||||
val remoteSettings = new RemoteSettings(settings.config, systemName)
|
||||
|
||||
val deployer = new RemoteDeployer(settings, classloader)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
|
||||
|
||||
@volatile
|
||||
private var _log = local.log
|
||||
def log: LoggingAdapter = _log
|
||||
|
||||
def rootPath = local.rootPath
|
||||
def locker = local.locker
|
||||
def deadLetters = local.deadLetters
|
||||
|
||||
val deathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
// these are only available after init()
|
||||
def rootGuardian = local.rootGuardian
|
||||
def guardian = local.guardian
|
||||
def systemGuardian = local.systemGuardian
|
||||
def terminationFuture = local.terminationFuture
|
||||
def dispatcher = local.dispatcher
|
||||
|
||||
def registerTempActor(actorRef: InternalActorRef, path: ActorPath) = local.registerTempActor(actorRef, path)
|
||||
def unregisterTempActor(path: ActorPath) = local.unregisterTempActor(path)
|
||||
def tempPath() = local.tempPath()
|
||||
def tempContainer = local.tempContainer
|
||||
|
||||
val deployer = new RemoteDeployer(settings)
|
||||
@volatile
|
||||
private var _transport: RemoteTransport = _
|
||||
def transport: RemoteTransport = _transport
|
||||
|
||||
val remote = new Remote(settings, remoteSettings)
|
||||
implicit val transports = remote.transports
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
|
||||
val log = Logging(eventStream, "RemoteActorRefProvider(" + remote.remoteAddress + ")")
|
||||
@volatile
|
||||
private var _remoteDaemon: InternalActorRef = _
|
||||
def remoteDaemon = _remoteDaemon
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(remote.remoteAddress)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath, deployer)
|
||||
|
||||
val deathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
@volatile
|
||||
private var _networkEventStream: NetworkEventStream = _
|
||||
def networkEventStream = _networkEventStream
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
local.init(system)
|
||||
remote.init(system, this)
|
||||
local.registerExtraNames(Map(("remote", remote.remoteDaemon)))
|
||||
terminationFuture.onComplete(_ ⇒ remote.transport.shutdown())
|
||||
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log)
|
||||
local.registerExtraNames(Map(("remote", remoteDaemon)))
|
||||
|
||||
_serialization = SerializationExtension(system)
|
||||
|
||||
_networkEventStream = new NetworkEventStream(system)
|
||||
system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
_transport = {
|
||||
val fqn = remoteSettings.RemoteTransport
|
||||
val args = Seq(
|
||||
classOf[RemoteSettings] -> remoteSettings,
|
||||
classOf[ActorSystemImpl] -> system,
|
||||
classOf[RemoteActorRefProvider] -> this)
|
||||
|
||||
ReflectiveAccess.createInstance[RemoteTransport](fqn, args, system.internalClassLoader) match {
|
||||
case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
|
||||
case Right(remote) ⇒ remote
|
||||
}
|
||||
}
|
||||
|
||||
_log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")")
|
||||
|
||||
// this enables reception of remote requests
|
||||
_transport.start()
|
||||
|
||||
val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case _ ⇒ //ignore other
|
||||
}
|
||||
}), "RemoteClientLifeCycleListener")
|
||||
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
terminationFuture.onComplete(_ ⇒ transport.shutdown())
|
||||
}
|
||||
|
||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy]): InternalActorRef = {
|
||||
|
|
@ -100,18 +159,12 @@ class RemoteActorRefProvider(
|
|||
})
|
||||
|
||||
deployment match {
|
||||
case Some(Deploy(_, _, _, RemoteScope(address))) ⇒
|
||||
// FIXME RK this should be done within the deployer, i.e. the whole parsing business
|
||||
address.parse(remote.transports) match {
|
||||
case Left(x) ⇒
|
||||
throw new ConfigurationException("cannot parse remote address: " + x)
|
||||
case Right(addr) ⇒
|
||||
case Some(Deploy(_, _, _, RemoteScope(addr))) ⇒
|
||||
if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment)
|
||||
else {
|
||||
val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements
|
||||
val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements
|
||||
useActorOnNode(rpath, props.creator, supervisor)
|
||||
new RemoteActorRef(this, remote.transport, rpath, supervisor, None)
|
||||
}
|
||||
new RemoteActorRef(this, transport, rpath, supervisor)
|
||||
}
|
||||
|
||||
case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment)
|
||||
|
|
@ -119,16 +172,14 @@ class RemoteActorRefProvider(
|
|||
}
|
||||
}
|
||||
|
||||
def actorFor(path: ActorPath): InternalActorRef = path.root match {
|
||||
case `rootPath` ⇒ actorFor(rootGuardian, path.elements)
|
||||
case RootActorPath(_: RemoteSystemAddress[_], _) ⇒ new RemoteActorRef(this, remote.transport, path, Nobody, None)
|
||||
case _ ⇒ local.actorFor(path)
|
||||
}
|
||||
def actorFor(path: ActorPath): InternalActorRef =
|
||||
if (path.address == rootPath.address || path.address == transport.address) actorFor(rootGuardian, path.elements)
|
||||
else new RemoteActorRef(this, transport, path, Nobody)
|
||||
|
||||
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
|
||||
case ParsedActorPath(address, elems) ⇒
|
||||
if (address == rootPath.address) actorFor(rootGuardian, elems)
|
||||
else new RemoteActorRef(this, remote.transport, new RootActorPath(address) / elems, Nobody, None)
|
||||
case ActorPathExtractor(address, elems) ⇒
|
||||
if (address == rootPath.address || address == transport.address) actorFor(rootGuardian, elems)
|
||||
else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody)
|
||||
case _ ⇒ local.actorFor(ref, path)
|
||||
}
|
||||
|
||||
|
|
@ -155,10 +206,9 @@ trait RemoteRef extends ActorRefScope {
|
|||
*/
|
||||
private[akka] class RemoteActorRef private[akka] (
|
||||
val provider: RemoteActorRefProvider,
|
||||
remote: RemoteSupport[ParsedTransportAddress],
|
||||
remote: RemoteTransport,
|
||||
val path: ActorPath,
|
||||
val getParent: InternalActorRef,
|
||||
loader: Option[ClassLoader])
|
||||
val getParent: InternalActorRef)
|
||||
extends InternalActorRef with RemoteRef {
|
||||
|
||||
def getChild(name: Iterator[String]): InternalActorRef = {
|
||||
|
|
@ -166,7 +216,7 @@ private[akka] class RemoteActorRef private[akka] (
|
|||
s.headOption match {
|
||||
case None ⇒ this
|
||||
case Some("..") ⇒ getParent getChild name
|
||||
case _ ⇒ new RemoteActorRef(provider, remote, path / s, Nobody, loader)
|
||||
case _ ⇒ new RemoteActorRef(provider, remote, path / s, Nobody)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -175,9 +225,9 @@ private[akka] class RemoteActorRef private[akka] (
|
|||
|
||||
def isTerminated: Boolean = !running
|
||||
|
||||
def sendSystemMessage(message: SystemMessage): Unit = remote.send(message, None, this, loader)
|
||||
def sendSystemMessage(message: SystemMessage): Unit = remote.send(message, None, this)
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), this, loader)
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), this)
|
||||
|
||||
def suspend(): Unit = sendSystemMessage(Suspend())
|
||||
|
||||
|
|
@ -188,7 +238,7 @@ private[akka] class RemoteActorRef private[akka] (
|
|||
def restart(cause: Throwable): Unit = sendSystemMessage(Recreate(cause))
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
private def writeReplace(): AnyRef = SerializedActorRef(path.toString)
|
||||
private def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
class RemoteDeathWatch(val local: LocalDeathWatch, val provider: RemoteActorRefProvider) extends DeathWatch {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,5 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote
|
||||
|
||||
|
|
@ -18,14 +18,14 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
*/
|
||||
class RemoteConnectionManager(
|
||||
system: ActorSystemImpl,
|
||||
remote: Remote,
|
||||
initialConnections: Map[ParsedTransportAddress, ActorRef] = Map.empty[ParsedTransportAddress, ActorRef])
|
||||
remote: RemoteActorRefProvider,
|
||||
initialConnections: Map[Address, ActorRef] = Map.empty[Address, ActorRef])
|
||||
extends ConnectionManager {
|
||||
|
||||
val log = Logging(system, "RemoteConnectionManager")
|
||||
|
||||
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
|
||||
case class State(version: Long, connections: Map[ParsedTransportAddress, ActorRef])
|
||||
case class State(version: Long, connections: Map[Address, ActorRef])
|
||||
extends VersionedIterable[ActorRef] {
|
||||
def iterable: Iterable[ActorRef] = connections.values
|
||||
}
|
||||
|
|
@ -51,7 +51,7 @@ class RemoteConnectionManager(
|
|||
|
||||
def size: Int = connections.connections.size
|
||||
|
||||
def connectionFor(address: ParsedTransportAddress): Option[ActorRef] = connections.connections.get(address)
|
||||
def connectionFor(address: Address): Option[ActorRef] = connections.connections.get(address)
|
||||
|
||||
def isEmpty: Boolean = connections.connections.isEmpty
|
||||
|
||||
|
|
@ -60,7 +60,7 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
@tailrec
|
||||
final def failOver(from: ParsedTransportAddress, to: ParsedTransportAddress) {
|
||||
final def failOver(from: Address, to: Address) {
|
||||
log.debug("Failing over connection from [{}] to [{}]", from, to)
|
||||
|
||||
val oldState = state.get
|
||||
|
|
@ -91,8 +91,8 @@ class RemoteConnectionManager(
|
|||
val oldState = state.get()
|
||||
var changed = false
|
||||
|
||||
var faultyAddress: ParsedTransportAddress = null
|
||||
var newConnections = Map.empty[ParsedTransportAddress, ActorRef]
|
||||
var faultyAddress: Address = null
|
||||
var newConnections = Map.empty[Address, ActorRef]
|
||||
|
||||
oldState.connections.keys foreach { address ⇒
|
||||
val actorRef: ActorRef = oldState.connections.get(address).get
|
||||
|
|
@ -118,7 +118,7 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
@tailrec
|
||||
final def putIfAbsent(address: ParsedTransportAddress, newConnectionFactory: () ⇒ ActorRef): ActorRef = {
|
||||
final def putIfAbsent(address: Address, newConnectionFactory: () ⇒ ActorRef): ActorRef = {
|
||||
|
||||
val oldState = state.get()
|
||||
val oldConnections = oldState.connections
|
||||
|
|
@ -145,6 +145,6 @@ class RemoteConnectionManager(
|
|||
}
|
||||
}
|
||||
|
||||
private[remote] def newConnection(remoteAddress: ParsedTransportAddress, actorPath: ActorPath) =
|
||||
new RemoteActorRef(remote.provider, remote.transport, actorPath, Nobody, None)
|
||||
private[remote] def newConnection(remoteAddress: Address, actorPath: ActorPath) =
|
||||
new RemoteActorRef(remote, remote.transport, actorPath, Nobody)
|
||||
}
|
||||
|
|
|
|||
83
akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
Normal file
83
akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
Normal file
|
|
@ -0,0 +1,83 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import scala.annotation.tailrec
|
||||
|
||||
import akka.actor.{ VirtualPathContainer, Terminated, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
|
||||
import akka.event.LoggingAdapter
|
||||
|
||||
sealed trait DaemonMsg
|
||||
case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||
case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
|
||||
|
||||
/**
|
||||
* Internal system "daemon" actor for remote internal communication.
|
||||
*
|
||||
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
|
||||
*
|
||||
* INTERNAL USE ONLY!
|
||||
*/
|
||||
private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter)
|
||||
extends VirtualPathContainer(system.provider, _path, _parent, _log) {
|
||||
|
||||
/**
|
||||
* Find the longest matching path which we know about and return that ref
|
||||
* (or ask that ref to continue searching if elements are left).
|
||||
*/
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
|
||||
@tailrec
|
||||
def rec(s: String, n: Int): (InternalActorRef, Int) = {
|
||||
getChild(s) match {
|
||||
case null ⇒
|
||||
val last = s.lastIndexOf('/')
|
||||
if (last == -1) (Nobody, n)
|
||||
else rec(s.substring(0, last), n + 1)
|
||||
case ref ⇒ (ref, n)
|
||||
}
|
||||
}
|
||||
|
||||
val full = Vector() ++ names
|
||||
rec(full.mkString("/"), 0) match {
|
||||
case (Nobody, _) ⇒ Nobody
|
||||
case (ref, 0) ⇒ ref
|
||||
case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator)
|
||||
}
|
||||
}
|
||||
|
||||
override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match {
|
||||
case message: DaemonMsg ⇒
|
||||
log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address)
|
||||
message match {
|
||||
case DaemonMsgCreate(factory, path, supervisor) ⇒
|
||||
path match {
|
||||
case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒
|
||||
// TODO RK currently the extracted “address” is just ignored, is that okay?
|
||||
// TODO RK canonicalize path so as not to duplicate it always #1446
|
||||
val subpath = elems.drop(1)
|
||||
val path = this.path / subpath
|
||||
val actor = system.provider.actorOf(system,
|
||||
Props(creator = factory),
|
||||
supervisor.asInstanceOf[InternalActorRef],
|
||||
path, true, None)
|
||||
addChild(subpath.mkString("/"), actor)
|
||||
system.deathWatch.subscribe(this, actor)
|
||||
case _ ⇒
|
||||
log.error("remote path does not match path from message [{}]", message)
|
||||
}
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
val other = system.actorFor(watcher.path.root / "remote")
|
||||
system.deathWatch.subscribe(other, watched)
|
||||
}
|
||||
|
||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
||||
|
||||
case t: Terminated ⇒ system.deathWatch.publish(t)
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -8,9 +8,9 @@ import akka.routing._
|
|||
import com.typesafe.config._
|
||||
import akka.config.ConfigurationException
|
||||
|
||||
case class RemoteScope(node: UnparsedSystemAddress[UnparsedTransportAddress]) extends Scope
|
||||
case class RemoteScope(node: Address) extends Scope
|
||||
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings) {
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings, _classloader: ClassLoader) extends Deployer(_settings, _classloader) {
|
||||
|
||||
override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -18,7 +18,7 @@ class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings
|
|||
super.parseConfig(path, config) match {
|
||||
case d @ Some(deploy) ⇒
|
||||
deploy.config.getString("remote") match {
|
||||
case RemoteAddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r)))
|
||||
case AddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r)))
|
||||
case str ⇒
|
||||
if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str)
|
||||
val nodes = deploy.config.getStringList("target.nodes").asScala
|
||||
|
|
|
|||
|
|
@ -1,340 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import akka.AkkaException
|
||||
import scala.reflect.BeanProperty
|
||||
import java.net.URI
|
||||
import java.net.URISyntaxException
|
||||
import java.net.InetAddress
|
||||
import java.net.UnknownHostException
|
||||
import akka.event.Logging
|
||||
|
||||
/**
|
||||
* Interface for remote transports to encode their addresses. The three parts
|
||||
* are named according to the URI spec (precisely java.net.URI) which is used
|
||||
* for parsing. That means that the address’ parts must conform to what an
|
||||
* URI expects, but otherwise each transport may assign a different meaning
|
||||
* to these parts.
|
||||
*/
|
||||
trait RemoteTransportAddress {
|
||||
def protocol: String
|
||||
def host: String
|
||||
def port: Int
|
||||
}
|
||||
|
||||
trait ParsedTransportAddress extends RemoteTransportAddress
|
||||
|
||||
case class RemoteNettyAddress(host: String, ip: Option[InetAddress], port: Int) extends ParsedTransportAddress {
|
||||
def protocol = "akka"
|
||||
|
||||
override def toString(): String = "akka://" + host + ":" + port
|
||||
}
|
||||
|
||||
object RemoteNettyAddress {
|
||||
def apply(host: String, port: Int): RemoteNettyAddress = {
|
||||
// TODO ticket #1639
|
||||
val ip = try Some(InetAddress.getByName(host)) catch { case _: UnknownHostException ⇒ None }
|
||||
new RemoteNettyAddress(host, ip, port)
|
||||
}
|
||||
def apply(s: String): RemoteNettyAddress = {
|
||||
val RE = """([^:]+):(\d+)""".r
|
||||
s match {
|
||||
case RE(h, p) ⇒ apply(h, Integer.parseInt(p))
|
||||
case _ ⇒ throw new IllegalArgumentException("cannot parse " + s + " as <host:port>")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class UnparsedTransportAddress(protocol: String, host: String, port: Int) extends RemoteTransportAddress {
|
||||
def parse(transports: TransportsMap): RemoteTransportAddress =
|
||||
transports.get(protocol)
|
||||
.map(_(host, port))
|
||||
.toRight("protocol " + protocol + " not known")
|
||||
.joinRight.fold(UnparseableTransportAddress(protocol, host, port, _), identity)
|
||||
}
|
||||
|
||||
case class UnparseableTransportAddress(protocol: String, host: String, port: Int, error: String) extends RemoteTransportAddress
|
||||
|
||||
case class RemoteSystemAddress[+T <: ParsedTransportAddress](system: String, transport: T) extends Address {
|
||||
def protocol = transport.protocol
|
||||
@transient
|
||||
lazy val hostPort = system + "@" + transport.host + ":" + transport.port
|
||||
}
|
||||
|
||||
case class UnparsedSystemAddress[+T <: RemoteTransportAddress](system: Option[String], transport: T) {
|
||||
def parse(transports: TransportsMap): Either[UnparsedSystemAddress[UnparseableTransportAddress], RemoteSystemAddress[ParsedTransportAddress]] =
|
||||
system match {
|
||||
case Some(sys) ⇒
|
||||
transport match {
|
||||
case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x))
|
||||
case y: UnparsedTransportAddress ⇒
|
||||
y.parse(transports) match {
|
||||
case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x))
|
||||
case y: UnparseableTransportAddress ⇒ Left(UnparsedSystemAddress(system, y))
|
||||
case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z)))
|
||||
}
|
||||
case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z)))
|
||||
}
|
||||
case None ⇒ Left(UnparsedSystemAddress(None, UnparseableTransportAddress(transport.protocol, transport.host, transport.port, "no system name specified")))
|
||||
}
|
||||
}
|
||||
|
||||
object RemoteAddressExtractor {
|
||||
def unapply(s: String): Option[UnparsedSystemAddress[UnparsedTransportAddress]] = {
|
||||
try {
|
||||
val uri = new URI(s)
|
||||
if (uri.getScheme == null || uri.getHost == null || uri.getPort == -1) None
|
||||
else Some(UnparsedSystemAddress(Option(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)))
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object RemoteActorPath {
|
||||
def unapply(addr: String): Option[(UnparsedSystemAddress[UnparsedTransportAddress], Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None
|
||||
else Some(UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)),
|
||||
ActorPath.split(uri.getPath).drop(1))
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ParsedActorPath {
|
||||
def unapply(addr: String)(implicit transports: TransportsMap): Option[(RemoteSystemAddress[ParsedTransportAddress], Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None
|
||||
else
|
||||
UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)).parse(transports) match {
|
||||
case Left(_) ⇒ None
|
||||
case Right(x) ⇒ Some(x, ActorPath.split(uri.getPath).drop(1))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteException(message: String) extends AkkaException(message)
|
||||
|
||||
trait RemoteModule {
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* Remote life-cycle events.
|
||||
*/
|
||||
sealed trait RemoteLifeCycleEvent {
|
||||
def logLevel: Logging.LogLevel
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteClient.
|
||||
*/
|
||||
trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent {
|
||||
def remoteAddress: ParsedTransportAddress
|
||||
}
|
||||
|
||||
case class RemoteClientError[T <: ParsedTransportAddress](
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteClientError@" +
|
||||
remoteAddress +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteClientDisconnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientDisconnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientConnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientConnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientStarted[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientStarted@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientShutdown[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientShutdown@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientWriteFailed[T <: ParsedTransportAddress](
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteClientWriteFailed@" +
|
||||
remoteAddress +
|
||||
": MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteServer.
|
||||
*/
|
||||
trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent
|
||||
|
||||
case class RemoteServerStarted[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerStarted@" + remote.name
|
||||
}
|
||||
|
||||
case class RemoteServerShutdown[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerShutdown@" + remote.name
|
||||
}
|
||||
|
||||
case class RemoteServerError[T <: ParsedTransportAddress](
|
||||
@BeanProperty val cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteServerError@" +
|
||||
remote.name +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientConnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientConnected@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientDisconnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientDisconnected@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientClosed[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientClosed@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerWriteFailed[T <: ParsedTransportAddress](
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteServerWriteFailed@" +
|
||||
remote +
|
||||
": ClientAddress[" +
|
||||
remoteAddress +
|
||||
"] MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
|
||||
*/
|
||||
class RemoteClientException[T <: ParsedTransportAddress] private[akka] (
|
||||
message: String,
|
||||
@BeanProperty val client: RemoteSupport[T],
|
||||
val remoteAddress: T, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
||||
abstract class RemoteSupport[-T <: ParsedTransportAddress](val system: ActorSystemImpl) {
|
||||
/**
|
||||
* Shuts down the remoting
|
||||
*/
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Gets the name of the server instance
|
||||
*/
|
||||
def name: String
|
||||
|
||||
/**
|
||||
* Starts up the remoting
|
||||
*/
|
||||
def start(loader: Option[ClassLoader]): Unit
|
||||
|
||||
/**
|
||||
* Shuts down a specific client connected to the supplied remote address returns true if successful
|
||||
*/
|
||||
def shutdownClientConnection(address: T): Boolean
|
||||
|
||||
/**
|
||||
* Restarts a specific client connected to the supplied remote address, but only if the client is not shut down
|
||||
*/
|
||||
def restartClientConnection(address: T): Boolean
|
||||
|
||||
/** Methods that needs to be implemented by a transport **/
|
||||
|
||||
protected[akka] def send(message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef,
|
||||
loader: Option[ClassLoader]): Unit
|
||||
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = {
|
||||
system.eventStream.publish(message)
|
||||
system.log.log(message.logLevel, "REMOTE: {}", message)
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
|
@ -9,92 +9,29 @@ import java.util.concurrent.TimeUnit.MILLISECONDS
|
|||
import java.net.InetAddress
|
||||
import akka.config.ConfigurationException
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.actor.Address
|
||||
import akka.actor.AddressExtractor
|
||||
|
||||
class RemoteSettings(val config: Config, val systemName: String) {
|
||||
|
||||
import config._
|
||||
|
||||
final val RemoteTransport = getString("akka.remote.transport")
|
||||
final val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold")
|
||||
final val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size")
|
||||
final val ShouldCompressData = getBoolean("akka.remote.use-compression")
|
||||
final val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS)
|
||||
final val InitalDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS)
|
||||
final val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS)
|
||||
final val BackoffTimeout = Duration(getMilliseconds("akka.remote.backoff-timeout"), MILLISECONDS)
|
||||
final val LogReceivedMessages = getBoolean("akka.remote.log-received-messages")
|
||||
final val LogSentMessages = getBoolean("akka.remote.log-sent-messages")
|
||||
val RemoteTransport = getString("akka.remote.transport")
|
||||
val LogReceive = getBoolean("akka.remote.log-received-messages")
|
||||
val LogSend = getBoolean("akka.remote.log-sent-messages")
|
||||
|
||||
// AccrualFailureDetector
|
||||
val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold")
|
||||
val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size")
|
||||
|
||||
// Gossiper
|
||||
val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS)
|
||||
val InitialDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS)
|
||||
val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS)
|
||||
// TODO cluster config will go into akka-cluster/reference.conf when we enable that module
|
||||
final val SeedNodes = Set.empty[RemoteNettyAddress] ++ getStringList("akka.cluster.seed-nodes").asScala.collect {
|
||||
case RemoteAddressExtractor(addr) ⇒ addr.transport
|
||||
val SeedNodes = Set.empty[Address] ++ getStringList("akka.cluster.seed-nodes").asScala.collect {
|
||||
case AddressExtractor(addr) ⇒ addr
|
||||
}
|
||||
|
||||
final val serverSettings = new RemoteServerSettings
|
||||
final val clientSettings = new RemoteClientSettings
|
||||
|
||||
class RemoteClientSettings {
|
||||
val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
|
||||
final val ConnectionTimeout = Duration(getMilliseconds("akka.remote.client.connection-timeout"), MILLISECONDS)
|
||||
final val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS)
|
||||
final val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS)
|
||||
final val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS)
|
||||
final val MessageFrameSize = getBytes("akka.remote.client.message-frame-size").toInt
|
||||
}
|
||||
|
||||
class RemoteServerSettings {
|
||||
import scala.collection.JavaConverters._
|
||||
final val MessageFrameSize = getBytes("akka.remote.server.message-frame-size").toInt
|
||||
final val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
final val RequireCookie = {
|
||||
val requireCookie = getBoolean("akka.remote.server.require-cookie")
|
||||
if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
|
||||
requireCookie
|
||||
}
|
||||
|
||||
final val UsePassiveConnections = getBoolean("akka.remote.use-passive-connections")
|
||||
|
||||
final val UntrustedMode = getBoolean("akka.remote.server.untrusted-mode")
|
||||
final val Hostname = getString("akka.remote.server.hostname") match {
|
||||
case "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
}
|
||||
final val Port = getInt("akka.remote.server.port") match {
|
||||
case 0 ⇒ try {
|
||||
val s = new java.net.ServerSocket(0)
|
||||
try s.getLocalPort finally s.close()
|
||||
} catch { case e ⇒ throw new ConfigurationException("Unable to obtain random port", e) }
|
||||
case other ⇒ other
|
||||
}
|
||||
|
||||
final val Backlog = getInt("akka.remote.server.backlog")
|
||||
|
||||
final val ExecutionPoolKeepAlive = Duration(getMilliseconds("akka.remote.server.execution-pool-keepalive"), MILLISECONDS)
|
||||
|
||||
final val ExecutionPoolSize = getInt("akka.remote.server.execution-pool-size") match {
|
||||
case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.server.execution-pool-size is less than 1")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
final val MaxChannelMemorySize = getBytes("akka.remote.server.max-channel-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-channel-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
final val MaxTotalMemorySize = getBytes("akka.remote.server.max-total-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-total-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
// TODO handle the system name right and move this to config file syntax
|
||||
final val URI = "akka://sys@" + Hostname + ":" + Port
|
||||
}
|
||||
val UntrustedMode = getBoolean("akka.remote.untrusted-mode")
|
||||
}
|
||||
334
akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
Normal file
334
akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
Normal file
|
|
@ -0,0 +1,334 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import scala.reflect.BeanProperty
|
||||
import akka.actor.{ Terminated, LocalRef, InternalActorRef, AutoReceivedMessage, AddressExtractor, Address, ActorSystemImpl, ActorSystem, ActorRef }
|
||||
import akka.dispatch.SystemMessage
|
||||
import akka.event.{ LoggingAdapter, Logging }
|
||||
import akka.remote.RemoteProtocol.{ RemoteMessageProtocol, RemoteControlProtocol, AkkaRemoteProtocol, ActorRefProtocol }
|
||||
import akka.AkkaException
|
||||
import akka.serialization.Serialization
|
||||
|
||||
/**
|
||||
* Remote life-cycle events.
|
||||
*/
|
||||
sealed trait RemoteLifeCycleEvent {
|
||||
def logLevel: Logging.LogLevel
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteClient.
|
||||
*/
|
||||
trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent {
|
||||
def remoteAddress: Address
|
||||
}
|
||||
|
||||
case class RemoteClientError(
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteClientError@" +
|
||||
remoteAddress +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteClientDisconnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientDisconnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientConnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientConnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientStarted(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientStarted@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientShutdown(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientShutdown@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientWriteFailed(
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteClientWriteFailed@" +
|
||||
remoteAddress +
|
||||
": MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteServer.
|
||||
*/
|
||||
trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent
|
||||
|
||||
case class RemoteServerStarted(
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerStarted@" + remote
|
||||
}
|
||||
|
||||
case class RemoteServerShutdown(
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerShutdown@" + remote
|
||||
}
|
||||
|
||||
case class RemoteServerError(
|
||||
@BeanProperty val cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteServerError@" +
|
||||
remote +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientConnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientConnected@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientDisconnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientDisconnected@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientClosed(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientClosed@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerWriteFailed(
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteServerWriteFailed@" +
|
||||
remote +
|
||||
": ClientAddress[" +
|
||||
remoteAddress +
|
||||
"] MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
|
||||
*/
|
||||
class RemoteClientException private[akka] (
|
||||
message: String,
|
||||
@BeanProperty val client: RemoteTransport,
|
||||
val remoteAddress: Address, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
||||
class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause)
|
||||
|
||||
/**
|
||||
* The remote transport is responsible for sending and receiving messages.
|
||||
* Each transport has an address, which it should provide in
|
||||
* Serialization.currentTransportAddress (thread-local) while serializing
|
||||
* actor references (which might also be part of messages). This address must
|
||||
* be available (i.e. fully initialized) by the time the first message is
|
||||
* received or when the start() method returns, whatever happens first.
|
||||
*/
|
||||
abstract class RemoteTransport {
|
||||
/**
|
||||
* Shuts down the remoting
|
||||
*/
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Address to be used in RootActorPath of refs generated for this transport.
|
||||
*/
|
||||
def address: Address
|
||||
|
||||
/**
|
||||
* The actor system, for which this transport is instantiated. Will publish to its eventStream.
|
||||
*/
|
||||
def system: ActorSystem
|
||||
|
||||
/**
|
||||
* Start up the transport, i.e. enable incoming connections.
|
||||
*/
|
||||
def start(): Unit
|
||||
|
||||
/**
|
||||
* Shuts down a specific client connected to the supplied remote address returns true if successful
|
||||
*/
|
||||
def shutdownClientConnection(address: Address): Boolean
|
||||
|
||||
/**
|
||||
* Restarts a specific client connected to the supplied remote address, but only if the client is not shut down
|
||||
*/
|
||||
def restartClientConnection(address: Address): Boolean
|
||||
|
||||
/** Methods that needs to be implemented by a transport **/
|
||||
|
||||
protected[akka] def send(message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef): Unit
|
||||
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = {
|
||||
system.eventStream.publish(message)
|
||||
system.log.log(message.logLevel, "REMOTE: {}", message)
|
||||
}
|
||||
|
||||
override def toString = address.toString
|
||||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) {
|
||||
|
||||
def originalReceiver = input.getRecipient.getPath
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath)
|
||||
else system.deadLetters
|
||||
|
||||
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
|
||||
|
||||
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, getClass.getClassLoader)
|
||||
|
||||
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
|
||||
}
|
||||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def log: LoggingAdapter
|
||||
|
||||
def system: ActorSystemImpl
|
||||
|
||||
def provider: RemoteActorRefProvider
|
||||
|
||||
def address: Address
|
||||
|
||||
protected def useUntrustedMode: Boolean
|
||||
|
||||
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setMessage(rmp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setInstruction(rcp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
/**
|
||||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
ActorRefProtocol.newBuilder.setPath(actor.path.toStringWithAddress(address)).build
|
||||
}
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
recipient: ActorRef,
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
||||
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
||||
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
||||
|
||||
Serialization.currentTransportAddress.withValue(address) {
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
}
|
||||
|
||||
messageBuilder
|
||||
}
|
||||
|
||||
def receiveMessage(remoteMessage: RemoteMessage) {
|
||||
val remoteDaemon = provider.remoteDaemon
|
||||
|
||||
remoteMessage.recipient match {
|
||||
case `remoteDaemon` ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage)
|
||||
remoteMessage.payload match {
|
||||
case m @ (_: DaemonMsg | _: Terminated) ⇒
|
||||
try remoteDaemon ! m catch {
|
||||
case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender)
|
||||
}
|
||||
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
||||
}
|
||||
case l: LocalRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
|
||||
remoteMessage.payload match {
|
||||
case msg: SystemMessage ⇒
|
||||
if (useUntrustedMode)
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message")
|
||||
else l.sendSystemMessage(msg)
|
||||
case _: AutoReceivedMessage if (useUntrustedMode) ⇒
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor")
|
||||
case m ⇒ l.!(m)(remoteMessage.sender)
|
||||
}
|
||||
case r: RemoteRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
|
||||
remoteMessage.originalReceiver match {
|
||||
case AddressExtractor(address) if address == provider.transport.address ⇒
|
||||
// if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed)
|
||||
r.!(remoteMessage.payload)(remoteMessage.sender)
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
}
|
||||
}
|
||||
347
akka-remote/src/main/scala/akka/remote/netty/Client.scala
Normal file
347
akka-remote/src/main/scala/akka/remote/netty/Client.scala
Normal file
|
|
@ -0,0 +1,347 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import org.jboss.netty.util.HashedWheelTimer
|
||||
import org.jboss.netty.bootstrap.ClientBootstrap
|
||||
import org.jboss.netty.channel.group.DefaultChannelGroup
|
||||
import org.jboss.netty.channel.{ ChannelHandler, StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected }
|
||||
import akka.actor.{ simpleName, Address }
|
||||
import akka.AkkaException
|
||||
import akka.event.Logging
|
||||
import akka.util.Switch
|
||||
import akka.actor.ActorRef
|
||||
import org.jboss.netty.channel.ChannelFutureListener
|
||||
import akka.remote.RemoteClientWriteFailed
|
||||
import java.net.InetAddress
|
||||
import org.jboss.netty.util.TimerTask
|
||||
import org.jboss.netty.util.Timeout
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
class RemoteClientMessageBufferException(message: String, cause: Throwable) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the abstract baseclass for netty remote clients, currently there's only an
|
||||
* ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that
|
||||
* reuses an already established connection.
|
||||
*/
|
||||
abstract class RemoteClient private[akka] (
|
||||
val netty: NettyRemoteTransport,
|
||||
val remoteAddress: Address) {
|
||||
|
||||
val log = Logging(netty.system, "RemoteClient")
|
||||
|
||||
val name = simpleName(this) + "@" + remoteAddress
|
||||
|
||||
private[remote] val runSwitch = new Switch()
|
||||
|
||||
private[remote] def isRunning = runSwitch.isOn
|
||||
|
||||
protected def currentChannel: Channel
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean
|
||||
|
||||
def shutdown(): Boolean
|
||||
|
||||
def isBoundTo(address: Address): Boolean = remoteAddress == address
|
||||
|
||||
/**
|
||||
* Converts the message to the wireprotocol and sends the message across the wire
|
||||
*/
|
||||
def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) {
|
||||
if (netty.remoteSettings.LogSend) log.debug("Sending message {} from {} to {}", message, senderOption, recipient)
|
||||
send((message, senderOption, recipient))
|
||||
} else {
|
||||
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", netty, remoteAddress)
|
||||
netty.notifyListeners(RemoteClientError(exception, netty, remoteAddress))
|
||||
throw exception
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the message across the wire
|
||||
*/
|
||||
private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = {
|
||||
try {
|
||||
val channel = currentChannel
|
||||
val f = channel.write(request)
|
||||
f.addListener(
|
||||
new ChannelFutureListener {
|
||||
def operationComplete(future: ChannelFuture) {
|
||||
if (future.isCancelled || !future.isSuccess) {
|
||||
netty.notifyListeners(RemoteClientWriteFailed(request, future.getCause, netty, remoteAddress))
|
||||
}
|
||||
}
|
||||
})
|
||||
// Check if we should back off
|
||||
if (!channel.isWritable) {
|
||||
val backoff = netty.settings.BackoffTimeout
|
||||
if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
||||
/**
|
||||
* RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node.
|
||||
*/
|
||||
class ActiveRemoteClient private[akka] (
|
||||
netty: NettyRemoteTransport,
|
||||
remoteAddress: Address,
|
||||
localAddress: Address)
|
||||
extends RemoteClient(netty, remoteAddress) {
|
||||
|
||||
import netty.settings
|
||||
|
||||
//TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
|
||||
@volatile
|
||||
private var bootstrap: ClientBootstrap = _
|
||||
@volatile
|
||||
private var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var executionHandler: ExecutionHandler = _
|
||||
|
||||
@volatile
|
||||
private var reconnectionTimeWindowStart = 0L
|
||||
|
||||
def notifyListeners(msg: RemoteLifeCycleEvent): Unit = netty.notifyListeners(msg)
|
||||
|
||||
def currentChannel = connection.getChannel
|
||||
|
||||
/**
|
||||
* Connect to remote server.
|
||||
*/
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
|
||||
|
||||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (settings.SecureCookie.nonEmpty) handshake.setCookie(settings.SecureCookie.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(localAddress.system)
|
||||
.setHostname(localAddress.host.get)
|
||||
.setPort(localAddress.port.get)
|
||||
.build)
|
||||
connection.getChannel.write(netty.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
def attemptReconnect(): Boolean = {
|
||||
val remoteIP = InetAddress.getByName(remoteAddress.host.get)
|
||||
log.debug("Remote client reconnecting to [{}|{}]", remoteAddress, remoteIP)
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get))
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
runSwitch switchOn {
|
||||
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
|
||||
|
||||
executionHandler = new ExecutionHandler(netty.executor)
|
||||
|
||||
val b = new ClientBootstrap(netty.clientChannelFactory)
|
||||
b.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, b, executionHandler, remoteAddress, this))
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("keepAlive", true)
|
||||
b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis)
|
||||
bootstrap = b
|
||||
|
||||
val remoteIP = InetAddress.getByName(remoteAddress.host.get)
|
||||
log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP)
|
||||
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get))
|
||||
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
true
|
||||
}
|
||||
} match {
|
||||
case true ⇒ true
|
||||
case false if reconnectIfAlreadyConnected ⇒
|
||||
connection.getChannel.close()
|
||||
openChannels.remove(connection.getChannel)
|
||||
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
attemptReconnect()
|
||||
|
||||
case false ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
notifyListeners(RemoteClientShutdown(netty, remoteAddress))
|
||||
try {
|
||||
if ((connection ne null) && (connection.getChannel ne null))
|
||||
connection.getChannel.close()
|
||||
} finally {
|
||||
try {
|
||||
if (openChannels ne null) openChannels.close.awaitUninterruptibly()
|
||||
} finally {
|
||||
connection = null
|
||||
executionHandler = null
|
||||
}
|
||||
}
|
||||
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
|
||||
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
|
||||
if (reconnectionTimeWindowStart == 0L) {
|
||||
reconnectionTimeWindowStart = System.currentTimeMillis
|
||||
true
|
||||
} else {
|
||||
val timeLeft = (settings.ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
|
||||
if (timeLeft)
|
||||
log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
|
||||
|
||||
timeLeft
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class ActiveRemoteClientHandler(
|
||||
val name: String,
|
||||
val bootstrap: ClientBootstrap,
|
||||
val remoteAddress: Address,
|
||||
val timer: HashedWheelTimer,
|
||||
val client: ActiveRemoteClient)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() }
|
||||
}, 0, TimeUnit.MILLISECONDS)
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
|
||||
try {
|
||||
event.getMessage match {
|
||||
case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒
|
||||
val rcp = arp.getInstruction
|
||||
rcp.getCommandType match {
|
||||
case CommandType.SHUTDOWN ⇒ runOnceNow { client.netty.shutdownClientConnection(remoteAddress) }
|
||||
case _ ⇒ //Ignore others
|
||||
}
|
||||
|
||||
case arp: AkkaRemoteProtocol if arp.hasMessage ⇒
|
||||
client.netty.receiveMessage(new RemoteMessage(arp.getMessage, client.netty.system))
|
||||
|
||||
case other ⇒
|
||||
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.netty, client.remoteAddress)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
|
||||
if (client.isWithinReconnectionTimeWindow) {
|
||||
timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) =
|
||||
if (client.isRunning) {
|
||||
client.openChannels.remove(event.getChannel)
|
||||
client.connect(reconnectIfAlreadyConnected = true)
|
||||
}
|
||||
}, client.netty.settings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS)
|
||||
} else runOnceNow {
|
||||
client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
}
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
try {
|
||||
client.notifyListeners(RemoteClientConnected(client.netty, client.remoteAddress))
|
||||
client.resetReconnectionTimeWindow
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
client.notifyListeners(RemoteClientDisconnected(client.netty, client.remoteAddress))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
val cause = event.getCause
|
||||
if (cause ne null) {
|
||||
client.notifyListeners(RemoteClientError(cause, client.netty, client.remoteAddress))
|
||||
cause match {
|
||||
case e: ReadTimeoutException ⇒
|
||||
runOnceNow {
|
||||
client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
case e: Exception ⇒ event.getChannel.close()
|
||||
}
|
||||
|
||||
} else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
class ActiveRemoteClientPipelineFactory(
|
||||
name: String,
|
||||
bootstrap: ClientBootstrap,
|
||||
executionHandler: ExecutionHandler,
|
||||
remoteAddress: Address,
|
||||
client: ActiveRemoteClient) extends ChannelPipelineFactory {
|
||||
|
||||
import client.netty.settings
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val timeout = new ReadTimeoutHandler(client.netty.timer, settings.ReadTimeout.length, settings.ReadTimeout.unit)
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(client.netty)
|
||||
val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.netty.timer, client)
|
||||
|
||||
new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient)
|
||||
}
|
||||
}
|
||||
|
||||
class PassiveRemoteClient(val currentChannel: Channel,
|
||||
netty: NettyRemoteTransport,
|
||||
remoteAddress: Address)
|
||||
extends RemoteClient(netty, remoteAddress) {
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn {
|
||||
netty.notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
}
|
||||
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
netty.notifyListeners(RemoteClientShutdown(netty, remoteAddress))
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1,412 +1,113 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote.netty
|
||||
|
||||
import akka.actor.{ ActorRef, simpleName }
|
||||
import akka.remote._
|
||||
import RemoteProtocol._
|
||||
import akka.util._
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.{ NioServerSocketChannelFactory, NioClientSocketChannelFactory }
|
||||
import org.jboss.netty.bootstrap.{ ServerBootstrap, ClientBootstrap }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
|
||||
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
|
||||
import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
|
||||
import scala.collection.mutable.HashMap
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic._
|
||||
import akka.AkkaException
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
import java.util.concurrent.Executors
|
||||
|
||||
import scala.collection.mutable.HashMap
|
||||
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder }
|
||||
import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor
|
||||
import org.jboss.netty.util.HashedWheelTimer
|
||||
|
||||
import akka.actor.{ Address, ActorSystemImpl, ActorRef }
|
||||
import akka.dispatch.MonitorableThreadFactory
|
||||
import akka.event.Logging
|
||||
import org.jboss.netty.channel._
|
||||
import akka.actor.ActorSystemImpl
|
||||
import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor }
|
||||
import java.util.concurrent._
|
||||
import locks.ReentrantReadWriteLock
|
||||
|
||||
class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the abstract baseclass for netty remote clients, currently there's only an
|
||||
* ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that
|
||||
* reuses an already established connection.
|
||||
*/
|
||||
abstract class RemoteClient private[akka] (
|
||||
val remoteSupport: NettyRemoteSupport,
|
||||
val remoteAddress: RemoteNettyAddress) {
|
||||
|
||||
val log = Logging(remoteSupport.system, "RemoteClient")
|
||||
|
||||
val name = simpleName(this) + "@" + remoteAddress
|
||||
|
||||
private[remote] val runSwitch = new Switch()
|
||||
|
||||
private[remote] def isRunning = runSwitch.isOn
|
||||
|
||||
protected def currentChannel: Channel
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean
|
||||
|
||||
def shutdown(): Boolean
|
||||
|
||||
def isBoundTo(address: RemoteNettyAddress): Boolean = remoteAddress == address
|
||||
|
||||
/**
|
||||
* Converts the message to the wireprotocol and sends the message across the wire
|
||||
*/
|
||||
def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) {
|
||||
if (remoteSupport.remote.remoteSettings.LogSentMessages)
|
||||
log.debug("Sending message [{}] from [{}] to [{}]", message, senderOption, recipient)
|
||||
|
||||
send((message, senderOption, recipient))
|
||||
} else {
|
||||
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress)
|
||||
remoteSupport.notifyListeners(RemoteClientError(exception, remoteSupport, remoteAddress))
|
||||
throw exception
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the message across the wire
|
||||
*/
|
||||
private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = {
|
||||
try {
|
||||
val channel = currentChannel
|
||||
val f = channel.write(request)
|
||||
f.addListener(
|
||||
new ChannelFutureListener {
|
||||
def operationComplete(future: ChannelFuture) {
|
||||
if (future.isCancelled || !future.isSuccess) {
|
||||
remoteSupport.notifyListeners(RemoteClientWriteFailed(request, future.getCause, remoteSupport, remoteAddress))
|
||||
}
|
||||
}
|
||||
})
|
||||
// Check if we should back off
|
||||
if (!channel.isWritable) {
|
||||
val backoff = remoteSupport.remote.remoteSettings.BackoffTimeout
|
||||
if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteClientError(e, remoteSupport, remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
||||
class PassiveRemoteClient(val currentChannel: Channel,
|
||||
remoteSupport: NettyRemoteSupport,
|
||||
remoteAddress: RemoteNettyAddress)
|
||||
extends RemoteClient(remoteSupport, remoteAddress) {
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn {
|
||||
remoteSupport.notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress))
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
}
|
||||
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
remoteSupport.notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress))
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node.
|
||||
*/
|
||||
class ActiveRemoteClient private[akka] (
|
||||
remoteSupport: NettyRemoteSupport,
|
||||
remoteAddress: RemoteNettyAddress,
|
||||
localAddress: RemoteSystemAddress[ParsedTransportAddress],
|
||||
val loader: Option[ClassLoader] = None)
|
||||
extends RemoteClient(remoteSupport, remoteAddress) {
|
||||
|
||||
if (remoteAddress.ip.isEmpty) throw new java.net.UnknownHostException(remoteAddress.host)
|
||||
|
||||
import remoteSupport.clientSettings._
|
||||
|
||||
//TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
|
||||
@volatile
|
||||
private var bootstrap: ClientBootstrap = _
|
||||
@volatile
|
||||
private var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var executionHandler: ExecutionHandler = _
|
||||
|
||||
@volatile
|
||||
private var reconnectionTimeWindowStart = 0L
|
||||
|
||||
def notifyListeners(msg: RemoteLifeCycleEvent): Unit = remoteSupport.notifyListeners(msg)
|
||||
|
||||
def currentChannel = connection.getChannel
|
||||
|
||||
/**
|
||||
* Connect to remote server.
|
||||
*/
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
|
||||
|
||||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (SecureCookie.nonEmpty) handshake.setCookie(SecureCookie.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(localAddress.system)
|
||||
.setHostname(localAddress.transport.host)
|
||||
.setPort(localAddress.transport.port)
|
||||
.build)
|
||||
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
def attemptReconnect(): Boolean = {
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port))
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
runSwitch switchOn {
|
||||
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
|
||||
|
||||
executionHandler = new ExecutionHandler(remoteSupport.executor)
|
||||
|
||||
bootstrap = new ClientBootstrap(remoteSupport.clientChannelFactory)
|
||||
bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this))
|
||||
bootstrap.setOption("tcpNoDelay", true)
|
||||
bootstrap.setOption("keepAlive", true)
|
||||
bootstrap.setOption("connectTimeoutMillis", ConnectionTimeout.toMillis)
|
||||
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port))
|
||||
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress))
|
||||
true
|
||||
}
|
||||
} match {
|
||||
case true ⇒ true
|
||||
case false if reconnectIfAlreadyConnected ⇒
|
||||
connection.getChannel.close()
|
||||
openChannels.remove(connection.getChannel)
|
||||
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
attemptReconnect()
|
||||
|
||||
case false ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress))
|
||||
try {
|
||||
if ((connection ne null) && (connection.getChannel ne null))
|
||||
connection.getChannel.close()
|
||||
} finally {
|
||||
try {
|
||||
if (openChannels ne null) openChannels.close.awaitUninterruptibly()
|
||||
} finally {
|
||||
connection = null
|
||||
executionHandler = null
|
||||
}
|
||||
}
|
||||
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
|
||||
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
|
||||
if (reconnectionTimeWindowStart == 0L) {
|
||||
reconnectionTimeWindowStart = System.currentTimeMillis
|
||||
true
|
||||
} else {
|
||||
val timeLeft = (ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
|
||||
if (timeLeft)
|
||||
log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
|
||||
|
||||
timeLeft
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
|
||||
}
|
||||
|
||||
class ActiveRemoteClientPipelineFactory(
|
||||
name: String,
|
||||
bootstrap: ClientBootstrap,
|
||||
executionHandler: ExecutionHandler,
|
||||
remoteAddress: RemoteNettyAddress,
|
||||
client: ActiveRemoteClient) extends ChannelPipelineFactory {
|
||||
|
||||
import client.remoteSupport.clientSettings._
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val timeout = new ReadTimeoutHandler(client.remoteSupport.timer, ReadTimeout.length, ReadTimeout.unit)
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(client.remoteSupport)
|
||||
val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.remoteSupport.timer, client)
|
||||
|
||||
new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient)
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteMessageEncoder(remoteSupport: NettyRemoteSupport) extends ProtobufEncoder {
|
||||
override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = {
|
||||
msg match {
|
||||
case (message: Any, sender: Option[_], recipient: ActorRef) ⇒
|
||||
super.encode(ctx, channel,
|
||||
remoteSupport.createMessageSendEnvelope(
|
||||
remoteSupport.createRemoteMessageProtocolBuilder(
|
||||
recipient,
|
||||
message,
|
||||
sender.asInstanceOf[Option[ActorRef]]).build))
|
||||
case _ ⇒ super.encode(ctx, channel, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class ActiveRemoteClientHandler(
|
||||
val name: String,
|
||||
val bootstrap: ClientBootstrap,
|
||||
val remoteAddress: RemoteNettyAddress,
|
||||
val timer: HashedWheelTimer,
|
||||
val client: ActiveRemoteClient)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() }
|
||||
}, 0, TimeUnit.MILLISECONDS)
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
|
||||
try {
|
||||
event.getMessage match {
|
||||
case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒
|
||||
val rcp = arp.getInstruction
|
||||
rcp.getCommandType match {
|
||||
case CommandType.SHUTDOWN ⇒ runOnceNow { client.remoteSupport.shutdownClientConnection(remoteAddress) }
|
||||
case _ ⇒ //Ignore others
|
||||
}
|
||||
|
||||
case arp: AkkaRemoteProtocol if arp.hasMessage ⇒
|
||||
client.remoteSupport.receiveMessage(new RemoteMessage(arp.getMessage, client.remoteSupport.system, client.loader))
|
||||
|
||||
case other ⇒
|
||||
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.remoteSupport, client.remoteAddress)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
|
||||
if (client.isWithinReconnectionTimeWindow) {
|
||||
timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) =
|
||||
if (client.isRunning) {
|
||||
client.openChannels.remove(event.getChannel)
|
||||
client.connect(reconnectIfAlreadyConnected = true)
|
||||
}
|
||||
}, client.remoteSupport.clientSettings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS)
|
||||
} else runOnceNow {
|
||||
client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
}
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
try {
|
||||
client.notifyListeners(RemoteClientConnected(client.remoteSupport, client.remoteAddress))
|
||||
client.resetReconnectionTimeWindow
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
client.notifyListeners(RemoteClientDisconnected(client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
val cause = event.getCause
|
||||
if (cause ne null) {
|
||||
client.notifyListeners(RemoteClientError(cause, client.remoteSupport, client.remoteAddress))
|
||||
cause match {
|
||||
case e: ReadTimeoutException ⇒
|
||||
runOnceNow {
|
||||
client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
case e: Exception ⇒ event.getChannel.close()
|
||||
}
|
||||
|
||||
} else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
import akka.remote.RemoteProtocol.AkkaRemoteProtocol
|
||||
import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef }
|
||||
|
||||
/**
|
||||
* Provides the implementation of the Netty remote support
|
||||
*/
|
||||
class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val address: RemoteSystemAddress[RemoteNettyAddress])
|
||||
extends RemoteSupport[RemoteNettyAddress](_system) with RemoteMarshallingOps {
|
||||
val log = Logging(system, "NettyRemoteSupport")
|
||||
class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: ActorSystemImpl, val provider: RemoteActorRefProvider)
|
||||
extends RemoteTransport with RemoteMarshallingOps {
|
||||
|
||||
val serverSettings = remote.remoteSettings.serverSettings
|
||||
val clientSettings = remote.remoteSettings.clientSettings
|
||||
val threadFactory = _system.threadFactory.copy(_system.threadFactory.name + "-remote")
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory)
|
||||
val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName)
|
||||
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory)
|
||||
|
||||
val executor = new OrderedMemoryAwareThreadPoolExecutor(
|
||||
serverSettings.ExecutionPoolSize,
|
||||
serverSettings.MaxChannelMemorySize,
|
||||
serverSettings.MaxTotalMemorySize,
|
||||
serverSettings.ExecutionPoolKeepAlive.length,
|
||||
serverSettings.ExecutionPoolKeepAlive.unit,
|
||||
threadFactory)
|
||||
settings.ExecutionPoolSize,
|
||||
settings.MaxChannelMemorySize,
|
||||
settings.MaxTotalMemorySize,
|
||||
settings.ExecutionPoolKeepalive.length,
|
||||
settings.ExecutionPoolKeepalive.unit,
|
||||
system.threadFactory)
|
||||
|
||||
val clientChannelFactory = new NioClientSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(threadFactory),
|
||||
Executors.newCachedThreadPool(threadFactory))
|
||||
Executors.newCachedThreadPool(system.threadFactory),
|
||||
Executors.newCachedThreadPool(system.threadFactory))
|
||||
|
||||
private val remoteClients = new HashMap[RemoteNettyAddress, RemoteClient]
|
||||
private val remoteClients = new HashMap[Address, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
||||
override protected def useUntrustedMode = serverSettings.UntrustedMode
|
||||
override protected def useUntrustedMode = remoteSettings.UntrustedMode
|
||||
|
||||
val server = try new NettyRemoteServer(this) catch {
|
||||
case ex ⇒ shutdown(); throw ex
|
||||
}
|
||||
|
||||
// the address is set in start() or from the RemoteServerHandler, whichever comes first
|
||||
private val _address = new AtomicReference[Address]
|
||||
private[akka] def setAddressFromChannel(ch: Channel) = {
|
||||
val addr = ch.getLocalAddress match {
|
||||
case sa: InetSocketAddress ⇒ sa
|
||||
case x ⇒ throw new RemoteTransportException("unknown local address type " + x.getClass, null)
|
||||
}
|
||||
_address.compareAndSet(null, Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(addr.getPort)))
|
||||
}
|
||||
|
||||
def address = _address.get
|
||||
|
||||
val log = Logging(system.eventStream, "NettyRemoteTransport(" + address + ")")
|
||||
|
||||
def start(): Unit = {
|
||||
server.start()
|
||||
setAddressFromChannel(server.channel)
|
||||
}
|
||||
|
||||
def shutdown(): Unit = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients foreach {
|
||||
case (_, client) ⇒ try client.shutdown() catch {
|
||||
case e ⇒ log.error(e, "failure while shutting down [{}]", client)
|
||||
}
|
||||
}
|
||||
remoteClients.clear()
|
||||
} finally {
|
||||
clientsLock.writeLock().unlock()
|
||||
try {
|
||||
if (server != null) server.shutdown()
|
||||
} finally {
|
||||
try {
|
||||
timer.stop()
|
||||
} finally {
|
||||
try {
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
} finally {
|
||||
executor.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected[akka] def send(
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef,
|
||||
loader: Option[ClassLoader]): Unit = {
|
||||
recipient: RemoteActorRef): Unit = {
|
||||
|
||||
val recipientAddress = recipient.path.address match {
|
||||
case RemoteSystemAddress(sys, transport) ⇒
|
||||
transport match {
|
||||
case x: RemoteNettyAddress ⇒ x
|
||||
case _ ⇒ throw new IllegalArgumentException("invoking NettyRemoteSupport.send with foreign target address " + transport)
|
||||
}
|
||||
}
|
||||
val recipientAddress = recipient.path.address
|
||||
|
||||
clientsLock.readLock.lock
|
||||
try {
|
||||
|
|
@ -421,7 +122,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
//Recheck for addition, race between upgrades
|
||||
case Some(client) ⇒ client //If already populated by other writer
|
||||
case None ⇒ //Populate map
|
||||
val client = new ActiveRemoteClient(this, recipientAddress, remote.remoteAddress, loader)
|
||||
val client = new ActiveRemoteClient(this, recipientAddress, address)
|
||||
client.connect()
|
||||
remoteClients += recipientAddress -> client
|
||||
client
|
||||
|
|
@ -439,7 +140,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def bindClient(remoteAddress: RemoteNettyAddress, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = {
|
||||
def bindClient(remoteAddress: Address, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
if (putIfAbsent && remoteClients.contains(remoteAddress)) false
|
||||
|
|
@ -453,7 +154,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def unbindClient(remoteAddress: RemoteNettyAddress): Unit = {
|
||||
def unbindClient(remoteAddress: Address): Unit = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients.foreach { case (k, v) ⇒ if (v.isBoundTo(remoteAddress)) { v.shutdown(); remoteClients.remove(k) } }
|
||||
|
|
@ -462,7 +163,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def shutdownClientConnection(remoteAddress: RemoteNettyAddress): Boolean = {
|
||||
def shutdownClientConnection(remoteAddress: Address): Boolean = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients.remove(remoteAddress) match {
|
||||
|
|
@ -474,7 +175,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def restartClientConnection(remoteAddress: RemoteNettyAddress): Boolean = {
|
||||
def restartClientConnection(remoteAddress: Address): Boolean = {
|
||||
clientsLock.readLock().lock()
|
||||
try {
|
||||
remoteClients.get(remoteAddress) match {
|
||||
|
|
@ -486,229 +187,24 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Server section
|
||||
*/
|
||||
@volatile
|
||||
private var currentServer: NettyRemoteServer = _
|
||||
}
|
||||
|
||||
def name = currentServer match {
|
||||
case null ⇒ remote.remoteAddress.toString
|
||||
case server ⇒ server.name
|
||||
}
|
||||
|
||||
private val _isRunning = new Switch(false)
|
||||
|
||||
def isRunning = _isRunning.isOn
|
||||
|
||||
def start(loader: Option[ClassLoader] = None): Unit =
|
||||
_isRunning switchOn { currentServer = new NettyRemoteServer(this, loader, address) }
|
||||
|
||||
/**
|
||||
* Common section
|
||||
*/
|
||||
|
||||
def shutdown(): Unit = _isRunning switchOff {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients foreach { case (_, client) ⇒ client.shutdown() }
|
||||
remoteClients.clear()
|
||||
} finally {
|
||||
clientsLock.writeLock().unlock()
|
||||
try {
|
||||
val s = currentServer
|
||||
currentServer = null
|
||||
s.shutdown()
|
||||
} finally {
|
||||
try {
|
||||
timer.stop()
|
||||
} finally {
|
||||
try {
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
} finally {
|
||||
executor.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends ProtobufEncoder {
|
||||
override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = {
|
||||
msg match {
|
||||
case (message: Any, sender: Option[_], recipient: ActorRef) ⇒
|
||||
super.encode(ctx, channel,
|
||||
remoteSupport.createMessageSendEnvelope(
|
||||
remoteSupport.createRemoteMessageProtocolBuilder(
|
||||
recipient,
|
||||
message,
|
||||
sender.asInstanceOf[Option[ActorRef]]).build))
|
||||
case _ ⇒ super.encode(ctx, channel, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class NettyRemoteServer(
|
||||
val remoteSupport: NettyRemoteSupport,
|
||||
val loader: Option[ClassLoader],
|
||||
val address: RemoteSystemAddress[RemoteNettyAddress]) {
|
||||
val log = Logging(remoteSupport.system, "NettyRemoteServer")
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
if (address.transport.ip.isEmpty) throw new java.net.UnknownHostException(address.transport.host)
|
||||
|
||||
val name = "NettyRemoteServer@" + address
|
||||
|
||||
private val factory = new NioServerSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(remoteSupport.threadFactory),
|
||||
Executors.newCachedThreadPool(remoteSupport.threadFactory))
|
||||
|
||||
private val executionHandler = new ExecutionHandler(remoteSupport.executor)
|
||||
|
||||
// group of open channels, used for clean-up
|
||||
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
|
||||
|
||||
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executionHandler, loader, remoteSupport)
|
||||
private val bootstrap: ServerBootstrap = {
|
||||
val b = new ServerBootstrap(factory)
|
||||
b.setPipelineFactory(pipelineFactory)
|
||||
b.setOption("backlog", Backlog)
|
||||
b.setOption("child.tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
b.setOption("child.reuseAddress", true)
|
||||
b
|
||||
}
|
||||
|
||||
openChannels.add(bootstrap.bind(new InetSocketAddress(address.transport.ip.get, address.transport.port)))
|
||||
|
||||
def shutdown() {
|
||||
try {
|
||||
val shutdownSignal = {
|
||||
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
|
||||
b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(address.system)
|
||||
.setHostname(address.transport.host)
|
||||
.setPort(address.transport.port)
|
||||
.build)
|
||||
if (SecureCookie.nonEmpty)
|
||||
b.setCookie(SecureCookie.get)
|
||||
b.build
|
||||
}
|
||||
openChannels.write(remoteSupport.createControlEnvelope(shutdownSignal)).awaitUninterruptibly
|
||||
openChannels.disconnect
|
||||
openChannels.close.awaitUninterruptibly
|
||||
bootstrap.releaseExternalResources()
|
||||
remoteSupport.notifyListeners(RemoteServerShutdown(remoteSupport))
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteServerPipelineFactory(
|
||||
val name: String,
|
||||
val openChannels: ChannelGroup,
|
||||
val executionHandler: ExecutionHandler,
|
||||
val loader: Option[ClassLoader],
|
||||
val remoteSupport: NettyRemoteSupport) extends ChannelPipelineFactory {
|
||||
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(remoteSupport)
|
||||
|
||||
val authenticator = if (RequireCookie) new RemoteServerAuthenticationHandler(SecureCookie) :: Nil else Nil
|
||||
val remoteServer = new RemoteServerHandler(name, openChannels, loader, remoteSupport)
|
||||
val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil
|
||||
new StaticChannelPipeline(stages: _*)
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler {
|
||||
val authenticated = new AnyRef
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match {
|
||||
case None ⇒ ctx.sendUpstream(event)
|
||||
case Some(cookie) ⇒
|
||||
ctx.getAttachment match {
|
||||
case `authenticated` ⇒ ctx.sendUpstream(event)
|
||||
case null ⇒ event.getMessage match {
|
||||
case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒
|
||||
val instruction = remoteProtocol.getInstruction
|
||||
instruction.getCookie match {
|
||||
case `cookie` ⇒
|
||||
ctx.setAttachment(authenticated)
|
||||
ctx.sendUpstream(event)
|
||||
case _ ⇒
|
||||
throw new SecurityException(
|
||||
"The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie")
|
||||
}
|
||||
case _ ⇒
|
||||
throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerHandler(
|
||||
val name: String,
|
||||
val openChannels: ChannelGroup,
|
||||
val applicationLoader: Option[ClassLoader],
|
||||
val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
val log = Logging(remoteSupport.system, "RemoteServerHandler")
|
||||
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
/**
|
||||
* ChannelOpen overridden to store open channels for a clean postStop of a node.
|
||||
* If a channel is closed before, it is automatically removed from the open channels group.
|
||||
*/
|
||||
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
remoteSupport.notifyListeners(RemoteServerClientConnected(remoteSupport, clientAddress))
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
remoteSupport.notifyListeners(RemoteServerClientDisconnected(remoteSupport, clientAddress))
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match {
|
||||
case s @ Some(address) ⇒
|
||||
if (UsePassiveConnections)
|
||||
remoteSupport.unbindClient(address)
|
||||
remoteSupport.notifyListeners(RemoteServerClientClosed(remoteSupport, s))
|
||||
case None ⇒
|
||||
remoteSupport.notifyListeners(RemoteServerClientClosed[RemoteNettyAddress](remoteSupport, None))
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
|
||||
event.getMessage match {
|
||||
case remote: AkkaRemoteProtocol if remote.hasMessage ⇒
|
||||
remoteSupport.receiveMessage(new RemoteMessage(remote.getMessage, remoteSupport.system, applicationLoader))
|
||||
|
||||
case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒
|
||||
val instruction = remote.getInstruction
|
||||
instruction.getCommandType match {
|
||||
case CommandType.CONNECT if UsePassiveConnections ⇒
|
||||
val origin = instruction.getOrigin
|
||||
val inbound = RemoteNettyAddress(origin.getHostname, origin.getPort)
|
||||
val client = new PassiveRemoteClient(event.getChannel, remoteSupport, inbound)
|
||||
remoteSupport.bindClient(inbound, client)
|
||||
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
||||
case _ ⇒ //Unknown command
|
||||
}
|
||||
case _ ⇒ //ignore
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
remoteSupport.notifyListeners(RemoteServerError(event.getCause, remoteSupport))
|
||||
event.getChannel.close()
|
||||
}
|
||||
|
||||
private def getClientAddress(c: Channel): Option[RemoteNettyAddress] =
|
||||
c.getRemoteAddress match {
|
||||
case inet: InetSocketAddress ⇒ Some(RemoteNettyAddress(inet.getHostName, Some(inet.getAddress), inet.getPort))
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
|
||||
|
||||
class DefaultDisposableChannelGroup(name: String) extends DefaultChannelGroup(name) {
|
||||
protected val guard = new ReentrantReadWriteLock
|
||||
|
|
|
|||
210
akka-remote/src/main/scala/akka/remote/netty/Server.scala
Normal file
210
akka-remote/src/main/scala/akka/remote/netty/Server.scala
Normal file
|
|
@ -0,0 +1,210 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.Executors
|
||||
import scala.Option.option2Iterable
|
||||
import org.jboss.netty.bootstrap.ServerBootstrap
|
||||
import org.jboss.netty.channel.ChannelHandler.Sharable
|
||||
import org.jboss.netty.channel.group.ChannelGroup
|
||||
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelHandler, Channel }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import akka.event.Logging
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteServerStarted, RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage }
|
||||
import akka.actor.Address
|
||||
import java.net.InetAddress
|
||||
import akka.actor.ActorSystemImpl
|
||||
import org.jboss.netty.channel.ChannelLocal
|
||||
import org.jboss.netty.channel.ChannelEvent
|
||||
|
||||
class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
||||
|
||||
import netty.settings
|
||||
|
||||
val ip = InetAddress.getByName(settings.Hostname)
|
||||
|
||||
private val factory = new NioServerSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(netty.system.threadFactory),
|
||||
Executors.newCachedThreadPool(netty.system.threadFactory))
|
||||
|
||||
private val executionHandler = new ExecutionHandler(netty.executor)
|
||||
|
||||
// group of open channels, used for clean-up
|
||||
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
|
||||
|
||||
private val bootstrap = {
|
||||
val b = new ServerBootstrap(factory)
|
||||
b.setPipelineFactory(new RemoteServerPipelineFactory(openChannels, executionHandler, netty))
|
||||
b.setOption("backlog", settings.Backlog)
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
b.setOption("reuseAddress", true)
|
||||
b
|
||||
}
|
||||
|
||||
@volatile
|
||||
private[akka] var channel: Channel = _
|
||||
|
||||
def start(): Unit = {
|
||||
channel = bootstrap.bind(new InetSocketAddress(ip, settings.DesiredPortFromConfig))
|
||||
openChannels.add(channel)
|
||||
netty.notifyListeners(RemoteServerStarted(netty))
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
try {
|
||||
val shutdownSignal = {
|
||||
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
|
||||
b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(settings.systemName)
|
||||
.setHostname(settings.Hostname)
|
||||
.setPort(settings.DesiredPortFromConfig)
|
||||
.build)
|
||||
if (settings.SecureCookie.nonEmpty)
|
||||
b.setCookie(settings.SecureCookie.get)
|
||||
b.build
|
||||
}
|
||||
openChannels.write(netty.createControlEnvelope(shutdownSignal)).awaitUninterruptibly
|
||||
openChannels.disconnect
|
||||
openChannels.close.awaitUninterruptibly
|
||||
bootstrap.releaseExternalResources()
|
||||
netty.notifyListeners(RemoteServerShutdown(netty))
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteServerPipelineFactory(
|
||||
val openChannels: ChannelGroup,
|
||||
val executionHandler: ExecutionHandler,
|
||||
val netty: NettyRemoteTransport) extends ChannelPipelineFactory {
|
||||
|
||||
import netty.settings
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(netty)
|
||||
|
||||
val authenticator = if (settings.RequireCookie) new RemoteServerAuthenticationHandler(settings.SecureCookie) :: Nil else Nil
|
||||
val remoteServer = new RemoteServerHandler(openChannels, netty)
|
||||
val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil
|
||||
new StaticChannelPipeline(stages: _*)
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler {
|
||||
val authenticated = new AnyRef
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match {
|
||||
case None ⇒ ctx.sendUpstream(event)
|
||||
case Some(cookie) ⇒
|
||||
ctx.getAttachment match {
|
||||
case `authenticated` ⇒ ctx.sendUpstream(event)
|
||||
case null ⇒ event.getMessage match {
|
||||
case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒
|
||||
val instruction = remoteProtocol.getInstruction
|
||||
instruction.getCookie match {
|
||||
case `cookie` ⇒
|
||||
ctx.setAttachment(authenticated)
|
||||
ctx.sendUpstream(event)
|
||||
case _ ⇒
|
||||
throw new SecurityException(
|
||||
"The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie")
|
||||
}
|
||||
case _ ⇒
|
||||
throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ChannelLocalSystem extends ChannelLocal[ActorSystemImpl] {
|
||||
override def initialValue(ch: Channel): ActorSystemImpl = null
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerHandler(
|
||||
val openChannels: ChannelGroup,
|
||||
val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
import netty.settings
|
||||
|
||||
private var addressToSet = true
|
||||
|
||||
// TODO look into moving that into onBind or similar, but verify that that is guaranteed to be the first to be called
|
||||
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
|
||||
if (addressToSet) {
|
||||
netty.setAddressFromChannel(event.getChannel)
|
||||
addressToSet = false
|
||||
}
|
||||
super.handleUpstream(ctx, event)
|
||||
}
|
||||
|
||||
/**
|
||||
* ChannelOpen overridden to store open channels for a clean postStop of a node.
|
||||
* If a channel is closed before, it is automatically removed from the open channels group.
|
||||
*/
|
||||
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
netty.notifyListeners(RemoteServerClientConnected(netty, clientAddress))
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
netty.notifyListeners(RemoteServerClientDisconnected(netty, clientAddress))
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match {
|
||||
case s @ Some(address) ⇒
|
||||
if (settings.UsePassiveConnections)
|
||||
netty.unbindClient(address)
|
||||
netty.notifyListeners(RemoteServerClientClosed(netty, s))
|
||||
case None ⇒
|
||||
netty.notifyListeners(RemoteServerClientClosed(netty, None))
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
|
||||
event.getMessage match {
|
||||
case remote: AkkaRemoteProtocol if remote.hasMessage ⇒
|
||||
netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system))
|
||||
|
||||
case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒
|
||||
val instruction = remote.getInstruction
|
||||
instruction.getCommandType match {
|
||||
case CommandType.CONNECT if settings.UsePassiveConnections ⇒
|
||||
val origin = instruction.getOrigin
|
||||
val inbound = Address("akka", origin.getSystem, Some(origin.getHostname), Some(origin.getPort))
|
||||
val client = new PassiveRemoteClient(event.getChannel, netty, inbound)
|
||||
netty.bindClient(inbound, client)
|
||||
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
||||
case _ ⇒ //Unknown command
|
||||
}
|
||||
case _ ⇒ //ignore
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
netty.notifyListeners(RemoteServerError(event.getCause, netty))
|
||||
event.getChannel.close()
|
||||
}
|
||||
|
||||
private def getClientAddress(c: Channel): Option[Address] =
|
||||
c.getRemoteAddress match {
|
||||
case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", Some(inet.getAddress.toString), Some(inet.getPort)))
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
63
akka-remote/src/main/scala/akka/remote/netty/Settings.scala
Normal file
63
akka-remote/src/main/scala/akka/remote/netty/Settings.scala
Normal file
|
|
@ -0,0 +1,63 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit._
|
||||
import java.net.InetAddress
|
||||
import akka.config.ConfigurationException
|
||||
|
||||
class NettySettings(config: Config, val systemName: String) {
|
||||
|
||||
import config._
|
||||
|
||||
val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS)
|
||||
|
||||
val SecureCookie: Option[String] = getString("secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
val RequireCookie = {
|
||||
val requireCookie = getBoolean("require-cookie")
|
||||
if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.netty.secure-cookie'.")
|
||||
requireCookie
|
||||
}
|
||||
|
||||
val UsePassiveConnections = getBoolean("use-passive-connections")
|
||||
|
||||
val ReconnectionTimeWindow = Duration(getMilliseconds("reconnection-time-window"), MILLISECONDS)
|
||||
val ReadTimeout = Duration(getMilliseconds("read-timeout"), MILLISECONDS)
|
||||
val ReconnectDelay = Duration(getMilliseconds("reconnect-delay"), MILLISECONDS)
|
||||
val MessageFrameSize = getBytes("message-frame-size").toInt
|
||||
|
||||
val Hostname = getString("hostname") match {
|
||||
case "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
}
|
||||
val DesiredPortFromConfig = getInt("port")
|
||||
|
||||
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
|
||||
|
||||
val Backlog = getInt("backlog")
|
||||
|
||||
val ExecutionPoolKeepalive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS)
|
||||
|
||||
val ExecutionPoolSize = getInt("execution-pool-size") match {
|
||||
case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val MaxChannelMemorySize = getBytes("max-channel-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-channel-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val MaxTotalMemorySize = getBytes("max-total-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-total-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,8 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka
|
||||
|
||||
package object remote {
|
||||
type TransportsMap = Map[String, (String, Int) ⇒ Either[String, RemoteTransportAddress]]
|
||||
}
|
||||
|
|
@ -12,7 +12,7 @@ import akka.actor.InternalActorRef
|
|||
import akka.actor.Props
|
||||
import akka.config.ConfigurationException
|
||||
import akka.remote.RemoteScope
|
||||
import akka.remote.RemoteAddressExtractor
|
||||
import akka.actor.AddressExtractor
|
||||
|
||||
/**
|
||||
* [[akka.routing.RouterConfig]] implementation for remote deployment on defined
|
||||
|
|
@ -46,7 +46,7 @@ class RemoteRouteeProvider(nodes: Iterable[String], _context: ActorContext, _res
|
|||
// need this iterator as instance variable since Resizer may call createRoutees several times
|
||||
private val nodeAddressIter = {
|
||||
val nodeAddresses = nodes map {
|
||||
case RemoteAddressExtractor(a) ⇒ a
|
||||
case AddressExtractor(a) ⇒ a
|
||||
case x ⇒ throw new ConfigurationException("unparseable remote node " + x)
|
||||
}
|
||||
Stream.continually(nodeAddresses).flatten.iterator
|
||||
|
|
|
|||
|
|
@ -21,8 +21,8 @@ trait AbstractRemoteActorMultiJvmSpec {
|
|||
case (idx, host) =>
|
||||
ConfigFactory.parseString("""
|
||||
akka {
|
||||
remote.server.hostname="%s"
|
||||
remote.server.port = "%d"
|
||||
remote.netty.hostname="%s"
|
||||
remote.netty.port = "%d"
|
||||
}""".format(host, 9990+idx, idx)) withFallback commonConfig
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -32,14 +32,4 @@ abstract class AkkaRemoteSpec(config: Config)
|
|||
extends AkkaSpec(config.withFallback(AkkaRemoteSpec.testConf))
|
||||
with MultiJvmSync {
|
||||
|
||||
/**
|
||||
* Helper function for accessing the underlying remoting.
|
||||
*/
|
||||
def remote: Remote = {
|
||||
system.asInstanceOf[ActorSystemImpl].provider match {
|
||||
case r: RemoteActorRefProvider ⇒ r.remote
|
||||
case _ ⇒ throw new Exception("Remoting is not enabled")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.testkit._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
|
|
@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! self.path.address.hostPort
|
||||
case "identify" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -53,7 +53,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(nodeConfigs(1)
|
|||
val actor = system.actorOf(Props[SomeActor], "service-hello")
|
||||
actor.isInstanceOf[RemoteActorRef] must be(true)
|
||||
|
||||
Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0))
|
||||
Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0))
|
||||
|
||||
barrier("done")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.testkit._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
|
|
@ -10,7 +10,7 @@ object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! self.path.address.hostPort
|
||||
case "identify" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -53,7 +53,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(NewRemoteActorMultiJvmS
|
|||
barrier("start")
|
||||
|
||||
val actor = system.actorOf(Props[SomeActor], "service-hello")
|
||||
Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0))
|
||||
Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0))
|
||||
|
||||
barrier("done")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.dispatch.Await
|
||||
|
|
@ -10,7 +10,7 @@ object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp
|
|||
override def NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -89,7 +89,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RandomRoutedRe
|
|||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).toString
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort
|
||||
replies = replies + (nodeName -> (replies(nodeName) + 1))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.dispatch.Await
|
||||
|
|
@ -11,7 +11,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJ
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -90,7 +90,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RoundRobin
|
|||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).toString
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort
|
||||
|
||||
replies = replies + (nodeName -> (replies(nodeName) + 1))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
|
@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMul
|
|||
override def NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -89,7 +89,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter
|
|||
}
|
||||
|
||||
val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) {
|
||||
case name: String ⇒ (name, 1)
|
||||
case ref: ActorRef ⇒ (ref.path.address.hostPort, 1)
|
||||
}).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) {
|
||||
case (m, (n, c)) ⇒ m + (n -> (m(n) + c))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -2,11 +2,12 @@ package akka.remote
|
|||
|
||||
import java.net.InetSocketAddress
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Address
|
||||
|
||||
class AccrualFailureDetectorSpec extends AkkaSpec {
|
||||
|
||||
"An AccrualFailureDetector" must {
|
||||
val conn = RemoteNettyAddress("localhost", 2552)
|
||||
val conn = Address("akka", "", Some("localhost"), Some(2552))
|
||||
|
||||
"mark node as available after a series of successful heartbeats" in {
|
||||
val fd = new AccrualFailureDetector()
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.remote
|
|||
|
||||
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
||||
|
||||
import akka.remote.netty.NettyRemoteSupport
|
||||
import akka.remote.netty.NettyRemoteTransport
|
||||
import akka.actor.Actor
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.DefaultTimeout
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ object RemoteCommunicationSpec {
|
|||
class RemoteCommunicationSpec extends AkkaSpec("""
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
port = 12345
|
||||
}
|
||||
|
|
@ -48,7 +48,7 @@ akka {
|
|||
|
||||
import RemoteCommunicationSpec._
|
||||
|
||||
val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config)
|
||||
val conf = ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config)
|
||||
val other = ActorSystem("remote_sys", conf)
|
||||
|
||||
val remote = other.actorOf(Props(new Actor {
|
||||
|
|
|
|||
|
|
@ -1,8 +1,13 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.util.duration._
|
||||
import akka.util.Duration
|
||||
import akka.remote.netty.NettyRemoteTransport
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RemoteConfigSpec extends AkkaSpec(
|
||||
|
|
@ -14,51 +19,49 @@ class RemoteConfigSpec extends AkkaSpec(
|
|||
}
|
||||
""") {
|
||||
|
||||
"RemoteExtension" must {
|
||||
"be able to parse remote and cluster config elements" in {
|
||||
"Remoting" must {
|
||||
|
||||
"be able to parse generic remote config elements" in {
|
||||
val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings
|
||||
|
||||
//SharedSettings
|
||||
|
||||
{
|
||||
import settings._
|
||||
|
||||
RemoteTransport must equal("akka.remote.netty.NettyRemoteSupport")
|
||||
BackoffTimeout must equal(0 seconds)
|
||||
LogReceivedMessages must equal(false)
|
||||
LogSentMessages must equal(false)
|
||||
RemoteTransport must be("akka.remote.netty.NettyRemoteTransport")
|
||||
UntrustedMode must be(false)
|
||||
RemoteSystemDaemonAckTimeout must be(30 seconds)
|
||||
|
||||
FailureDetectorThreshold must be(8)
|
||||
FailureDetectorMaxSampleSize must be(1000)
|
||||
|
||||
InitialDelayForGossip must be(5 seconds)
|
||||
GossipFrequency must be(1 second)
|
||||
SeedNodes must be(Set())
|
||||
}
|
||||
|
||||
//ServerSettings
|
||||
"be able to parse Netty config elements" in {
|
||||
val settings =
|
||||
system.asInstanceOf[ExtendedActorSystem]
|
||||
.provider.asInstanceOf[RemoteActorRefProvider]
|
||||
.transport.asInstanceOf[NettyRemoteTransport]
|
||||
.settings
|
||||
import settings._
|
||||
|
||||
{
|
||||
import settings.serverSettings._
|
||||
BackoffTimeout must be(Duration.Zero)
|
||||
SecureCookie must be(None)
|
||||
UsePassiveConnections must equal(true)
|
||||
Port must equal(2552)
|
||||
MessageFrameSize must equal(1048576L)
|
||||
RequireCookie must equal(false)
|
||||
UntrustedMode must equal(false)
|
||||
Backlog must equal(4096)
|
||||
ExecutionPoolKeepAlive must equal(1 minute)
|
||||
ExecutionPoolSize must equal(4)
|
||||
MaxChannelMemorySize must equal(0)
|
||||
MaxTotalMemorySize must equal(0)
|
||||
RequireCookie must be(false)
|
||||
UsePassiveConnections must be(true)
|
||||
Hostname must not be "" // will be set to the local IP
|
||||
DesiredPortFromConfig must be(2552)
|
||||
MessageFrameSize must be(1048576)
|
||||
ConnectionTimeout must be(2 minutes)
|
||||
Backlog must be(4096)
|
||||
ExecutionPoolKeepalive must be(1 minute)
|
||||
ExecutionPoolSize must be(4)
|
||||
MaxChannelMemorySize must be(0)
|
||||
MaxTotalMemorySize must be(0)
|
||||
ReconnectDelay must be(5 seconds)
|
||||
ReadTimeout must be(1 hour)
|
||||
ReconnectionTimeWindow must be(10 minutes)
|
||||
}
|
||||
|
||||
//ClientSettings
|
||||
|
||||
{
|
||||
import settings.clientSettings._
|
||||
SecureCookie must be(None)
|
||||
ReconnectDelay must equal(5 seconds)
|
||||
ReadTimeout must equal(1 hour)
|
||||
ReconnectionTimeWindow must equal(10 minutes)
|
||||
ConnectionTimeout must equal(10 seconds)
|
||||
}
|
||||
|
||||
// TODO cluster config will go into akka-cluster/reference.conf when we enable that module
|
||||
settings.SeedNodes must be('empty)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,14 +16,14 @@ akka {
|
|||
/watchers.remote = "akka://other@127.0.0.1:2666"
|
||||
}
|
||||
}
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = "127.0.0.1"
|
||||
port = 2665
|
||||
port = 0
|
||||
}
|
||||
}
|
||||
""")) with ImplicitSender with DefaultTimeout with DeathWatchSpec {
|
||||
|
||||
val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.server.port=2666").withFallback(system.settings.config))
|
||||
val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.port=2666").withFallback(system.settings.config))
|
||||
|
||||
override def atTermination() {
|
||||
other.shutdown()
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) {
|
|||
service,
|
||||
deployment.get.config,
|
||||
RoundRobinRouter(3),
|
||||
RemoteScope(UnparsedSystemAddress(Some("sys"), UnparsedTransportAddress("akka", "wallace", 2552))))))
|
||||
RemoteScope(Address("akka", "sys", Some("wallace"), Some(2552))))))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import com.typesafe.config._
|
|||
object RemoteRouterSpec {
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self.path
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -20,15 +20,15 @@ object RemoteRouterSpec {
|
|||
class RemoteRouterSpec extends AkkaSpec("""
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
port = 12345
|
||||
port = 0
|
||||
}
|
||||
actor.deployment {
|
||||
/blub {
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka://remote_sys@localhost:12346"]
|
||||
target.nodes = ["akka://remote_sys@localhost:12347"]
|
||||
}
|
||||
/elastic-blub {
|
||||
router = round-robin
|
||||
|
|
@ -36,7 +36,7 @@ akka {
|
|||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
}
|
||||
target.nodes = ["akka://remote_sys@localhost:12346"]
|
||||
target.nodes = ["akka://remote_sys@localhost:12347"]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -44,7 +44,7 @@ akka {
|
|||
|
||||
import RemoteRouterSpec._
|
||||
|
||||
val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config)
|
||||
val conf = ConfigFactory.parseString("akka.remote.netty.port=12347").withFallback(system.settings.config)
|
||||
val other = ActorSystem("remote_sys", conf)
|
||||
|
||||
override def atTermination() {
|
||||
|
|
@ -56,26 +56,26 @@ akka {
|
|||
"deploy its children on remote host driven by configuration" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
"deploy its children on remote host driven by programatic definition" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2),
|
||||
Seq("akka://remote_sys@localhost:12346"))), "blub2")
|
||||
Seq("akka://remote_sys@localhost:12347"))), "blub2")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
"deploy dynamic resizable number of children on remote host driven by configuration" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
58
akka-sbt-plugin/sample/project/Build.scala
Normal file
58
akka-sbt-plugin/sample/project/Build.scala
Normal file
|
|
@ -0,0 +1,58 @@
|
|||
|
||||
import sbt._
|
||||
import Keys._
|
||||
import akka.sbt.AkkaKernelPlugin
|
||||
import akka.sbt.AkkaKernelPlugin.{ Dist, outputDirectory, distJvmOptions}
|
||||
|
||||
object HelloKernelBuild extends Build {
|
||||
val Organization = "akka.sample"
|
||||
val Version = "2.0-SNAPSHOT"
|
||||
val ScalaVersion = "2.9.1"
|
||||
|
||||
lazy val HelloKernel = Project(
|
||||
id = "hello-kernel",
|
||||
base = file("."),
|
||||
settings = defaultSettings ++ AkkaKernelPlugin.distSettings ++ Seq(
|
||||
libraryDependencies ++= Dependencies.helloKernel,
|
||||
distJvmOptions in Dist := "-Xms256M -Xmx1024M",
|
||||
outputDirectory in Dist := file("target/hello-dist")
|
||||
)
|
||||
)
|
||||
|
||||
lazy val buildSettings = Defaults.defaultSettings ++ Seq(
|
||||
organization := Organization,
|
||||
version := Version,
|
||||
scalaVersion := ScalaVersion,
|
||||
crossPaths := false,
|
||||
organizationName := "Typesafe Inc.",
|
||||
organizationHomepage := Some(url("http://www.typesafe.com"))
|
||||
)
|
||||
|
||||
lazy val defaultSettings = buildSettings ++ Seq(
|
||||
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/",
|
||||
|
||||
// compile options
|
||||
scalacOptions ++= Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"),
|
||||
javacOptions ++= Seq("-Xlint:unchecked", "-Xlint:deprecation")
|
||||
|
||||
)
|
||||
}
|
||||
|
||||
object Dependencies {
|
||||
import Dependency._
|
||||
|
||||
val helloKernel = Seq(
|
||||
akkaKernel, akkaSlf4j, logback
|
||||
)
|
||||
}
|
||||
|
||||
object Dependency {
|
||||
// Versions
|
||||
object V {
|
||||
val Akka = "2.0-SNAPSHOT"
|
||||
}
|
||||
|
||||
val akkaKernel = "com.typesafe.akka" % "akka-kernel" % V.Akka
|
||||
val akkaSlf4j = "com.typesafe.akka" % "akka-slf4j" % V.Akka
|
||||
val logback = "ch.qos.logback" % "logback-classic" % "1.0.0"
|
||||
}
|
||||
1
akka-sbt-plugin/sample/project/build.properties
Normal file
1
akka-sbt-plugin/sample/project/build.properties
Normal file
|
|
@ -0,0 +1 @@
|
|||
sbt.version=0.11.2
|
||||
3
akka-sbt-plugin/sample/project/plugins.sbt
Normal file
3
akka-sbt-plugin/sample/project/plugins.sbt
Normal file
|
|
@ -0,0 +1,3 @@
|
|||
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/"
|
||||
|
||||
addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT")
|
||||
4
akka-sbt-plugin/sample/src/main/config/application.conf
Normal file
4
akka-sbt-plugin/sample/src/main/config/application.conf
Normal file
|
|
@ -0,0 +1,4 @@
|
|||
akka {
|
||||
loglevel = INFO
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
}
|
||||
15
akka-sbt-plugin/sample/src/main/config/logback.xml
Normal file
15
akka-sbt-plugin/sample/src/main/config/logback.xml
Normal file
|
|
@ -0,0 +1,15 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
|
||||
<configuration>
|
||||
|
||||
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
|
||||
<layout>
|
||||
<pattern>%date{ISO8601} %-5level %X{akkaSource} %X{sourceThread} - %msg%n</pattern>
|
||||
</layout>
|
||||
</appender>
|
||||
|
||||
<root level="info">
|
||||
<appender-ref ref="STDOUT" />
|
||||
</root>
|
||||
|
||||
</configuration>
|
||||
37
akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala
Normal file
37
akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala
Normal file
|
|
@ -0,0 +1,37 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package sample.kernel.hello
|
||||
|
||||
import akka.actor.{ Actor, ActorSystem, ActorLogging, Props }
|
||||
import akka.kernel.Bootable
|
||||
|
||||
case object Start
|
||||
|
||||
class HelloActor extends Actor with ActorLogging {
|
||||
val worldActor = context.actorOf(Props[WorldActor], name = "world")
|
||||
|
||||
def receive = {
|
||||
case Start ⇒ worldActor ! "Hello"
|
||||
case message: String ⇒
|
||||
log.info("Received message [{}]", message)
|
||||
}
|
||||
}
|
||||
|
||||
class WorldActor extends Actor {
|
||||
def receive = {
|
||||
case message: String ⇒ sender ! (message.toUpperCase + " world!")
|
||||
}
|
||||
}
|
||||
|
||||
class HelloKernel extends Bootable {
|
||||
val system = ActorSystem("hellokernel")
|
||||
|
||||
def startup = {
|
||||
system.actorOf(Props[HelloActor], name = "hello") ! Start
|
||||
}
|
||||
|
||||
def shutdown = {
|
||||
system.shutdown()
|
||||
}
|
||||
}
|
||||
|
|
@ -126,7 +126,7 @@ object AkkaKernelPlugin extends Plugin {
|
|||
"""|#!/bin/sh
|
||||
|
|
||||
|AKKA_HOME="$(cd "$(cd "$(dirname "$0")"; pwd -P)"/..; pwd)"
|
||||
|AKKA_CLASSPATH="$AKKA_HOME/lib/*:$AKKA_HOME/config"
|
||||
|AKKA_CLASSPATH="$AKKA_HOME/config:$AKKA_HOME/lib/*"
|
||||
|JAVA_OPTS="%s"
|
||||
|
|
||||
|java $JAVA_OPTS -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" %s "$@"
|
||||
|
|
@ -135,7 +135,7 @@ object AkkaKernelPlugin extends Plugin {
|
|||
private def distBatScript =
|
||||
"""|@echo off
|
||||
|set AKKA_HOME=%%~dp0..
|
||||
|set AKKA_CLASSPATH=%%AKKA_HOME%%\lib\*;%%AKKA_HOME%%\config
|
||||
|set AKKA_CLASSPATH=%%AKKA_HOME%%\config;%%AKKA_HOME%%\lib\*
|
||||
|set JAVA_OPTS=%s
|
||||
|
|
||||
|java %%JAVA_OPTS%% -cp "%%AKKA_CLASSPATH%%" -Dakka.home="%%AKKA_HOME%%" %s %%*
|
||||
|
|
|
|||
|
|
@ -5,24 +5,22 @@
|
|||
package akka.tutorial.first.java;
|
||||
|
||||
//#imports
|
||||
import akka.actor.Props;
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.InternalActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.UntypedActorFactory;
|
||||
import akka.japi.Creator;
|
||||
import akka.routing.*;
|
||||
import akka.util.Timeout;
|
||||
import akka.routing.RoundRobinRouter;
|
||||
import akka.util.Duration;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
//#imports
|
||||
|
||||
//#app
|
||||
public class Pi {
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
public static void main(String[] args) {
|
||||
Pi pi = new Pi();
|
||||
pi.calculate(4, 10000, 10000);
|
||||
}
|
||||
|
|
@ -61,6 +59,25 @@ public class Pi {
|
|||
return value;
|
||||
}
|
||||
}
|
||||
|
||||
static class PiApproximation {
|
||||
private final double pi;
|
||||
private final Duration duration;
|
||||
|
||||
public PiApproximation(double pi, Duration duration) {
|
||||
this.pi = pi;
|
||||
this.duration = duration;
|
||||
}
|
||||
|
||||
public double getPi() {
|
||||
return pi;
|
||||
}
|
||||
|
||||
public Duration getDuration() {
|
||||
return duration;
|
||||
}
|
||||
}
|
||||
|
||||
//#messages
|
||||
|
||||
//#worker
|
||||
|
|
@ -74,42 +91,42 @@ public class Pi {
|
|||
}
|
||||
return acc;
|
||||
}
|
||||
|
||||
//#calculatePiFor
|
||||
|
||||
public void onReceive(Object message) {
|
||||
if (message instanceof Work) {
|
||||
Work work = (Work) message;
|
||||
double result = calculatePiFor(work.getStart(), work.getNrOfElements());
|
||||
getSender().tell(new Result(result));
|
||||
getSender().tell(new Result(result), getSelf());
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
unhandled(message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#worker
|
||||
|
||||
//#master
|
||||
public static class Master extends UntypedActor {
|
||||
private final int nrOfMessages;
|
||||
private final int nrOfElements;
|
||||
private final CountDownLatch latch;
|
||||
|
||||
private double pi;
|
||||
private int nrOfResults;
|
||||
private long start;
|
||||
private final long start = System.currentTimeMillis();
|
||||
|
||||
private ActorRef router;
|
||||
private final ActorRef listener;
|
||||
private final ActorRef workerRouter;
|
||||
|
||||
public Master(final int nrOfWorkers, int nrOfMessages,
|
||||
int nrOfElements, CountDownLatch latch) {
|
||||
public Master(final int nrOfWorkers, int nrOfMessages, int nrOfElements, ActorRef listener) {
|
||||
this.nrOfMessages = nrOfMessages;
|
||||
this.nrOfElements = nrOfElements;
|
||||
this.latch = latch;
|
||||
this.listener = listener;
|
||||
|
||||
//#create-router
|
||||
router = this.getContext().actorOf(
|
||||
new Props(Worker.class).withRouter(new RoundRobinRouter(nrOfWorkers)),
|
||||
"pi");
|
||||
workerRouter = this.getContext().actorOf(new Props(Worker.class).withRouter(new RoundRobinRouter(nrOfWorkers)),
|
||||
"workerRouter");
|
||||
//#create-router
|
||||
}
|
||||
|
||||
|
|
@ -118,59 +135,63 @@ public class Pi {
|
|||
//#handle-messages
|
||||
if (message instanceof Calculate) {
|
||||
for (int start = 0; start < nrOfMessages; start++) {
|
||||
router.tell(new Work(start, nrOfElements), getSelf());
|
||||
workerRouter.tell(new Work(start, nrOfElements), getSelf());
|
||||
}
|
||||
} else if (message instanceof Result) {
|
||||
Result result = (Result) message;
|
||||
pi += result.getValue();
|
||||
nrOfResults += 1;
|
||||
if (nrOfResults == nrOfMessages) getContext().stop(getSelf());
|
||||
} else throw new IllegalArgumentException("Unknown message [" + message + "]");
|
||||
if (nrOfResults == nrOfMessages) {
|
||||
// Send the result to the listener
|
||||
Duration duration = Duration.create(System.currentTimeMillis() - start, TimeUnit.MILLISECONDS);
|
||||
listener.tell(new PiApproximation(pi, duration), getSelf());
|
||||
// Stops this actor and all its supervised children
|
||||
getContext().stop(getSelf());
|
||||
}
|
||||
} else {
|
||||
unhandled(message);
|
||||
}
|
||||
//#handle-messages
|
||||
}
|
||||
//#master-receive
|
||||
|
||||
@Override
|
||||
public void preStart() {
|
||||
start = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postStop() {
|
||||
System.out.println(String.format(
|
||||
"\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis",
|
||||
pi, (System.currentTimeMillis() - start)));
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
//#master
|
||||
|
||||
//#result-listener
|
||||
public static class Listener extends UntypedActor {
|
||||
public void onReceive(Object message) {
|
||||
if (message instanceof PiApproximation) {
|
||||
PiApproximation approximation = (PiApproximation) message;
|
||||
System.out.println(String.format("\n\tPi approximation: \t\t%s\n\tCalculation time: \t%s",
|
||||
approximation.getPi(), approximation.getDuration()));
|
||||
getContext().system().shutdown();
|
||||
} else {
|
||||
unhandled(message);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#result-listener
|
||||
//#actors-and-messages
|
||||
|
||||
public void calculate(final int nrOfWorkers,
|
||||
final int nrOfElements,
|
||||
final int nrOfMessages)
|
||||
throws Exception {
|
||||
public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages) {
|
||||
// Create an Akka system
|
||||
final ActorSystem system = ActorSystem.create();
|
||||
ActorSystem system = ActorSystem.create("PiSystem");
|
||||
|
||||
// this latch is only plumbing to know when the calculation is completed
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
// create the result listener, which will print the result and shutdown the system
|
||||
final ActorRef listener = system.actorOf(new Props(Listener.class), "listener");
|
||||
|
||||
// create the master
|
||||
ActorRef master = system.actorOf(new Props(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch);
|
||||
return new Master(nrOfWorkers, nrOfMessages, nrOfElements, listener);
|
||||
}
|
||||
}));
|
||||
}), "master");
|
||||
|
||||
// start the calculation
|
||||
master.tell(new Calculate());
|
||||
|
||||
// wait for master to shut down
|
||||
latch.await();
|
||||
|
||||
// Shut down the system
|
||||
system.shutdown();
|
||||
}
|
||||
}
|
||||
//#app
|
||||
|
|
|
|||
|
|
@ -1,6 +0,0 @@
|
|||
akka.actor.deployment {
|
||||
/user/master/pi {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
}
|
||||
}
|
||||
|
|
@ -4,9 +4,10 @@
|
|||
package akka.tutorial.first.scala
|
||||
|
||||
//#imports
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import akka.actor._
|
||||
import akka.routing._
|
||||
import akka.routing.RoundRobinRouter
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
//#imports
|
||||
|
||||
//#app
|
||||
|
|
@ -20,6 +21,7 @@ object Pi extends App {
|
|||
case object Calculate extends PiMessage
|
||||
case class Work(start: Int, nrOfElements: Int) extends PiMessage
|
||||
case class Result(value: Double) extends PiMessage
|
||||
case class PiApproximation(pi: Double, duration: Duration)
|
||||
//#messages
|
||||
|
||||
//#worker
|
||||
|
|
@ -42,66 +44,67 @@ object Pi extends App {
|
|||
//#worker
|
||||
|
||||
//#master
|
||||
class Master(
|
||||
nrOfWorkers: Int, nrOfMessages: Int, nrOfElements: Int, latch: CountDownLatch)
|
||||
class Master(nrOfWorkers: Int, nrOfMessages: Int, nrOfElements: Int, listener: ActorRef)
|
||||
extends Actor {
|
||||
|
||||
var pi: Double = _
|
||||
var nrOfResults: Int = _
|
||||
var start: Long = _
|
||||
val start: Long = System.currentTimeMillis
|
||||
|
||||
//#create-router
|
||||
val router = context.actorOf(
|
||||
Props[Worker].withRouter(RoundRobinRouter(nrOfWorkers)), "pi")
|
||||
val workerRouter = context.actorOf(
|
||||
Props[Worker].withRouter(RoundRobinRouter(nrOfWorkers)), name = "workerRouter")
|
||||
//#create-router
|
||||
|
||||
//#master-receive
|
||||
def receive = {
|
||||
//#handle-messages
|
||||
case Calculate ⇒
|
||||
for (i ← 0 until nrOfMessages) router ! Work(i * nrOfElements, nrOfElements)
|
||||
for (i ← 0 until nrOfMessages) workerRouter ! Work(i * nrOfElements, nrOfElements)
|
||||
case Result(value) ⇒
|
||||
pi += value
|
||||
nrOfResults += 1
|
||||
if (nrOfResults == nrOfMessages) {
|
||||
// Send the result to the listener
|
||||
listener ! PiApproximation(pi, duration = (System.currentTimeMillis - start).millis)
|
||||
// Stops this actor and all its supervised children
|
||||
if (nrOfResults == nrOfMessages) context.stop(self)
|
||||
context.stop(self)
|
||||
}
|
||||
//#handle-messages
|
||||
}
|
||||
//#master-receive
|
||||
|
||||
override def preStart() {
|
||||
start = System.currentTimeMillis
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
println("\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis"
|
||||
.format(pi, (System.currentTimeMillis - start)))
|
||||
latch.countDown()
|
||||
}
|
||||
}
|
||||
//#master
|
||||
|
||||
//#result-listener
|
||||
class Listener extends Actor {
|
||||
def receive = {
|
||||
case PiApproximation(pi, duration) ⇒
|
||||
println("\n\tPi approximation: \t\t%s\n\tCalculation time: \t%s"
|
||||
.format(pi, duration))
|
||||
context.system.shutdown()
|
||||
}
|
||||
}
|
||||
//#result-listener
|
||||
|
||||
//#actors-and-messages
|
||||
|
||||
def calculate(nrOfWorkers: Int, nrOfElements: Int, nrOfMessages: Int) {
|
||||
// Create an Akka system
|
||||
val system = ActorSystem("PiSystem")
|
||||
|
||||
// this latch is only plumbing to know when the calculation is completed
|
||||
val latch = new CountDownLatch(1)
|
||||
// create the result listener, which will print the result and shutdown the system
|
||||
val listener = system.actorOf(Props[Listener], name = "listener")
|
||||
|
||||
// create the master
|
||||
val master = system.actorOf(Props(new Master(
|
||||
nrOfWorkers, nrOfMessages, nrOfElements, latch)),
|
||||
"master")
|
||||
nrOfWorkers, nrOfMessages, nrOfElements, listener)),
|
||||
name = "master")
|
||||
|
||||
// start the calculation
|
||||
master ! Calculate
|
||||
|
||||
// wait for master to shut down
|
||||
latch.await()
|
||||
|
||||
// Shut down the system
|
||||
system.shutdown()
|
||||
}
|
||||
}
|
||||
//#app
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue