Merge branch 'master' of github.com:jboner/akka

This commit is contained in:
Jonas Bonér 2012-01-30 16:12:24 +01:00
commit 945ab2c6ab
53 changed files with 1779 additions and 1740 deletions

View file

@ -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 {
case RelativeActorPath(elems) system.actorFor("/").path / elems
})
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") }
}
}
}

View file

@ -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)
}
}

View file

@ -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 paths 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 =

View file

@ -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
*/

View file

@ -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 parents 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
}

View file

@ -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

View file

@ -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 ActorSystems
* 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
}

View file

@ -23,7 +23,7 @@ case object LocalScope extends Scope
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;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(

View file

@ -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(

View file

@ -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(

View file

@ -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
}
}

View file

@ -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)

View file

@ -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) }

View file

@ -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 {

View file

@ -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 threads 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
-----------------------------

View file

@ -18,45 +18,53 @@ 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;
}
}
public static final class Queue {
final Object o;
public Queue(Object o) {
this.o = o;
}
}
public static final Object flush = new Object();
public static final class Batch {
final List<Object> objects;
public Batch(List<Object> objects) {
this.objects = objects;
}
}
//#data
//#base
static abstract class MyFSMBase extends UntypedActor {
/*
* 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;
/*
* Then come all the mutator methods:
*/
@ -64,21 +72,23 @@ public class FSMDocTestBase {
this.target = target;
queue = new ArrayList<Object>();
}
protected void setState(State s) {
if (state != s) {
transition(state, s);
state = s;
}
}
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;
}
@ -89,13 +99,14 @@ public class FSMDocTestBase {
protected boolean isInitialized() {
return target != null;
}
protected State getState() {
return state;
}
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,53 +115,62 @@ public class FSMDocTestBase {
*/
abstract protected void transition(State old, State next);
}
//#base
//#actor
static public class MyFSM extends MyFSMBase {
private final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
@Override
public void onReceive(Object o) {
if (getState() == State.IDLE) {
if (o instanceof SetTarget)
if (o instanceof SetTarget)
init(((SetTarget) o).ref);
else whenUnhandled(o);
else
whenUnhandled(o);
} else if (getState() == State.ACTIVE) {
if (o == flush)
if (o == flush)
setState(State.IDLE);
else whenUnhandled(o);
else
whenUnhandled(o);
}
}
@Override
public void transition(State old, State next) {
if (old == State.ACTIVE) {
getTarget().tell(new Batch(drainQueue()));
}
}
private void whenUnhandled(Object o) {
if (o instanceof Queue && isInitialized()) {
enqueue(((Queue) o).o);
setState(State.ACTIVE);
} else {
log.warning("received unknown message {} in state {}", o, getState());
}
}
}
//#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() {
final ActorRef buncher = system.actorOf(new Props(MyFSM.class));
@ -165,10 +185,10 @@ public class FSMDocTestBase {
assert b.objects.contains(1);
assert b.objects.contains(2);
}
@org.junit.After
public void cleanup() {
system.shutdown();
}
}

View file

@ -211,16 +211,17 @@ public class UntypedActorDocTestBase {
//#gracefulStop
system.shutdown();
}
class Result {
final int x;
final String s;
public Result(int x, String s) {
this.x = x;
this.s = s;
}
}
@Test
public void usePatternsAskPipeTo() {
ActorSystem system = ActorSystem.create("MySystem");
@ -229,13 +230,13 @@ public class UntypedActorDocTestBase {
ActorRef actorC = system.actorOf(new Props(MyUntypedActor.class));
//#ask-pipeTo
final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS));
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
futures.add(ask(actorB, "reqeest", t)); // using timeout from above
final Future<Iterable<Object>> aggregate = Futures.sequence(futures, system.dispatcher());
final Future<Result> transformed = aggregate.map(new akka.japi.Function<Iterable<Object>, Result>() {
public Result apply(Iterable<Object> coll) {
final Iterator<Object> it = coll.iterator();
@ -244,9 +245,10 @@ public class UntypedActorDocTestBase {
return new Result(x, s);
}
});
pipeTo(transformed, actorC);
//#ask-pipeTo
system.shutdown();
}
public static class MyActor extends UntypedActor {

View file

@ -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)

View file

@ -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)
}

View file

@ -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 {

View file

@ -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)) {

View file

@ -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))
}
}

View file

@ -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
})

View file

@ -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)
}

View file

@ -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)
}
}
}

View file

@ -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)
if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment)
else {
val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements
useActorOnNode(rpath, props.creator, supervisor)
new RemoteActorRef(this, remote.transport, rpath, supervisor, None)
}
case Some(Deploy(_, _, _, RemoteScope(addr)))
if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment)
else {
val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements
useActorOnNode(rpath, props.creator, supervisor)
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 {

View file

@ -0,0 +1,5 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote

View file

@ -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)
}

View 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)
}
}

View file

@ -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

View file

@ -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
}

View file

@ -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")
}

View 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)
}
}
}

View 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)
}
}

View file

@ -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

View 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
}
}

View 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
}
}

View file

@ -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]]
}

View file

@ -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,8 +46,8 @@ 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 x throw new ConfigurationException("unparseable remote node " + x)
case AddressExtractor(a) a
case x throw new ConfigurationException("unparseable remote node " + x)
}
Stream.continually(nodeAddresses).flatten.iterator
}

View file

@ -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
}
}

View file

@ -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")
}
}
}

View file

@ -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")
}

View file

@ -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")
}

View file

@ -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))
}
}

View file

@ -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))
}

View file

@ -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))
}

View file

@ -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()

View file

@ -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

View file

@ -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 {

View file

@ -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
import settings._
//SharedSettings
RemoteTransport must be("akka.remote.netty.NettyRemoteTransport")
UntrustedMode must be(false)
RemoteSystemDaemonAckTimeout must be(30 seconds)
{
import settings._
FailureDetectorThreshold must be(8)
FailureDetectorMaxSampleSize must be(1000)
RemoteTransport must equal("akka.remote.netty.NettyRemoteSupport")
BackoffTimeout must equal(0 seconds)
LogReceivedMessages must equal(false)
LogSentMessages must equal(false)
}
//ServerSettings
{
import settings.serverSettings._
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)
}
//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)
InitialDelayForGossip must be(5 seconds)
GossipFrequency must be(1 second)
SeedNodes must be(Set())
}
"be able to parse Netty config elements" in {
val settings =
system.asInstanceOf[ExtendedActorSystem]
.provider.asInstanceOf[RemoteActorRefProvider]
.transport.asInstanceOf[NettyRemoteTransport]
.settings
import settings._
BackoffTimeout must be(Duration.Zero)
SecureCookie must be(None)
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)
}
}
}

View file

@ -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()

View file

@ -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))))))
}
}

View file

@ -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"
}
}