Merge branch 'master' of github.com:jboner/akka
This commit is contained in:
commit
945ab2c6ab
53 changed files with 1779 additions and 1740 deletions
|
|
@ -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") }
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package akka.actor
|
||||
import scala.annotation.tailrec
|
||||
import java.net.MalformedURLException
|
||||
|
||||
object ActorPath {
|
||||
def split(s: String): List[String] = {
|
||||
|
|
@ -16,6 +17,11 @@ object ActorPath {
|
|||
rec(s.length, Nil)
|
||||
}
|
||||
|
||||
def fromString(s: String): ActorPath = s match {
|
||||
case ActorPathExtractor(addr, elems) ⇒ RootActorPath(addr) / elems
|
||||
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
||||
}
|
||||
|
||||
val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r
|
||||
}
|
||||
|
||||
|
|
@ -87,6 +93,12 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
|
|||
*/
|
||||
def root: RootActorPath
|
||||
|
||||
/**
|
||||
* Generate String representation, replacing the Address in the RootActor
|
||||
* Path with the given one unless this path’s address includes host and port
|
||||
* information.
|
||||
*/
|
||||
def toStringWithAddress(address: Address): String
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -105,6 +117,10 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
|||
|
||||
override val toString = address + name
|
||||
|
||||
def toStringWithAddress(addr: Address): String =
|
||||
if (address.host.isDefined) address + name
|
||||
else addr + name
|
||||
|
||||
def compareTo(other: ActorPath) = other match {
|
||||
case r: RootActorPath ⇒ toString compareTo r.toString
|
||||
case c: ChildActorPath ⇒ 1
|
||||
|
|
@ -151,6 +167,15 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
|||
rec(parent, new StringBuilder(32).append(name)).toString
|
||||
}
|
||||
|
||||
override def toStringWithAddress(addr: Address) = {
|
||||
@tailrec
|
||||
def rec(p: ActorPath, s: StringBuilder): StringBuilder = p match {
|
||||
case r: RootActorPath ⇒ s.insert(0, r.toStringWithAddress(addr))
|
||||
case _ ⇒ rec(p.parent, s.insert(0, '/').insert(0, p.name))
|
||||
}
|
||||
rec(parent, new StringBuilder(32).append(name)).toString
|
||||
}
|
||||
|
||||
override def equals(other: Any): Boolean = {
|
||||
@tailrec
|
||||
def rec(left: ActorPath, right: ActorPath): Boolean =
|
||||
|
|
|
|||
|
|
@ -211,7 +211,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe
|
|||
* This is an internal look-up failure token, not useful for anything else.
|
||||
*/
|
||||
private[akka] case object Nobody extends MinimalActorRef {
|
||||
val path = new RootActorPath(new LocalAddress("all-systems"), "/Nobody")
|
||||
val path = new RootActorPath(Address("akka", "all-systems"), "/Nobody")
|
||||
def provider = throw new UnsupportedOperationException("Nobody does not provide")
|
||||
}
|
||||
|
||||
|
|
@ -329,13 +329,13 @@ private[akka] class LocalActorRef private[akka] (
|
|||
def restart(cause: Throwable): Unit = actorCell.restart(cause)
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path.toString)
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
/**
|
||||
* Memento pattern for serializing ActorRefs transparently
|
||||
*/
|
||||
case class SerializedActorRef(path: String) {
|
||||
case class SerializedActorRef private (path: String) {
|
||||
import akka.serialization.Serialization.currentSystem
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
|
|
@ -349,6 +349,15 @@ case class SerializedActorRef(path: String) {
|
|||
}
|
||||
}
|
||||
|
||||
object SerializedActorRef {
|
||||
def apply(path: ActorPath): SerializedActorRef = {
|
||||
Serialization.currentTransportAddress.value match {
|
||||
case null ⇒ new SerializedActorRef(path.toString)
|
||||
case addr ⇒ new SerializedActorRef(path.toStringWithAddress(addr))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Trait for ActorRef implementations where all methods contain default stubs.
|
||||
*/
|
||||
|
|
@ -375,7 +384,7 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
|
|||
def restart(cause: Throwable): Unit = ()
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path.toString)
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
private[akka] object MinimalActorRef {
|
||||
|
|
@ -398,57 +407,39 @@ private[akka] object DeadLetterActorRef {
|
|||
val serialized = new SerializedDeadLetterActorRef
|
||||
}
|
||||
|
||||
private[akka] trait DeadLetterActorRefLike extends MinimalActorRef {
|
||||
|
||||
def eventStream: EventStream
|
||||
|
||||
@volatile
|
||||
private var _path: ActorPath = _
|
||||
def path: ActorPath = {
|
||||
assert(_path != null)
|
||||
_path
|
||||
}
|
||||
|
||||
@volatile
|
||||
private var _provider: ActorRefProvider = _
|
||||
def provider = _provider
|
||||
|
||||
private[akka] def init(provider: ActorRefProvider, path: ActorPath) {
|
||||
_path = path
|
||||
_provider = provider
|
||||
}
|
||||
|
||||
override def isTerminated(): Boolean = true
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRefLike {
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
/**
|
||||
* This special dead letter reference has a name: it is that which is returned
|
||||
* by a local look-up which is unsuccessful.
|
||||
*/
|
||||
private[akka] class EmptyLocalActorRef(
|
||||
val eventStream: EventStream,
|
||||
_provider: ActorRefProvider,
|
||||
_dispatcher: MessageDispatcher,
|
||||
_path: ActorPath) extends DeadLetterActorRefLike {
|
||||
val provider: ActorRefProvider,
|
||||
val path: ActorPath,
|
||||
val eventStream: EventStream) extends MinimalActorRef {
|
||||
|
||||
init(_provider, _path)
|
||||
override def isTerminated(): Boolean = true
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match {
|
||||
case d: DeadLetter ⇒ // do NOT form endless loops
|
||||
case d: DeadLetter ⇒ // do NOT form endless loops, since deadLetters will resend!
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal implementation of the dead letter destination: will publish any
|
||||
* received message to the eventStream, wrapped as [[akka.actor.DeadLetter]].
|
||||
*/
|
||||
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
|
||||
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match {
|
||||
case d: DeadLetter ⇒ eventStream.publish(d)
|
||||
case _ ⇒ eventStream.publish(DeadLetter(message, sender, this))
|
||||
}
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal implementation detail used for paths like “/temp”
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -33,11 +33,22 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def systemGuardian: InternalActorRef
|
||||
|
||||
/**
|
||||
* Dead letter destination for this provider.
|
||||
*/
|
||||
def deadLetters: ActorRef
|
||||
|
||||
/**
|
||||
* Reference to the death watch service.
|
||||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* Care-taker of actor refs which await final termination but cannot be kept
|
||||
* in their parent’s children list because the name shall be freed.
|
||||
*/
|
||||
def locker: Locker
|
||||
|
||||
/**
|
||||
* The root path for all actors within this actor system, including remote
|
||||
* address if enabled.
|
||||
|
|
@ -281,25 +292,30 @@ class LocalActorRefProvider(
|
|||
val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
val deadLetters: InternalActorRef,
|
||||
val rootPath: ActorPath,
|
||||
val deployer: Deployer) extends ActorRefProvider {
|
||||
|
||||
// this is the constructor needed for reflectively instantiating the provider
|
||||
def this(_systemName: String,
|
||||
settings: ActorSystem.Settings,
|
||||
eventStream: EventStream,
|
||||
scheduler: Scheduler,
|
||||
deadLetters: InternalActorRef) =
|
||||
classloader: ClassLoader) =
|
||||
this(_systemName,
|
||||
settings,
|
||||
eventStream,
|
||||
scheduler,
|
||||
deadLetters,
|
||||
new RootActorPath(LocalAddress(_systemName)),
|
||||
new Deployer(settings))
|
||||
new Deployer(settings, classloader))
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
|
||||
val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")")
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
|
||||
|
||||
val deathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
val locker: Locker = new Locker(scheduler, settings.ReaperInterval, this, rootPath / "locker", deathWatch)
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
*/
|
||||
|
|
@ -455,8 +471,6 @@ class LocalActorRefProvider(
|
|||
tempContainer.removeChild(path.name)
|
||||
}
|
||||
|
||||
val deathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
def init(_system: ActorSystemImpl) {
|
||||
system = _system
|
||||
// chain death watchers so that killing guardian stops the application
|
||||
|
|
@ -472,7 +486,7 @@ class LocalActorRefProvider(
|
|||
deadLetters
|
||||
} else if (elems.head.isEmpty) actorFor(rootGuardian, elems.tail)
|
||||
else actorFor(ref, elems)
|
||||
case LocalActorPath(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems)
|
||||
case ActorPathExtractor(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems)
|
||||
case _ ⇒
|
||||
log.debug("look-up of unknown path '{}' failed", path)
|
||||
deadLetters
|
||||
|
|
@ -492,7 +506,7 @@ class LocalActorRefProvider(
|
|||
} else ref.getChild(path.iterator) match {
|
||||
case Nobody ⇒
|
||||
log.debug("look-up of path sequence '{}' failed", path)
|
||||
new EmptyLocalActorRef(eventStream, system.provider, dispatcher, ref.path / path)
|
||||
new EmptyLocalActorRef(system.provider, ref.path / path, eventStream)
|
||||
case x ⇒ x
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -267,7 +267,7 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
* (below which the logging actors reside) and the execute all registered
|
||||
* termination handlers (see [[ActorSystem.registerOnTermination]]).
|
||||
*/
|
||||
def shutdown()
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Registers the provided extension and creates its payload, if this extension isn't already registered
|
||||
|
|
@ -322,6 +322,14 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
*/
|
||||
def deathWatch: DeathWatch
|
||||
|
||||
/**
|
||||
* ClassLoader which is used for reflective accesses internally. This is set
|
||||
* to the context class loader, if one is set, or the class loader which
|
||||
* loaded the ActorSystem implementation. The context class loader is also
|
||||
* set on all threads created by the ActorSystem, if one was set during
|
||||
* creation.
|
||||
*/
|
||||
def internalClassLoader: ClassLoader
|
||||
}
|
||||
|
||||
class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem {
|
||||
|
|
@ -331,8 +339,10 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
|
||||
import ActorSystem._
|
||||
|
||||
final val settings = new Settings(applicationConfig, name)
|
||||
final val threadFactory = new MonitorableThreadFactory(name, settings.Daemonicity)
|
||||
final val settings: Settings = new Settings(applicationConfig, name)
|
||||
|
||||
final val threadFactory: MonitorableThreadFactory =
|
||||
MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader))
|
||||
|
||||
def logConfiguration(): Unit = log.info(settings.toString)
|
||||
|
||||
|
|
@ -377,18 +387,32 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
import settings._
|
||||
|
||||
// this provides basic logging (to stdout) until .start() is called below
|
||||
val eventStream = new EventStream(DebugEventStream)
|
||||
val eventStream: EventStream = new EventStream(DebugEventStream)
|
||||
eventStream.startStdoutLogger(settings)
|
||||
|
||||
// unfortunately we need logging before we know the rootpath address, which wants to be inserted here
|
||||
@volatile
|
||||
private var _log = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass)
|
||||
def log = _log
|
||||
val log: LoggingAdapter = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass)
|
||||
|
||||
val scheduler = createScheduler()
|
||||
val scheduler: Scheduler = createScheduler()
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(eventStream)
|
||||
val deadLetterMailbox = new Mailbox(null) {
|
||||
val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader
|
||||
|
||||
val provider: ActorRefProvider = {
|
||||
val arguments = Seq(
|
||||
classOf[String] -> name,
|
||||
classOf[Settings] -> settings,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[Scheduler] -> scheduler,
|
||||
classOf[ClassLoader] -> internalClassLoader)
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
}
|
||||
|
||||
def deadLetters: ActorRef = provider.deadLetters
|
||||
|
||||
val deadLetterMailbox: Mailbox = new Mailbox(null) {
|
||||
becomeClosed()
|
||||
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||
override def dequeue() = null
|
||||
|
|
@ -399,28 +423,12 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
override def numberOfMessages = 0
|
||||
}
|
||||
|
||||
val provider: ActorRefProvider = {
|
||||
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(b) ⇒ b
|
||||
}
|
||||
val arguments = Seq(
|
||||
classOf[String] -> name,
|
||||
classOf[Settings] -> settings,
|
||||
classOf[EventStream] -> eventStream,
|
||||
classOf[Scheduler] -> scheduler,
|
||||
classOf[InternalActorRef] -> deadLetters)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
def locker: Locker = provider.locker
|
||||
|
||||
ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(p) ⇒ p
|
||||
}
|
||||
}
|
||||
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
|
||||
threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader))
|
||||
|
||||
val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler))
|
||||
val dispatcher = dispatchers.defaultGlobalDispatcher
|
||||
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
|
||||
|
||||
def terminationFuture: Future[Unit] = provider.terminationFuture
|
||||
def lookupRoot: InternalActorRef = provider.rootGuardian
|
||||
|
|
@ -434,21 +442,13 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
private lazy val _start: this.type = {
|
||||
// the provider is expected to start default loggers, LocalActorRefProvider does this
|
||||
provider.init(this)
|
||||
_log = new BusLogging(eventStream, "ActorSystem(" + lookupRoot.path.address + ")", this.getClass)
|
||||
deadLetters.init(provider, lookupRoot.path / "deadLetters")
|
||||
registerOnTermination(stopScheduler())
|
||||
// this starts the reaper actor and the user-configured logging subscribers, which are also actors
|
||||
_locker = new Locker(scheduler, ReaperInterval, provider, lookupRoot.path / "locker", deathWatch)
|
||||
loadExtensions()
|
||||
if (LogConfigOnStart) logConfiguration()
|
||||
this
|
||||
}
|
||||
|
||||
@volatile
|
||||
private var _locker: Locker = _ // initialized in start()
|
||||
def locker = _locker
|
||||
|
||||
def start() = _start
|
||||
def start(): this.type = _start
|
||||
|
||||
private lazy val terminationCallbacks = {
|
||||
val callbacks = new TerminationCallbacks
|
||||
|
|
@ -460,9 +460,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) }
|
||||
def awaitTermination() = awaitTermination(Duration.Inf)
|
||||
|
||||
def shutdown() {
|
||||
stop(guardian)
|
||||
}
|
||||
def shutdown(): Unit = stop(guardian)
|
||||
|
||||
/**
|
||||
* Create the scheduler service. This one needs one special behavior: if
|
||||
|
|
@ -547,8 +545,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
private def loadExtensions() {
|
||||
import scala.collection.JavaConversions._
|
||||
settings.config.getStringList("akka.extensions") foreach { fqcn ⇒
|
||||
import ReflectiveAccess._
|
||||
getObjectFor[AnyRef](fqcn).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
|
||||
import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs }
|
||||
getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
|
||||
case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup());
|
||||
case Right(p: ExtensionId[_]) ⇒ registerExtension(p);
|
||||
case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||
|
|
@ -558,7 +556,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
|
|||
}
|
||||
}
|
||||
|
||||
override def toString = lookupRoot.path.root.address.toString
|
||||
override def toString: String = lookupRoot.path.root.address.toString
|
||||
|
||||
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
|
||||
private val lock = new ReentrantGuard
|
||||
|
|
|
|||
|
|
@ -9,17 +9,41 @@ import java.net.URISyntaxException
|
|||
* The address specifies the physical location under which an Actor can be
|
||||
* reached. Examples are local addresses, identified by the ActorSystem’s
|
||||
* name, and remote addresses, identified by protocol, host and port.
|
||||
*
|
||||
* This class is final to allow use as a case class (copy method etc.); if
|
||||
* for example a remote transport would want to associate additional
|
||||
* information with an address, then this must be done externally.
|
||||
*/
|
||||
abstract class Address {
|
||||
def protocol: String
|
||||
def hostPort: String
|
||||
final case class Address(protocol: String, system: String, host: Option[String], port: Option[Int]) {
|
||||
|
||||
def this(protocol: String, system: String) = this(protocol, system, None, None)
|
||||
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
|
||||
|
||||
@transient
|
||||
override lazy val toString = protocol + "://" + hostPort
|
||||
override lazy val toString = {
|
||||
val sb = new StringBuilder(protocol)
|
||||
sb.append("://")
|
||||
sb.append(hostPort)
|
||||
sb.toString
|
||||
}
|
||||
|
||||
@transient
|
||||
lazy val hostPort = {
|
||||
val sb = new StringBuilder(system)
|
||||
if (host.isDefined) {
|
||||
sb.append('@')
|
||||
sb.append(host.get)
|
||||
}
|
||||
if (port.isDefined) {
|
||||
sb.append(':')
|
||||
sb.append(port.get)
|
||||
}
|
||||
sb.toString
|
||||
}
|
||||
}
|
||||
|
||||
case class LocalAddress(systemName: String) extends Address {
|
||||
def protocol = "akka"
|
||||
def hostPort = systemName
|
||||
object Address {
|
||||
def apply(protocol: String, system: String) = new Address(protocol, system)
|
||||
}
|
||||
|
||||
object RelativeActorPath {
|
||||
|
|
@ -32,12 +56,34 @@ object RelativeActorPath {
|
|||
}
|
||||
}
|
||||
|
||||
object LocalActorPath {
|
||||
def unapply(addr: String): Option[(LocalAddress, Iterable[String])] = {
|
||||
object AddressExtractor {
|
||||
def unapply(addr: String): Option[Address] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme != "akka" || uri.getUserInfo != null || uri.getHost == null || uri.getPath == null) None
|
||||
else Some(LocalAddress(uri.getHost), ActorPath.split(uri.getPath).drop(1))
|
||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None
|
||||
else {
|
||||
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
|
||||
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
|
||||
if (uri.getPort < 0) None else Some(uri.getPort))
|
||||
Some(addr)
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ActorPathExtractor {
|
||||
def unapply(addr: String): Option[(Address, Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None
|
||||
else {
|
||||
val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost,
|
||||
if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost),
|
||||
if (uri.getPort < 0) None else Some(uri.getPort))
|
||||
Some((addr, ActorPath.split(uri.getPath).drop(1)))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ case object LocalScope extends Scope
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Deployer(val settings: ActorSystem.Settings) {
|
||||
class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
|
@ -41,7 +41,6 @@ class Deployer(val settings: ActorSystem.Settings) {
|
|||
def deploy(d: Deploy): Unit = deployments.put(d.path, d)
|
||||
|
||||
protected def parseConfig(key: String, config: Config): Option[Deploy] = {
|
||||
import akka.util.ReflectiveAccess.getClassFor
|
||||
|
||||
val deployment = config.withFallback(default)
|
||||
|
||||
|
|
@ -65,8 +64,8 @@ class Deployer(val settings: ActorSystem.Settings) {
|
|||
case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer)
|
||||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
||||
case fqn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config])
|
||||
ReflectiveAccess.createInstance[RouterConfig](fqn, constructorSignature, Array[AnyRef](deployment)) match {
|
||||
val args = Seq(classOf[Config] -> deployment)
|
||||
ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match {
|
||||
case Right(router) ⇒ router
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -321,8 +321,8 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
BoundedMailbox(capacity, duration)
|
||||
}
|
||||
case fqcn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config])
|
||||
ReflectiveAccess.createInstance[MailboxType](fqcn, constructorSignature, Array[AnyRef](config)) match {
|
||||
val args = Seq(classOf[Config] -> config)
|
||||
ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -19,13 +19,15 @@ trait DispatcherPrerequisites {
|
|||
def eventStream: EventStream
|
||||
def deadLetterMailbox: Mailbox
|
||||
def scheduler: Scheduler
|
||||
def classloader: ClassLoader
|
||||
}
|
||||
|
||||
case class DefaultDispatcherPrerequisites(
|
||||
val threadFactory: ThreadFactory,
|
||||
val eventStream: EventStream,
|
||||
val deadLetterMailbox: Mailbox,
|
||||
val scheduler: Scheduler) extends DispatcherPrerequisites
|
||||
val scheduler: Scheduler,
|
||||
val classloader: ClassLoader) extends DispatcherPrerequisites
|
||||
|
||||
object Dispatchers {
|
||||
/**
|
||||
|
|
@ -134,8 +136,8 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||
case fqn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, constructorSignature, Array[AnyRef](cfg, prerequisites)) match {
|
||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match {
|
||||
case Right(configurator) ⇒ configurator
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -159,6 +159,7 @@ object MonitorableThreadFactory {
|
|||
|
||||
case class MonitorableThreadFactory(name: String,
|
||||
daemonic: Boolean,
|
||||
contextClassLoader: Option[ClassLoader],
|
||||
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing)
|
||||
extends ThreadFactory {
|
||||
protected val counter = new AtomicLong
|
||||
|
|
@ -167,6 +168,7 @@ case class MonitorableThreadFactory(name: String,
|
|||
val t = new Thread(runnable, name + counter.incrementAndGet())
|
||||
t.setUncaughtExceptionHandler(exceptionHandler)
|
||||
t.setDaemon(daemonic)
|
||||
contextClassLoader foreach (t.setContextClassLoader(_))
|
||||
t
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -101,7 +101,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
if loggerName != StandardOutLoggerName
|
||||
} yield {
|
||||
try {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName) match {
|
||||
ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match {
|
||||
case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName)
|
||||
case Left(exception) ⇒ throw exception
|
||||
}
|
||||
|
|
@ -648,7 +648,7 @@ object Logging {
|
|||
* <code>akka.stdout-loglevel</code> in <code>akka.conf</code>.
|
||||
*/
|
||||
class StandardOutLogger extends MinimalActorRef with StdOutLogger {
|
||||
val path: ActorPath = new RootActorPath(LocalAddress("all-systems"), "/StandardOutLogger")
|
||||
val path: ActorPath = new RootActorPath(Address("akka", "all-systems"), "/StandardOutLogger")
|
||||
def provider: ActorRefProvider = throw new UnsupportedOperationException("StandardOutLogger does not provide")
|
||||
override val toString = "StandardOutLogger"
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message)
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.util.ReflectiveAccess
|
|||
import scala.util.DynamicVariable
|
||||
import com.typesafe.config.Config
|
||||
import akka.config.ConfigurationException
|
||||
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem }
|
||||
import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address }
|
||||
|
||||
case class NoSerializerFoundException(m: String) extends AkkaException(m)
|
||||
|
||||
|
|
@ -27,6 +27,12 @@ object Serialization {
|
|||
*/
|
||||
val currentSystem = new DynamicVariable[ActorSystem](null)
|
||||
|
||||
/**
|
||||
* This holds a reference to the current transport address to be inserted
|
||||
* into local actor refs during serialization.
|
||||
*/
|
||||
val currentTransportAddress = new DynamicVariable[Address](null)
|
||||
|
||||
class Settings(val config: Config) {
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -75,10 +81,10 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
def deserialize(bytes: Array[Byte],
|
||||
serializerId: Int,
|
||||
clazz: Option[Class[_]],
|
||||
classLoader: Option[ClassLoader]): Either[Exception, AnyRef] =
|
||||
classLoader: ClassLoader): Either[Exception, AnyRef] =
|
||||
try {
|
||||
currentSystem.withValue(system) {
|
||||
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, classLoader))
|
||||
Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader)))
|
||||
}
|
||||
} catch { case e: Exception ⇒ Left(e) }
|
||||
|
||||
|
|
|
|||
|
|
@ -38,6 +38,15 @@ object ReflectiveAccess {
|
|||
}
|
||||
}
|
||||
|
||||
def createInstance[T](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
|
||||
createInstance(clazz, args.map(_._1).toArray, args.map(_._2).toArray)
|
||||
|
||||
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)], classloader: ClassLoader): Either[Exception, T] =
|
||||
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, classloader)
|
||||
|
||||
def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Exception, T] =
|
||||
createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, loader)
|
||||
|
||||
//Obtains a reference to fqn.MODULE$
|
||||
def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, T] = try {
|
||||
getClassFor(fqn, classloader) match {
|
||||
|
|
|
|||
|
|
@ -217,6 +217,20 @@ and parsed by the actor system can be displayed like this:
|
|||
println(system.settings());
|
||||
// this is a shortcut for system.settings().config().root().render()
|
||||
|
||||
A Word About ClassLoaders
|
||||
-------------------------
|
||||
|
||||
In several places of the configuration file it is possible to specify the
|
||||
fully-qualified class name of something to be instantiated by Akka. This is
|
||||
done using Java reflection, which in turn uses a :class:`ClassLoader`. Getting
|
||||
the right one in challenging environments like application containers or OSGi
|
||||
bundles is not always trivial, the current approach of Akka is that each
|
||||
:class:`ActorSystem` implementation stores the current thread’s context class
|
||||
loader (if available, otherwise just its own loader as in
|
||||
``this.getClass.getClassLoader``) and uses that for all reflective accesses.
|
||||
This implies that putting Akka on the boot class path will yield
|
||||
:class:`NullPointerException` from strange places: this is simply not
|
||||
supported.
|
||||
|
||||
Application specific settings
|
||||
-----------------------------
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.actor.mailbox
|
||||
|
||||
import akka.actor.{ ActorContext, ActorRef }
|
||||
import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem }
|
||||
import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox }
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol }
|
||||
|
|
@ -15,7 +15,7 @@ private[akka] object DurableExecutableMailboxConfig {
|
|||
abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue {
|
||||
import DurableExecutableMailboxConfig._
|
||||
|
||||
def system = owner.system
|
||||
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
|
||||
def ownerPath = owner.self.path
|
||||
val ownerPathString = ownerPath.elements.mkString("/")
|
||||
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
|
||||
|
|
@ -44,7 +44,7 @@ trait DurableMessageSerialization {
|
|||
def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = owner.system.actorFor(refProtocol.getPath)
|
||||
|
||||
val durableMessage = RemoteMessageProtocol.parseFrom(bytes)
|
||||
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage)
|
||||
val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage, getClass.getClassLoader)
|
||||
val sender = deserializeActorRef(durableMessage.getSender)
|
||||
|
||||
new Envelope(message, sender)(owner.system)
|
||||
|
|
|
|||
|
|
@ -16,11 +16,9 @@ import org.bson.DefaultBSONSerializer
|
|||
|
||||
import akka.remote.RemoteProtocol.MessageProtocol
|
||||
import akka.remote.MessageSerializer
|
||||
import akka.actor.{ ActorSystem, ActorSystemImpl }
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
||||
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
val systemImpl = system.asInstanceOf[ActorSystemImpl]
|
||||
class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
|
||||
|
||||
protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = {
|
||||
|
||||
|
|
@ -67,10 +65,10 @@ class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObjec
|
|||
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
|
||||
system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
|
||||
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
|
||||
val msg = MessageSerializer.deserialize(system, msgData)
|
||||
val msg = MessageSerializer.deserialize(system, msgData, system.internalClassLoader)
|
||||
val ownerPath = doc.as[String]("ownerPath")
|
||||
val senderPath = doc.as[String]("senderPath")
|
||||
val sender = systemImpl.actorFor(senderPath)
|
||||
val sender = system.actorFor(senderPath)
|
||||
|
||||
MongoDurableMessage(ownerPath, msg, sender)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -38,33 +38,86 @@ akka {
|
|||
|
||||
remote {
|
||||
|
||||
# Which implementation of akka.remote.RemoteSupport to use
|
||||
# Which implementation of akka.remote.RemoteTransport to use
|
||||
# default is a TCP-based remote transport based on Netty
|
||||
transport = "akka.remote.netty.NettyRemoteSupport"
|
||||
transport = "akka.remote.netty.NettyRemoteTransport"
|
||||
|
||||
# In case of increased latency / overflow how long
|
||||
# should we wait (blocking the sender) until we deem the send to be cancelled?
|
||||
# 0 means "never backoff", any positive number will indicate time to block at most.
|
||||
backoff-timeout = 0ms
|
||||
|
||||
use-compression = off
|
||||
|
||||
# Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
|
||||
# or using 'akka.util.Crypt.generateSecureCookie'
|
||||
secure-cookie = ""
|
||||
# Enable untrusted mode for full security of server managed actors, allows
|
||||
# untrusted clients to connect.
|
||||
untrusted-mode = off
|
||||
|
||||
# Timeout for ACK of cluster operations, lik checking actor out etc.
|
||||
remote-daemon-ack-timeout = 30s
|
||||
|
||||
# Reuse inbound connections for outbound messages
|
||||
use-passive-connections = on
|
||||
|
||||
# If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged
|
||||
log-received-messages = off
|
||||
|
||||
# If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged
|
||||
log-sent-messages = off
|
||||
|
||||
# Each property is annotated with (I) or (O) or (I&O), where I stands for “inbound” and O for “outbound” connections.
|
||||
# The NettyRemoteTransport always starts the server role to allow inbound connections, and it starts
|
||||
# active client connections whenever sending to a destination which is not yet connected; if configured
|
||||
# it reuses inbound connections for replies, which is called a passive client connection (i.e. from server
|
||||
# to client).
|
||||
netty {
|
||||
|
||||
# (O) In case of increased latency / overflow how long
|
||||
# should we wait (blocking the sender) until we deem the send to be cancelled?
|
||||
# 0 means "never backoff", any positive number will indicate time to block at most.
|
||||
backoff-timeout = 0ms
|
||||
|
||||
# (I&O) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
|
||||
# or using 'akka.util.Crypt.generateSecureCookie'
|
||||
secure-cookie = ""
|
||||
|
||||
# (I) Should the remote server require that it peers share the same secure-cookie
|
||||
# (defined in the 'remote' section)?
|
||||
require-cookie = off
|
||||
|
||||
# (I) Reuse inbound connections for outbound messages
|
||||
use-passive-connections = on
|
||||
|
||||
# (I) The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# (I) The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA), use 0 if you want a random available port
|
||||
port = 2552
|
||||
|
||||
# (I&O) Increase this if you want to be able to send messages with large payloads
|
||||
message-frame-size = 1 MiB
|
||||
|
||||
# (O) Timeout duration
|
||||
connection-timeout = 120s
|
||||
|
||||
# (I) Sets the size of the connection backlog
|
||||
backlog = 4096
|
||||
|
||||
# (I) Length in akka.time-unit how long core threads will be kept alive if idling
|
||||
execution-pool-keepalive = 60s
|
||||
|
||||
# (I) Size of the core pool of the remote execution unit
|
||||
execution-pool-size = 4
|
||||
|
||||
# (I) Maximum channel size, 0 for off
|
||||
max-channel-memory-size = 0b
|
||||
|
||||
# (I) Maximum total size of all channels, 0 for off
|
||||
max-total-memory-size = 0b
|
||||
|
||||
# (O) Time between reconnect attempts for active clients
|
||||
reconnect-delay = 5s
|
||||
|
||||
# (O) Inactivity period after which active client connection is shutdown; will be
|
||||
# re-established in case of new communication requests
|
||||
read-timeout = 3600s
|
||||
|
||||
# (O) Maximum time window that a client should try to reconnect for
|
||||
reconnection-time-window = 600s
|
||||
}
|
||||
|
||||
# accrual failure detection config
|
||||
failure-detector {
|
||||
|
||||
|
|
@ -94,58 +147,6 @@ akka {
|
|||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA), use 0 if you want a random available port
|
||||
port = 2552
|
||||
|
||||
# Increase this if you want to be able to send messages with large payloads
|
||||
message-frame-size = 1 MiB
|
||||
|
||||
# Should the remote server require that it peers share the same secure-cookie
|
||||
# (defined in the 'remote' section)?
|
||||
require-cookie = off
|
||||
|
||||
# Enable untrusted mode for full security of server managed actors, allows
|
||||
# untrusted clients to connect.
|
||||
untrusted-mode = off
|
||||
|
||||
# Sets the size of the connection backlog
|
||||
backlog = 4096
|
||||
|
||||
# Length in akka.time-unit how long core threads will be kept alive if idling
|
||||
execution-pool-keepalive = 60s
|
||||
|
||||
# Size of the core pool of the remote execution unit
|
||||
execution-pool-size = 4
|
||||
|
||||
# Maximum channel size, 0 for off
|
||||
max-channel-memory-size = 0b
|
||||
|
||||
# Maximum total size of all channels, 0 for off
|
||||
max-total-memory-size = 0b
|
||||
}
|
||||
|
||||
client {
|
||||
# Time before an attempted connection is considered failed
|
||||
connection-timeout = 10s
|
||||
|
||||
#Time between each reconnection attempt
|
||||
reconnect-delay = 5s
|
||||
|
||||
# Maximum time window that a client should try to reconnect for
|
||||
reconnection-time-window = 600s
|
||||
|
||||
#Period of time of connection inactivity to be tolerated before hanging up
|
||||
read-timeout = 3600s
|
||||
|
||||
#Max size per message
|
||||
message-frame-size = 1 MiB
|
||||
}
|
||||
}
|
||||
|
||||
cluster {
|
||||
|
|
|
|||
|
|
@ -5,11 +5,10 @@
|
|||
package akka.remote
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.collection.immutable.Map
|
||||
import scala.annotation.tailrec
|
||||
|
||||
import System.{ currentTimeMillis ⇒ newTimestamp }
|
||||
import akka.actor.{ ActorSystem, Address }
|
||||
|
||||
/**
|
||||
* Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper:
|
||||
|
|
@ -33,9 +32,9 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
*/
|
||||
private case class State(
|
||||
version: Long = 0L,
|
||||
failureStats: Map[ParsedTransportAddress, FailureStats] = Map.empty[ParsedTransportAddress, FailureStats],
|
||||
intervalHistory: Map[ParsedTransportAddress, Vector[Long]] = Map.empty[ParsedTransportAddress, Vector[Long]],
|
||||
timestamps: Map[ParsedTransportAddress, Long] = Map.empty[ParsedTransportAddress, Long])
|
||||
failureStats: Map[Address, FailureStats] = Map.empty[Address, FailureStats],
|
||||
intervalHistory: Map[Address, Vector[Long]] = Map.empty[Address, Vector[Long]],
|
||||
timestamps: Map[Address, Long] = Map.empty[Address, Long])
|
||||
|
||||
private val state = new AtomicReference[State](State())
|
||||
|
||||
|
|
@ -43,13 +42,13 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Returns true if the connection is considered to be up and healthy
|
||||
* and returns false otherwise.
|
||||
*/
|
||||
def isAvailable(connection: ParsedTransportAddress): Boolean = phi(connection) < threshold
|
||||
def isAvailable(connection: Address): Boolean = phi(connection) < threshold
|
||||
|
||||
/**
|
||||
* Records a heartbeat for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def heartbeat(connection: ParsedTransportAddress) {
|
||||
final def heartbeat(connection: Address) {
|
||||
val oldState = state.get
|
||||
|
||||
val latestTimestamp = oldState.timestamps.get(connection)
|
||||
|
|
@ -130,7 +129,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Implementations of 'Cumulative Distribution Function' for Exponential Distribution.
|
||||
* For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597].
|
||||
*/
|
||||
def phi(connection: ParsedTransportAddress): Double = {
|
||||
def phi(connection: Address): Double = {
|
||||
val oldState = state.get
|
||||
val oldTimestamp = oldState.timestamps.get(connection)
|
||||
if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections
|
||||
|
|
@ -145,7 +144,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10
|
|||
* Removes the heartbeat management for a connection.
|
||||
*/
|
||||
@tailrec
|
||||
final def remove(connection: ParsedTransportAddress) {
|
||||
final def remove(connection: Address) {
|
||||
val oldState = state.get
|
||||
|
||||
if (oldState.failureStats.contains(connection)) {
|
||||
|
|
|
|||
|
|
@ -26,8 +26,8 @@ import akka.pattern.ask
|
|||
* Interface for node membership change listener.
|
||||
*/
|
||||
trait NodeMembershipChangeListener {
|
||||
def nodeConnected(node: ParsedTransportAddress)
|
||||
def nodeDisconnected(node: ParsedTransportAddress)
|
||||
def nodeConnected(node: Address)
|
||||
def nodeDisconnected(node: Address)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -35,9 +35,9 @@ trait NodeMembershipChangeListener {
|
|||
*/
|
||||
case class Gossip(
|
||||
version: VectorClock,
|
||||
node: ParsedTransportAddress,
|
||||
availableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress],
|
||||
unavailableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress])
|
||||
node: Address,
|
||||
availableNodes: Set[Address] = Set.empty[Address],
|
||||
unavailableNodes: Set[Address] = Set.empty[Address])
|
||||
|
||||
// ====== START - NEW GOSSIP IMPLEMENTATION ======
|
||||
/*
|
||||
|
|
@ -93,7 +93,7 @@ case class Gossip(
|
|||
* gossip to random seed with certain probability depending on number of unreachable, seed and live nodes.
|
||||
* </pre>
|
||||
*/
|
||||
class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
||||
class Gossiper(remote: RemoteActorRefProvider, system: ActorSystemImpl) {
|
||||
|
||||
/**
|
||||
* Represents the state for this Gossiper. Implemented using optimistic lockless concurrency,
|
||||
|
|
@ -107,27 +107,19 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
private val serialization = remote.serialization
|
||||
private val log = Logging(system, "Gossiper")
|
||||
private val failureDetector = remote.failureDetector
|
||||
private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[ParsedTransportAddress, ActorRef])
|
||||
private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[Address, ActorRef])
|
||||
|
||||
private val seeds = {
|
||||
val seeds = remoteSettings.SeedNodes flatMap {
|
||||
case x: UnparsedTransportAddress ⇒
|
||||
x.parse(remote.transports) match {
|
||||
case y: ParsedTransportAddress ⇒ Some(y)
|
||||
case _ ⇒ None
|
||||
}
|
||||
case _ ⇒ None
|
||||
}
|
||||
if (seeds.isEmpty) throw new ConfigurationException(
|
||||
if (remoteSettings.SeedNodes.isEmpty) throw new ConfigurationException(
|
||||
"At least one seed node must be defined in the configuration [akka.cluster.seed-nodes]")
|
||||
else seeds
|
||||
else remoteSettings.SeedNodes
|
||||
}
|
||||
|
||||
private val address = remote.remoteAddress
|
||||
private val address = remote.transport.address
|
||||
private val nodeFingerprint = address.##
|
||||
|
||||
private val random = SecureRandom.getInstance("SHA1PRNG")
|
||||
private val initalDelayForGossip = remoteSettings.InitalDelayForGossip
|
||||
private val initalDelayForGossip = remoteSettings.InitialDelayForGossip
|
||||
private val gossipFrequency = remoteSettings.GossipFrequency
|
||||
|
||||
private val state = new AtomicReference[State](State(currentGossip = newGossip()))
|
||||
|
|
@ -165,7 +157,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
node ← oldAvailableNodes
|
||||
if connectionManager.connectionFor(node).isEmpty
|
||||
} {
|
||||
val connectionFactory = () ⇒ system.actorFor(RootActorPath(RemoteSystemAddress(system.name, gossipingNode)) / "remote")
|
||||
val connectionFactory = () ⇒ system.actorFor(RootActorPath(gossipingNode) / "remote")
|
||||
connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node
|
||||
oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes
|
||||
}
|
||||
|
|
@ -239,7 +231,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
/**
|
||||
* Gossips set of nodes passed in as argument. Returns 'true' if it gossiped to a "seed" node.
|
||||
*/
|
||||
private def gossipTo(nodes: Set[ParsedTransportAddress]): Boolean = {
|
||||
private def gossipTo(nodes: Set[Address]): Boolean = {
|
||||
val peers = nodes filter (_ != address) // filter out myself
|
||||
val peer = selectRandomNode(peers)
|
||||
val oldState = state.get
|
||||
|
|
@ -297,8 +289,8 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
|
||||
private def newGossip(): Gossip = Gossip(
|
||||
version = VectorClock(),
|
||||
node = address.transport,
|
||||
availableNodes = Set(address.transport))
|
||||
node = address,
|
||||
availableNodes = Set(address))
|
||||
|
||||
private def incrementVersionForGossip(from: Gossip): Gossip = {
|
||||
val newVersion = from.version.increment(nodeFingerprint, newTimestamp)
|
||||
|
|
@ -313,7 +305,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) {
|
|||
}
|
||||
}
|
||||
|
||||
private def selectRandomNode(nodes: Set[ParsedTransportAddress]): ParsedTransportAddress = {
|
||||
private def selectRandomNode(nodes: Set[Address]): Address = {
|
||||
nodes.toList(random.nextInt(nodes.size))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,11 +12,11 @@ import akka.util.ReflectiveAccess
|
|||
|
||||
object MessageSerializer {
|
||||
|
||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = {
|
||||
def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = {
|
||||
val clazz = if (messageProtocol.hasMessageManifest) {
|
||||
Option(ReflectiveAccess.getClassFor[AnyRef](
|
||||
messageProtocol.getMessageManifest.toStringUtf8,
|
||||
classLoader.getOrElse(ReflectiveAccess.loader)) match {
|
||||
classLoader) match {
|
||||
case Left(e) ⇒ throw e
|
||||
case Right(r) ⇒ r
|
||||
})
|
||||
|
|
|
|||
|
|
@ -5,7 +5,8 @@
|
|||
package akka.remote
|
||||
|
||||
import scala.collection.mutable
|
||||
import akka.actor.{ Actor, Props, ActorSystemImpl }
|
||||
|
||||
import akka.actor.{ Props, Address, ActorSystemImpl, Actor }
|
||||
|
||||
/**
|
||||
* Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc.
|
||||
|
|
@ -15,10 +16,10 @@ object NetworkEventStream {
|
|||
|
||||
private sealed trait NetworkEventStreamEvent
|
||||
|
||||
private case class Register(listener: Listener, connectionAddress: ParsedTransportAddress)
|
||||
private case class Register(listener: Listener, connectionAddress: Address)
|
||||
extends NetworkEventStreamEvent
|
||||
|
||||
private case class Unregister(listener: Listener, connectionAddress: ParsedTransportAddress)
|
||||
private case class Unregister(listener: Listener, connectionAddress: Address)
|
||||
extends NetworkEventStreamEvent
|
||||
|
||||
/**
|
||||
|
|
@ -33,8 +34,8 @@ object NetworkEventStream {
|
|||
*/
|
||||
private class Channel extends Actor {
|
||||
|
||||
val listeners = new mutable.HashMap[ParsedTransportAddress, mutable.Set[Listener]]() {
|
||||
override def default(k: ParsedTransportAddress) = mutable.Set.empty[Listener]
|
||||
val listeners = new mutable.HashMap[Address, mutable.Set[Listener]]() {
|
||||
override def default(k: Address) = mutable.Set.empty[Listener]
|
||||
}
|
||||
|
||||
def receive = {
|
||||
|
|
@ -65,12 +66,12 @@ class NetworkEventStream(system: ActorSystemImpl) {
|
|||
/**
|
||||
* Registers a network event stream listener (asyncronously).
|
||||
*/
|
||||
def register(listener: Listener, connectionAddress: ParsedTransportAddress) =
|
||||
def register(listener: Listener, connectionAddress: Address) =
|
||||
sender ! Register(listener, connectionAddress)
|
||||
|
||||
/**
|
||||
* Unregisters a network event stream listener (asyncronously) .
|
||||
*/
|
||||
def unregister(listener: Listener, connectionAddress: ParsedTransportAddress) =
|
||||
def unregister(listener: Listener, connectionAddress: Address) =
|
||||
sender ! Unregister(listener, connectionAddress)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,274 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import akka.event._
|
||||
import akka.util._
|
||||
import akka.serialization.{ Serialization, SerializationExtension }
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.SystemMessage
|
||||
import scala.annotation.tailrec
|
||||
import akka.remote.RemoteProtocol.{ ActorRefProtocol, AkkaRemoteProtocol, RemoteControlProtocol, RemoteMessageProtocol }
|
||||
|
||||
/**
|
||||
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
||||
*/
|
||||
class Remote(val settings: ActorSystem.Settings, val remoteSettings: RemoteSettings) {
|
||||
|
||||
import settings._
|
||||
|
||||
// TODO make this really pluggable
|
||||
val transports: TransportsMap = Map("akka" -> ((h, p) ⇒ Right(RemoteNettyAddress(h, p))))
|
||||
val remoteAddress: RemoteSystemAddress[ParsedTransportAddress] = {
|
||||
val unparsedAddress = remoteSettings.serverSettings.URI match {
|
||||
case RemoteAddressExtractor(a) ⇒ a
|
||||
case x ⇒ throw new IllegalArgumentException("cannot parse URI " + x)
|
||||
}
|
||||
val parsed = unparsedAddress.parse(transports) match {
|
||||
case Left(x) ⇒ throw new IllegalArgumentException(x.transport.error)
|
||||
case Right(x) ⇒ x
|
||||
}
|
||||
parsed.copy(system = settings.name)
|
||||
}
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
|
||||
@volatile
|
||||
private var _computeGridDispatcher: MessageDispatcher = _
|
||||
def computeGridDispatcher = _computeGridDispatcher
|
||||
|
||||
@volatile
|
||||
private var _remoteDaemon: InternalActorRef = _
|
||||
def remoteDaemon = _remoteDaemon
|
||||
|
||||
@volatile
|
||||
private var _eventStream: NetworkEventStream = _
|
||||
def eventStream = _eventStream
|
||||
|
||||
@volatile
|
||||
private var _transport: RemoteSupport[ParsedTransportAddress] = _
|
||||
def transport = _transport
|
||||
|
||||
@volatile
|
||||
private var _provider: RemoteActorRefProvider = _
|
||||
def provider = _provider
|
||||
|
||||
def init(system: ActorSystemImpl, provider: RemoteActorRefProvider) = {
|
||||
|
||||
val log = Logging(system, "Remote")
|
||||
|
||||
_provider = provider
|
||||
_serialization = SerializationExtension(system)
|
||||
_computeGridDispatcher = system.dispatchers.lookup("akka.remote.compute-grid-dispatcher")
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log)
|
||||
_eventStream = new NetworkEventStream(system)
|
||||
_transport = {
|
||||
val arguments = Seq(
|
||||
classOf[ActorSystemImpl] -> system,
|
||||
classOf[Remote] -> this,
|
||||
classOf[RemoteSystemAddress[_ <: ParsedTransportAddress]] -> remoteAddress)
|
||||
val types: Array[Class[_]] = arguments map (_._1) toArray
|
||||
val values: Array[AnyRef] = arguments map (_._2) toArray
|
||||
|
||||
ReflectiveAccess.createInstance[RemoteSupport[ParsedTransportAddress]](remoteSettings.RemoteTransport, types, values) match {
|
||||
case Left(problem) ⇒
|
||||
|
||||
log.error(problem, "Could not load remote transport layer")
|
||||
throw problem
|
||||
|
||||
case Right(remote) ⇒
|
||||
|
||||
remote.start(Option(Thread.currentThread().getContextClassLoader)) //TODO Any application loader here?
|
||||
|
||||
val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case _ ⇒ //ignore other
|
||||
}
|
||||
}), "RemoteClientLifeCycleListener")
|
||||
|
||||
system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
remote
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Starting remote server on [{}@{}]", system.name, remoteAddress)
|
||||
}
|
||||
}
|
||||
|
||||
sealed trait DaemonMsg
|
||||
case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||
case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
|
||||
|
||||
/**
|
||||
* Internal system "daemon" actor for remote internal communication.
|
||||
*
|
||||
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
|
||||
*/
|
||||
class RemoteSystemDaemon(system: ActorSystemImpl, remote: Remote, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter)
|
||||
extends VirtualPathContainer(system.provider, _path, _parent, _log) {
|
||||
|
||||
/**
|
||||
* Find the longest matching path which we know about and return that ref
|
||||
* (or ask that ref to continue searching if elements are left).
|
||||
*/
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
|
||||
@tailrec
|
||||
def rec(s: String, n: Int): (InternalActorRef, Int) = {
|
||||
getChild(s) match {
|
||||
case null ⇒
|
||||
val last = s.lastIndexOf('/')
|
||||
if (last == -1) (Nobody, n)
|
||||
else rec(s.substring(0, last), n + 1)
|
||||
case ref ⇒ (ref, n)
|
||||
}
|
||||
}
|
||||
|
||||
val full = Vector() ++ names
|
||||
rec(full.mkString("/"), 0) match {
|
||||
case (Nobody, _) ⇒ Nobody
|
||||
case (ref, 0) ⇒ ref
|
||||
case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator)
|
||||
}
|
||||
}
|
||||
|
||||
override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match {
|
||||
case message: DaemonMsg ⇒
|
||||
log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address.hostPort)
|
||||
message match {
|
||||
case DaemonMsgCreate(factory, path, supervisor) ⇒
|
||||
import remote.remoteAddress
|
||||
implicit val t = remote.transports
|
||||
|
||||
path match {
|
||||
case ParsedActorPath(`remoteAddress`, elems) if elems.nonEmpty && elems.head == "remote" ⇒
|
||||
// TODO RK canonicalize path so as not to duplicate it always #1446
|
||||
val subpath = elems.drop(1)
|
||||
val path = remote.remoteDaemon.path / subpath
|
||||
val actor = system.provider.actorOf(system,
|
||||
Props(creator = factory),
|
||||
supervisor.asInstanceOf[InternalActorRef],
|
||||
path, true, None)
|
||||
addChild(subpath.mkString("/"), actor)
|
||||
system.deathWatch.subscribe(this, actor)
|
||||
case _ ⇒
|
||||
log.error("remote path does not match path from message [{}]", message)
|
||||
}
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
val other = system.actorFor(watcher.path.root / "remote")
|
||||
system.deathWatch.subscribe(other, watched)
|
||||
}
|
||||
|
||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
||||
|
||||
case t: Terminated ⇒ system.deathWatch.publish(t)
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, classLoader: Option[ClassLoader] = None) {
|
||||
|
||||
def originalReceiver = input.getRecipient.getPath
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath)
|
||||
else system.deadLetters
|
||||
|
||||
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
|
||||
|
||||
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, classLoader)
|
||||
|
||||
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
|
||||
}
|
||||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def log: LoggingAdapter
|
||||
|
||||
def system: ActorSystem
|
||||
|
||||
def remote: Remote
|
||||
|
||||
protected def useUntrustedMode: Boolean
|
||||
|
||||
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setMessage(rmp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setInstruction(rcp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
/**
|
||||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
ActorRefProtocol.newBuilder.setPath(actor.path.toString).build
|
||||
}
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
recipient: ActorRef,
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
||||
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
|
||||
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
||||
|
||||
messageBuilder
|
||||
}
|
||||
|
||||
def receiveMessage(remoteMessage: RemoteMessage) {
|
||||
if (remote.remoteSettings.LogReceivedMessages)
|
||||
log.debug("received message [{}]", remoteMessage)
|
||||
|
||||
val remoteDaemon = remote.remoteDaemon
|
||||
|
||||
remoteMessage.recipient match {
|
||||
case `remoteDaemon` ⇒
|
||||
remoteMessage.payload match {
|
||||
case m @ (_: DaemonMsg | _: Terminated) ⇒
|
||||
try remoteDaemon ! m catch {
|
||||
case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender)
|
||||
}
|
||||
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
||||
}
|
||||
case l: LocalRef ⇒
|
||||
remoteMessage.payload match {
|
||||
case msg: SystemMessage ⇒
|
||||
if (useUntrustedMode)
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message")
|
||||
else l.sendSystemMessage(msg)
|
||||
case _: AutoReceivedMessage if (useUntrustedMode) ⇒
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor")
|
||||
case m ⇒ l.!(m)(remoteMessage.sender)
|
||||
}
|
||||
case r: RemoteActorRef ⇒
|
||||
implicit val t = remote.transports
|
||||
remoteMessage.originalReceiver match {
|
||||
case ParsedActorPath(address, _) if address == remote.remoteDaemon.path.address ⇒
|
||||
r.!(remoteMessage.payload)(remoteMessage.sender)
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -6,9 +6,14 @@ package akka.remote
|
|||
|
||||
import akka.actor._
|
||||
import akka.dispatch._
|
||||
import akka.event.{ DeathWatch, Logging }
|
||||
import akka.event.{ DeathWatch, Logging, LoggingAdapter }
|
||||
import akka.event.EventStream
|
||||
import akka.config.ConfigurationException
|
||||
import java.util.concurrent.{ TimeoutException }
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.serialization.Serialization
|
||||
import akka.serialization.SerializationExtension
|
||||
|
||||
/**
|
||||
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
|
||||
|
|
@ -18,39 +23,93 @@ class RemoteActorRefProvider(
|
|||
val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
_deadLetters: InternalActorRef) extends ActorRefProvider {
|
||||
val classloader: ClassLoader) extends ActorRefProvider {
|
||||
|
||||
val remoteSettings = new RemoteSettings(settings.config, systemName)
|
||||
|
||||
val deployer = new RemoteDeployer(settings, classloader)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
|
||||
|
||||
@volatile
|
||||
private var _log = local.log
|
||||
def log: LoggingAdapter = _log
|
||||
|
||||
def rootPath = local.rootPath
|
||||
def locker = local.locker
|
||||
def deadLetters = local.deadLetters
|
||||
|
||||
val deathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
// these are only available after init()
|
||||
def rootGuardian = local.rootGuardian
|
||||
def guardian = local.guardian
|
||||
def systemGuardian = local.systemGuardian
|
||||
def terminationFuture = local.terminationFuture
|
||||
def dispatcher = local.dispatcher
|
||||
|
||||
def registerTempActor(actorRef: InternalActorRef, path: ActorPath) = local.registerTempActor(actorRef, path)
|
||||
def unregisterTempActor(path: ActorPath) = local.unregisterTempActor(path)
|
||||
def tempPath() = local.tempPath()
|
||||
def tempContainer = local.tempContainer
|
||||
|
||||
val deployer = new RemoteDeployer(settings)
|
||||
@volatile
|
||||
private var _transport: RemoteTransport = _
|
||||
def transport: RemoteTransport = _transport
|
||||
|
||||
val remote = new Remote(settings, remoteSettings)
|
||||
implicit val transports = remote.transports
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
|
||||
val log = Logging(eventStream, "RemoteActorRefProvider(" + remote.remoteAddress + ")")
|
||||
@volatile
|
||||
private var _remoteDaemon: InternalActorRef = _
|
||||
def remoteDaemon = _remoteDaemon
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(remote.remoteAddress)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath, deployer)
|
||||
|
||||
val deathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
@volatile
|
||||
private var _networkEventStream: NetworkEventStream = _
|
||||
def networkEventStream = _networkEventStream
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
local.init(system)
|
||||
remote.init(system, this)
|
||||
local.registerExtraNames(Map(("remote", remote.remoteDaemon)))
|
||||
terminationFuture.onComplete(_ ⇒ remote.transport.shutdown())
|
||||
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log)
|
||||
local.registerExtraNames(Map(("remote", remoteDaemon)))
|
||||
|
||||
_serialization = SerializationExtension(system)
|
||||
|
||||
_networkEventStream = new NetworkEventStream(system)
|
||||
system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
_transport = {
|
||||
val fqn = remoteSettings.RemoteTransport
|
||||
val args = Seq(
|
||||
classOf[RemoteSettings] -> remoteSettings,
|
||||
classOf[ActorSystemImpl] -> system,
|
||||
classOf[RemoteActorRefProvider] -> this)
|
||||
|
||||
ReflectiveAccess.createInstance[RemoteTransport](fqn, args, system.internalClassLoader) match {
|
||||
case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
|
||||
case Right(remote) ⇒ remote
|
||||
}
|
||||
}
|
||||
|
||||
_log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")")
|
||||
|
||||
// this enables reception of remote requests
|
||||
_transport.start()
|
||||
|
||||
val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case _ ⇒ //ignore other
|
||||
}
|
||||
}), "RemoteClientLifeCycleListener")
|
||||
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
terminationFuture.onComplete(_ ⇒ transport.shutdown())
|
||||
}
|
||||
|
||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy]): InternalActorRef = {
|
||||
|
|
@ -100,18 +159,12 @@ class RemoteActorRefProvider(
|
|||
})
|
||||
|
||||
deployment match {
|
||||
case Some(Deploy(_, _, _, RemoteScope(address))) ⇒
|
||||
// FIXME RK this should be done within the deployer, i.e. the whole parsing business
|
||||
address.parse(remote.transports) match {
|
||||
case Left(x) ⇒
|
||||
throw new ConfigurationException("cannot parse remote address: " + x)
|
||||
case Right(addr) ⇒
|
||||
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 {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,5 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote
|
||||
|
||||
|
|
@ -18,14 +18,14 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
*/
|
||||
class RemoteConnectionManager(
|
||||
system: ActorSystemImpl,
|
||||
remote: Remote,
|
||||
initialConnections: Map[ParsedTransportAddress, ActorRef] = Map.empty[ParsedTransportAddress, ActorRef])
|
||||
remote: RemoteActorRefProvider,
|
||||
initialConnections: Map[Address, ActorRef] = Map.empty[Address, ActorRef])
|
||||
extends ConnectionManager {
|
||||
|
||||
val log = Logging(system, "RemoteConnectionManager")
|
||||
|
||||
// FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc.
|
||||
case class State(version: Long, connections: Map[ParsedTransportAddress, ActorRef])
|
||||
case class State(version: Long, connections: Map[Address, ActorRef])
|
||||
extends VersionedIterable[ActorRef] {
|
||||
def iterable: Iterable[ActorRef] = connections.values
|
||||
}
|
||||
|
|
@ -51,7 +51,7 @@ class RemoteConnectionManager(
|
|||
|
||||
def size: Int = connections.connections.size
|
||||
|
||||
def connectionFor(address: ParsedTransportAddress): Option[ActorRef] = connections.connections.get(address)
|
||||
def connectionFor(address: Address): Option[ActorRef] = connections.connections.get(address)
|
||||
|
||||
def isEmpty: Boolean = connections.connections.isEmpty
|
||||
|
||||
|
|
@ -60,7 +60,7 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
@tailrec
|
||||
final def failOver(from: ParsedTransportAddress, to: ParsedTransportAddress) {
|
||||
final def failOver(from: Address, to: Address) {
|
||||
log.debug("Failing over connection from [{}] to [{}]", from, to)
|
||||
|
||||
val oldState = state.get
|
||||
|
|
@ -91,8 +91,8 @@ class RemoteConnectionManager(
|
|||
val oldState = state.get()
|
||||
var changed = false
|
||||
|
||||
var faultyAddress: ParsedTransportAddress = null
|
||||
var newConnections = Map.empty[ParsedTransportAddress, ActorRef]
|
||||
var faultyAddress: Address = null
|
||||
var newConnections = Map.empty[Address, ActorRef]
|
||||
|
||||
oldState.connections.keys foreach { address ⇒
|
||||
val actorRef: ActorRef = oldState.connections.get(address).get
|
||||
|
|
@ -118,7 +118,7 @@ class RemoteConnectionManager(
|
|||
}
|
||||
|
||||
@tailrec
|
||||
final def putIfAbsent(address: ParsedTransportAddress, newConnectionFactory: () ⇒ ActorRef): ActorRef = {
|
||||
final def putIfAbsent(address: Address, newConnectionFactory: () ⇒ ActorRef): ActorRef = {
|
||||
|
||||
val oldState = state.get()
|
||||
val oldConnections = oldState.connections
|
||||
|
|
@ -145,6 +145,6 @@ class RemoteConnectionManager(
|
|||
}
|
||||
}
|
||||
|
||||
private[remote] def newConnection(remoteAddress: ParsedTransportAddress, actorPath: ActorPath) =
|
||||
new RemoteActorRef(remote.provider, remote.transport, actorPath, Nobody, None)
|
||||
private[remote] def newConnection(remoteAddress: Address, actorPath: ActorPath) =
|
||||
new RemoteActorRef(remote, remote.transport, actorPath, Nobody)
|
||||
}
|
||||
|
|
|
|||
83
akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
Normal file
83
akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala
Normal file
|
|
@ -0,0 +1,83 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import scala.annotation.tailrec
|
||||
|
||||
import akka.actor.{ VirtualPathContainer, Terminated, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
|
||||
import akka.event.LoggingAdapter
|
||||
|
||||
sealed trait DaemonMsg
|
||||
case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||
case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg
|
||||
|
||||
/**
|
||||
* Internal system "daemon" actor for remote internal communication.
|
||||
*
|
||||
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
|
||||
*
|
||||
* INTERNAL USE ONLY!
|
||||
*/
|
||||
private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter)
|
||||
extends VirtualPathContainer(system.provider, _path, _parent, _log) {
|
||||
|
||||
/**
|
||||
* Find the longest matching path which we know about and return that ref
|
||||
* (or ask that ref to continue searching if elements are left).
|
||||
*/
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
|
||||
@tailrec
|
||||
def rec(s: String, n: Int): (InternalActorRef, Int) = {
|
||||
getChild(s) match {
|
||||
case null ⇒
|
||||
val last = s.lastIndexOf('/')
|
||||
if (last == -1) (Nobody, n)
|
||||
else rec(s.substring(0, last), n + 1)
|
||||
case ref ⇒ (ref, n)
|
||||
}
|
||||
}
|
||||
|
||||
val full = Vector() ++ names
|
||||
rec(full.mkString("/"), 0) match {
|
||||
case (Nobody, _) ⇒ Nobody
|
||||
case (ref, 0) ⇒ ref
|
||||
case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator)
|
||||
}
|
||||
}
|
||||
|
||||
override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match {
|
||||
case message: DaemonMsg ⇒
|
||||
log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address)
|
||||
message match {
|
||||
case DaemonMsgCreate(factory, path, supervisor) ⇒
|
||||
path match {
|
||||
case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒
|
||||
// TODO RK currently the extracted “address” is just ignored, is that okay?
|
||||
// TODO RK canonicalize path so as not to duplicate it always #1446
|
||||
val subpath = elems.drop(1)
|
||||
val path = this.path / subpath
|
||||
val actor = system.provider.actorOf(system,
|
||||
Props(creator = factory),
|
||||
supervisor.asInstanceOf[InternalActorRef],
|
||||
path, true, None)
|
||||
addChild(subpath.mkString("/"), actor)
|
||||
system.deathWatch.subscribe(this, actor)
|
||||
case _ ⇒
|
||||
log.error("remote path does not match path from message [{}]", message)
|
||||
}
|
||||
case DaemonMsgWatch(watcher, watched) ⇒
|
||||
val other = system.actorFor(watcher.path.root / "remote")
|
||||
system.deathWatch.subscribe(other, watched)
|
||||
}
|
||||
|
||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
||||
|
||||
case t: Terminated ⇒ system.deathWatch.publish(t)
|
||||
|
||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -8,9 +8,9 @@ import akka.routing._
|
|||
import com.typesafe.config._
|
||||
import akka.config.ConfigurationException
|
||||
|
||||
case class RemoteScope(node: UnparsedSystemAddress[UnparsedTransportAddress]) extends Scope
|
||||
case class RemoteScope(node: Address) extends Scope
|
||||
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings) {
|
||||
class RemoteDeployer(_settings: ActorSystem.Settings, _classloader: ClassLoader) extends Deployer(_settings, _classloader) {
|
||||
|
||||
override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -18,7 +18,7 @@ class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings
|
|||
super.parseConfig(path, config) match {
|
||||
case d @ Some(deploy) ⇒
|
||||
deploy.config.getString("remote") match {
|
||||
case RemoteAddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r)))
|
||||
case AddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r)))
|
||||
case str ⇒
|
||||
if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str)
|
||||
val nodes = deploy.config.getStringList("target.nodes").asScala
|
||||
|
|
|
|||
|
|
@ -1,340 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import akka.actor._
|
||||
import akka.AkkaException
|
||||
import scala.reflect.BeanProperty
|
||||
import java.net.URI
|
||||
import java.net.URISyntaxException
|
||||
import java.net.InetAddress
|
||||
import java.net.UnknownHostException
|
||||
import akka.event.Logging
|
||||
|
||||
/**
|
||||
* Interface for remote transports to encode their addresses. The three parts
|
||||
* are named according to the URI spec (precisely java.net.URI) which is used
|
||||
* for parsing. That means that the address’ parts must conform to what an
|
||||
* URI expects, but otherwise each transport may assign a different meaning
|
||||
* to these parts.
|
||||
*/
|
||||
trait RemoteTransportAddress {
|
||||
def protocol: String
|
||||
def host: String
|
||||
def port: Int
|
||||
}
|
||||
|
||||
trait ParsedTransportAddress extends RemoteTransportAddress
|
||||
|
||||
case class RemoteNettyAddress(host: String, ip: Option[InetAddress], port: Int) extends ParsedTransportAddress {
|
||||
def protocol = "akka"
|
||||
|
||||
override def toString(): String = "akka://" + host + ":" + port
|
||||
}
|
||||
|
||||
object RemoteNettyAddress {
|
||||
def apply(host: String, port: Int): RemoteNettyAddress = {
|
||||
// TODO ticket #1639
|
||||
val ip = try Some(InetAddress.getByName(host)) catch { case _: UnknownHostException ⇒ None }
|
||||
new RemoteNettyAddress(host, ip, port)
|
||||
}
|
||||
def apply(s: String): RemoteNettyAddress = {
|
||||
val RE = """([^:]+):(\d+)""".r
|
||||
s match {
|
||||
case RE(h, p) ⇒ apply(h, Integer.parseInt(p))
|
||||
case _ ⇒ throw new IllegalArgumentException("cannot parse " + s + " as <host:port>")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class UnparsedTransportAddress(protocol: String, host: String, port: Int) extends RemoteTransportAddress {
|
||||
def parse(transports: TransportsMap): RemoteTransportAddress =
|
||||
transports.get(protocol)
|
||||
.map(_(host, port))
|
||||
.toRight("protocol " + protocol + " not known")
|
||||
.joinRight.fold(UnparseableTransportAddress(protocol, host, port, _), identity)
|
||||
}
|
||||
|
||||
case class UnparseableTransportAddress(protocol: String, host: String, port: Int, error: String) extends RemoteTransportAddress
|
||||
|
||||
case class RemoteSystemAddress[+T <: ParsedTransportAddress](system: String, transport: T) extends Address {
|
||||
def protocol = transport.protocol
|
||||
@transient
|
||||
lazy val hostPort = system + "@" + transport.host + ":" + transport.port
|
||||
}
|
||||
|
||||
case class UnparsedSystemAddress[+T <: RemoteTransportAddress](system: Option[String], transport: T) {
|
||||
def parse(transports: TransportsMap): Either[UnparsedSystemAddress[UnparseableTransportAddress], RemoteSystemAddress[ParsedTransportAddress]] =
|
||||
system match {
|
||||
case Some(sys) ⇒
|
||||
transport match {
|
||||
case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x))
|
||||
case y: UnparsedTransportAddress ⇒
|
||||
y.parse(transports) match {
|
||||
case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x))
|
||||
case y: UnparseableTransportAddress ⇒ Left(UnparsedSystemAddress(system, y))
|
||||
case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z)))
|
||||
}
|
||||
case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z)))
|
||||
}
|
||||
case None ⇒ Left(UnparsedSystemAddress(None, UnparseableTransportAddress(transport.protocol, transport.host, transport.port, "no system name specified")))
|
||||
}
|
||||
}
|
||||
|
||||
object RemoteAddressExtractor {
|
||||
def unapply(s: String): Option[UnparsedSystemAddress[UnparsedTransportAddress]] = {
|
||||
try {
|
||||
val uri = new URI(s)
|
||||
if (uri.getScheme == null || uri.getHost == null || uri.getPort == -1) None
|
||||
else Some(UnparsedSystemAddress(Option(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)))
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object RemoteActorPath {
|
||||
def unapply(addr: String): Option[(UnparsedSystemAddress[UnparsedTransportAddress], Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None
|
||||
else Some(UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)),
|
||||
ActorPath.split(uri.getPath).drop(1))
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ParsedActorPath {
|
||||
def unapply(addr: String)(implicit transports: TransportsMap): Option[(RemoteSystemAddress[ParsedTransportAddress], Iterable[String])] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None
|
||||
else
|
||||
UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)).parse(transports) match {
|
||||
case Left(_) ⇒ None
|
||||
case Right(x) ⇒ Some(x, ActorPath.split(uri.getPath).drop(1))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteException(message: String) extends AkkaException(message)
|
||||
|
||||
trait RemoteModule {
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* Remote life-cycle events.
|
||||
*/
|
||||
sealed trait RemoteLifeCycleEvent {
|
||||
def logLevel: Logging.LogLevel
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteClient.
|
||||
*/
|
||||
trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent {
|
||||
def remoteAddress: ParsedTransportAddress
|
||||
}
|
||||
|
||||
case class RemoteClientError[T <: ParsedTransportAddress](
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteClientError@" +
|
||||
remoteAddress +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteClientDisconnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientDisconnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientConnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientConnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientStarted[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientStarted@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientShutdown[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientShutdown@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientWriteFailed[T <: ParsedTransportAddress](
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteClientWriteFailed@" +
|
||||
remoteAddress +
|
||||
": MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteServer.
|
||||
*/
|
||||
trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent
|
||||
|
||||
case class RemoteServerStarted[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerStarted@" + remote.name
|
||||
}
|
||||
|
||||
case class RemoteServerShutdown[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerShutdown@" + remote.name
|
||||
}
|
||||
|
||||
case class RemoteServerError[T <: ParsedTransportAddress](
|
||||
@BeanProperty val cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteServerError@" +
|
||||
remote.name +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientConnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientConnected@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientDisconnected[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientDisconnected@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientClosed[T <: ParsedTransportAddress](
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientClosed@" +
|
||||
remote.name +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerWriteFailed[T <: ParsedTransportAddress](
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteSupport[T],
|
||||
@BeanProperty remoteAddress: Option[T]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteServerWriteFailed@" +
|
||||
remote +
|
||||
": ClientAddress[" +
|
||||
remoteAddress +
|
||||
"] MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
|
||||
*/
|
||||
class RemoteClientException[T <: ParsedTransportAddress] private[akka] (
|
||||
message: String,
|
||||
@BeanProperty val client: RemoteSupport[T],
|
||||
val remoteAddress: T, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
||||
abstract class RemoteSupport[-T <: ParsedTransportAddress](val system: ActorSystemImpl) {
|
||||
/**
|
||||
* Shuts down the remoting
|
||||
*/
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Gets the name of the server instance
|
||||
*/
|
||||
def name: String
|
||||
|
||||
/**
|
||||
* Starts up the remoting
|
||||
*/
|
||||
def start(loader: Option[ClassLoader]): Unit
|
||||
|
||||
/**
|
||||
* Shuts down a specific client connected to the supplied remote address returns true if successful
|
||||
*/
|
||||
def shutdownClientConnection(address: T): Boolean
|
||||
|
||||
/**
|
||||
* Restarts a specific client connected to the supplied remote address, but only if the client is not shut down
|
||||
*/
|
||||
def restartClientConnection(address: T): Boolean
|
||||
|
||||
/** Methods that needs to be implemented by a transport **/
|
||||
|
||||
protected[akka] def send(message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef,
|
||||
loader: Option[ClassLoader]): Unit
|
||||
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = {
|
||||
system.eventStream.publish(message)
|
||||
system.log.log(message.logLevel, "REMOTE: {}", message)
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
|
@ -9,92 +9,29 @@ import java.util.concurrent.TimeUnit.MILLISECONDS
|
|||
import java.net.InetAddress
|
||||
import akka.config.ConfigurationException
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.actor.Address
|
||||
import akka.actor.AddressExtractor
|
||||
|
||||
class RemoteSettings(val config: Config, val systemName: String) {
|
||||
|
||||
import config._
|
||||
|
||||
final val RemoteTransport = getString("akka.remote.transport")
|
||||
final val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold")
|
||||
final val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size")
|
||||
final val ShouldCompressData = getBoolean("akka.remote.use-compression")
|
||||
final val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS)
|
||||
final val InitalDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS)
|
||||
final val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS)
|
||||
final val BackoffTimeout = Duration(getMilliseconds("akka.remote.backoff-timeout"), MILLISECONDS)
|
||||
final val LogReceivedMessages = getBoolean("akka.remote.log-received-messages")
|
||||
final val LogSentMessages = getBoolean("akka.remote.log-sent-messages")
|
||||
val RemoteTransport = getString("akka.remote.transport")
|
||||
val LogReceive = getBoolean("akka.remote.log-received-messages")
|
||||
val LogSend = getBoolean("akka.remote.log-sent-messages")
|
||||
|
||||
// AccrualFailureDetector
|
||||
val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold")
|
||||
val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size")
|
||||
|
||||
// Gossiper
|
||||
val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS)
|
||||
val InitialDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS)
|
||||
val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS)
|
||||
// TODO cluster config will go into akka-cluster/reference.conf when we enable that module
|
||||
final val SeedNodes = Set.empty[RemoteNettyAddress] ++ getStringList("akka.cluster.seed-nodes").asScala.collect {
|
||||
case RemoteAddressExtractor(addr) ⇒ addr.transport
|
||||
val SeedNodes = Set.empty[Address] ++ getStringList("akka.cluster.seed-nodes").asScala.collect {
|
||||
case AddressExtractor(addr) ⇒ addr
|
||||
}
|
||||
|
||||
final val serverSettings = new RemoteServerSettings
|
||||
final val clientSettings = new RemoteClientSettings
|
||||
|
||||
class RemoteClientSettings {
|
||||
val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
|
||||
final val ConnectionTimeout = Duration(getMilliseconds("akka.remote.client.connection-timeout"), MILLISECONDS)
|
||||
final val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS)
|
||||
final val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS)
|
||||
final val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS)
|
||||
final val MessageFrameSize = getBytes("akka.remote.client.message-frame-size").toInt
|
||||
}
|
||||
|
||||
class RemoteServerSettings {
|
||||
import scala.collection.JavaConverters._
|
||||
final val MessageFrameSize = getBytes("akka.remote.server.message-frame-size").toInt
|
||||
final val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
final val RequireCookie = {
|
||||
val requireCookie = getBoolean("akka.remote.server.require-cookie")
|
||||
if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
|
||||
requireCookie
|
||||
}
|
||||
|
||||
final val UsePassiveConnections = getBoolean("akka.remote.use-passive-connections")
|
||||
|
||||
final val UntrustedMode = getBoolean("akka.remote.server.untrusted-mode")
|
||||
final val Hostname = getString("akka.remote.server.hostname") match {
|
||||
case "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
}
|
||||
final val Port = getInt("akka.remote.server.port") match {
|
||||
case 0 ⇒ try {
|
||||
val s = new java.net.ServerSocket(0)
|
||||
try s.getLocalPort finally s.close()
|
||||
} catch { case e ⇒ throw new ConfigurationException("Unable to obtain random port", e) }
|
||||
case other ⇒ other
|
||||
}
|
||||
|
||||
final val Backlog = getInt("akka.remote.server.backlog")
|
||||
|
||||
final val ExecutionPoolKeepAlive = Duration(getMilliseconds("akka.remote.server.execution-pool-keepalive"), MILLISECONDS)
|
||||
|
||||
final val ExecutionPoolSize = getInt("akka.remote.server.execution-pool-size") match {
|
||||
case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.server.execution-pool-size is less than 1")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
final val MaxChannelMemorySize = getBytes("akka.remote.server.max-channel-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-channel-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
final val MaxTotalMemorySize = getBytes("akka.remote.server.max-total-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-total-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
// TODO handle the system name right and move this to config file syntax
|
||||
final val URI = "akka://sys@" + Hostname + ":" + Port
|
||||
}
|
||||
val UntrustedMode = getBoolean("akka.remote.untrusted-mode")
|
||||
}
|
||||
334
akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
Normal file
334
akka-remote/src/main/scala/akka/remote/RemoteTransport.scala
Normal file
|
|
@ -0,0 +1,334 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote
|
||||
|
||||
import scala.reflect.BeanProperty
|
||||
import akka.actor.{ Terminated, LocalRef, InternalActorRef, AutoReceivedMessage, AddressExtractor, Address, ActorSystemImpl, ActorSystem, ActorRef }
|
||||
import akka.dispatch.SystemMessage
|
||||
import akka.event.{ LoggingAdapter, Logging }
|
||||
import akka.remote.RemoteProtocol.{ RemoteMessageProtocol, RemoteControlProtocol, AkkaRemoteProtocol, ActorRefProtocol }
|
||||
import akka.AkkaException
|
||||
import akka.serialization.Serialization
|
||||
|
||||
/**
|
||||
* Remote life-cycle events.
|
||||
*/
|
||||
sealed trait RemoteLifeCycleEvent {
|
||||
def logLevel: Logging.LogLevel
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteClient.
|
||||
*/
|
||||
trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent {
|
||||
def remoteAddress: Address
|
||||
}
|
||||
|
||||
case class RemoteClientError(
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteClientError@" +
|
||||
remoteAddress +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteClientDisconnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientDisconnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientConnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteClientConnected@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientStarted(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientStarted@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientShutdown(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteClientShutdown@" + remoteAddress
|
||||
}
|
||||
|
||||
case class RemoteClientWriteFailed(
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteClientWriteFailed@" +
|
||||
remoteAddress +
|
||||
": MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteServer.
|
||||
*/
|
||||
trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent
|
||||
|
||||
case class RemoteServerStarted(
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerStarted@" + remote
|
||||
}
|
||||
|
||||
case class RemoteServerShutdown(
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.InfoLevel
|
||||
override def toString =
|
||||
"RemoteServerShutdown@" + remote
|
||||
}
|
||||
|
||||
case class RemoteServerError(
|
||||
@BeanProperty val cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.ErrorLevel
|
||||
override def toString =
|
||||
"RemoteServerError@" +
|
||||
remote +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientConnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientConnected@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientDisconnected(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientDisconnected@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerClientClosed(
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.DebugLevel
|
||||
override def toString =
|
||||
"RemoteServerClientClosed@" +
|
||||
remote +
|
||||
": Client[" +
|
||||
(if (clientAddress.isDefined) clientAddress.get else "no address") +
|
||||
"]"
|
||||
}
|
||||
|
||||
case class RemoteServerWriteFailed(
|
||||
@BeanProperty request: AnyRef,
|
||||
@BeanProperty cause: Throwable,
|
||||
@BeanProperty remote: RemoteTransport,
|
||||
@BeanProperty remoteAddress: Option[Address]) extends RemoteServerLifeCycleEvent {
|
||||
override def logLevel = Logging.WarningLevel
|
||||
override def toString =
|
||||
"RemoteServerWriteFailed@" +
|
||||
remote +
|
||||
": ClientAddress[" +
|
||||
remoteAddress +
|
||||
"] MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
/**
|
||||
* Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
|
||||
*/
|
||||
class RemoteClientException private[akka] (
|
||||
message: String,
|
||||
@BeanProperty val client: RemoteTransport,
|
||||
val remoteAddress: Address, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
||||
class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause)
|
||||
|
||||
/**
|
||||
* The remote transport is responsible for sending and receiving messages.
|
||||
* Each transport has an address, which it should provide in
|
||||
* Serialization.currentTransportAddress (thread-local) while serializing
|
||||
* actor references (which might also be part of messages). This address must
|
||||
* be available (i.e. fully initialized) by the time the first message is
|
||||
* received or when the start() method returns, whatever happens first.
|
||||
*/
|
||||
abstract class RemoteTransport {
|
||||
/**
|
||||
* Shuts down the remoting
|
||||
*/
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Address to be used in RootActorPath of refs generated for this transport.
|
||||
*/
|
||||
def address: Address
|
||||
|
||||
/**
|
||||
* The actor system, for which this transport is instantiated. Will publish to its eventStream.
|
||||
*/
|
||||
def system: ActorSystem
|
||||
|
||||
/**
|
||||
* Start up the transport, i.e. enable incoming connections.
|
||||
*/
|
||||
def start(): Unit
|
||||
|
||||
/**
|
||||
* Shuts down a specific client connected to the supplied remote address returns true if successful
|
||||
*/
|
||||
def shutdownClientConnection(address: Address): Boolean
|
||||
|
||||
/**
|
||||
* Restarts a specific client connected to the supplied remote address, but only if the client is not shut down
|
||||
*/
|
||||
def restartClientConnection(address: Address): Boolean
|
||||
|
||||
/** Methods that needs to be implemented by a transport **/
|
||||
|
||||
protected[akka] def send(message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef): Unit
|
||||
|
||||
protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = {
|
||||
system.eventStream.publish(message)
|
||||
system.log.log(message.logLevel, "REMOTE: {}", message)
|
||||
}
|
||||
|
||||
override def toString = address.toString
|
||||
}
|
||||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) {
|
||||
|
||||
def originalReceiver = input.getRecipient.getPath
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath)
|
||||
else system.deadLetters
|
||||
|
||||
lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver)
|
||||
|
||||
lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, getClass.getClassLoader)
|
||||
|
||||
override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender
|
||||
}
|
||||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def log: LoggingAdapter
|
||||
|
||||
def system: ActorSystemImpl
|
||||
|
||||
def provider: RemoteActorRefProvider
|
||||
|
||||
def address: Address
|
||||
|
||||
protected def useUntrustedMode: Boolean
|
||||
|
||||
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setMessage(rmp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = {
|
||||
val arp = AkkaRemoteProtocol.newBuilder
|
||||
arp.setInstruction(rcp)
|
||||
arp.build
|
||||
}
|
||||
|
||||
/**
|
||||
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
||||
*/
|
||||
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
||||
ActorRefProtocol.newBuilder.setPath(actor.path.toStringWithAddress(address)).build
|
||||
}
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
recipient: ActorRef,
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
||||
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
||||
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
||||
|
||||
Serialization.currentTransportAddress.withValue(address) {
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
}
|
||||
|
||||
messageBuilder
|
||||
}
|
||||
|
||||
def receiveMessage(remoteMessage: RemoteMessage) {
|
||||
val remoteDaemon = provider.remoteDaemon
|
||||
|
||||
remoteMessage.recipient match {
|
||||
case `remoteDaemon` ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage)
|
||||
remoteMessage.payload match {
|
||||
case m @ (_: DaemonMsg | _: Terminated) ⇒
|
||||
try remoteDaemon ! m catch {
|
||||
case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender)
|
||||
}
|
||||
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
||||
}
|
||||
case l: LocalRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
|
||||
remoteMessage.payload match {
|
||||
case msg: SystemMessage ⇒
|
||||
if (useUntrustedMode)
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message")
|
||||
else l.sendSystemMessage(msg)
|
||||
case _: AutoReceivedMessage if (useUntrustedMode) ⇒
|
||||
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor")
|
||||
case m ⇒ l.!(m)(remoteMessage.sender)
|
||||
}
|
||||
case r: RemoteRef ⇒
|
||||
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
|
||||
remoteMessage.originalReceiver match {
|
||||
case AddressExtractor(address) if address == provider.transport.address ⇒
|
||||
// if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed)
|
||||
r.!(remoteMessage.payload)(remoteMessage.sender)
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
||||
}
|
||||
}
|
||||
}
|
||||
347
akka-remote/src/main/scala/akka/remote/netty/Client.scala
Normal file
347
akka-remote/src/main/scala/akka/remote/netty/Client.scala
Normal file
|
|
@ -0,0 +1,347 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import org.jboss.netty.util.HashedWheelTimer
|
||||
import org.jboss.netty.bootstrap.ClientBootstrap
|
||||
import org.jboss.netty.channel.group.DefaultChannelGroup
|
||||
import org.jboss.netty.channel.{ ChannelHandler, StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected }
|
||||
import akka.actor.{ simpleName, Address }
|
||||
import akka.AkkaException
|
||||
import akka.event.Logging
|
||||
import akka.util.Switch
|
||||
import akka.actor.ActorRef
|
||||
import org.jboss.netty.channel.ChannelFutureListener
|
||||
import akka.remote.RemoteClientWriteFailed
|
||||
import java.net.InetAddress
|
||||
import org.jboss.netty.util.TimerTask
|
||||
import org.jboss.netty.util.Timeout
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
class RemoteClientMessageBufferException(message: String, cause: Throwable) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the abstract baseclass for netty remote clients, currently there's only an
|
||||
* ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that
|
||||
* reuses an already established connection.
|
||||
*/
|
||||
abstract class RemoteClient private[akka] (
|
||||
val netty: NettyRemoteTransport,
|
||||
val remoteAddress: Address) {
|
||||
|
||||
val log = Logging(netty.system, "RemoteClient")
|
||||
|
||||
val name = simpleName(this) + "@" + remoteAddress
|
||||
|
||||
private[remote] val runSwitch = new Switch()
|
||||
|
||||
private[remote] def isRunning = runSwitch.isOn
|
||||
|
||||
protected def currentChannel: Channel
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean
|
||||
|
||||
def shutdown(): Boolean
|
||||
|
||||
def isBoundTo(address: Address): Boolean = remoteAddress == address
|
||||
|
||||
/**
|
||||
* Converts the message to the wireprotocol and sends the message across the wire
|
||||
*/
|
||||
def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) {
|
||||
if (netty.remoteSettings.LogSend) log.debug("Sending message {} from {} to {}", message, senderOption, recipient)
|
||||
send((message, senderOption, recipient))
|
||||
} else {
|
||||
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", netty, remoteAddress)
|
||||
netty.notifyListeners(RemoteClientError(exception, netty, remoteAddress))
|
||||
throw exception
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the message across the wire
|
||||
*/
|
||||
private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = {
|
||||
try {
|
||||
val channel = currentChannel
|
||||
val f = channel.write(request)
|
||||
f.addListener(
|
||||
new ChannelFutureListener {
|
||||
def operationComplete(future: ChannelFuture) {
|
||||
if (future.isCancelled || !future.isSuccess) {
|
||||
netty.notifyListeners(RemoteClientWriteFailed(request, future.getCause, netty, remoteAddress))
|
||||
}
|
||||
}
|
||||
})
|
||||
// Check if we should back off
|
||||
if (!channel.isWritable) {
|
||||
val backoff = netty.settings.BackoffTimeout
|
||||
if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
||||
/**
|
||||
* RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node.
|
||||
*/
|
||||
class ActiveRemoteClient private[akka] (
|
||||
netty: NettyRemoteTransport,
|
||||
remoteAddress: Address,
|
||||
localAddress: Address)
|
||||
extends RemoteClient(netty, remoteAddress) {
|
||||
|
||||
import netty.settings
|
||||
|
||||
//TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
|
||||
@volatile
|
||||
private var bootstrap: ClientBootstrap = _
|
||||
@volatile
|
||||
private var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var executionHandler: ExecutionHandler = _
|
||||
|
||||
@volatile
|
||||
private var reconnectionTimeWindowStart = 0L
|
||||
|
||||
def notifyListeners(msg: RemoteLifeCycleEvent): Unit = netty.notifyListeners(msg)
|
||||
|
||||
def currentChannel = connection.getChannel
|
||||
|
||||
/**
|
||||
* Connect to remote server.
|
||||
*/
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
|
||||
|
||||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (settings.SecureCookie.nonEmpty) handshake.setCookie(settings.SecureCookie.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(localAddress.system)
|
||||
.setHostname(localAddress.host.get)
|
||||
.setPort(localAddress.port.get)
|
||||
.build)
|
||||
connection.getChannel.write(netty.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
def attemptReconnect(): Boolean = {
|
||||
val remoteIP = InetAddress.getByName(remoteAddress.host.get)
|
||||
log.debug("Remote client reconnecting to [{}|{}]", remoteAddress, remoteIP)
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get))
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
runSwitch switchOn {
|
||||
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
|
||||
|
||||
executionHandler = new ExecutionHandler(netty.executor)
|
||||
|
||||
val b = new ClientBootstrap(netty.clientChannelFactory)
|
||||
b.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, b, executionHandler, remoteAddress, this))
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("keepAlive", true)
|
||||
b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis)
|
||||
bootstrap = b
|
||||
|
||||
val remoteIP = InetAddress.getByName(remoteAddress.host.get)
|
||||
log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP)
|
||||
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get))
|
||||
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
true
|
||||
}
|
||||
} match {
|
||||
case true ⇒ true
|
||||
case false if reconnectIfAlreadyConnected ⇒
|
||||
connection.getChannel.close()
|
||||
openChannels.remove(connection.getChannel)
|
||||
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
attemptReconnect()
|
||||
|
||||
case false ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
notifyListeners(RemoteClientShutdown(netty, remoteAddress))
|
||||
try {
|
||||
if ((connection ne null) && (connection.getChannel ne null))
|
||||
connection.getChannel.close()
|
||||
} finally {
|
||||
try {
|
||||
if (openChannels ne null) openChannels.close.awaitUninterruptibly()
|
||||
} finally {
|
||||
connection = null
|
||||
executionHandler = null
|
||||
}
|
||||
}
|
||||
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
|
||||
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
|
||||
if (reconnectionTimeWindowStart == 0L) {
|
||||
reconnectionTimeWindowStart = System.currentTimeMillis
|
||||
true
|
||||
} else {
|
||||
val timeLeft = (settings.ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
|
||||
if (timeLeft)
|
||||
log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
|
||||
|
||||
timeLeft
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class ActiveRemoteClientHandler(
|
||||
val name: String,
|
||||
val bootstrap: ClientBootstrap,
|
||||
val remoteAddress: Address,
|
||||
val timer: HashedWheelTimer,
|
||||
val client: ActiveRemoteClient)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() }
|
||||
}, 0, TimeUnit.MILLISECONDS)
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
|
||||
try {
|
||||
event.getMessage match {
|
||||
case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒
|
||||
val rcp = arp.getInstruction
|
||||
rcp.getCommandType match {
|
||||
case CommandType.SHUTDOWN ⇒ runOnceNow { client.netty.shutdownClientConnection(remoteAddress) }
|
||||
case _ ⇒ //Ignore others
|
||||
}
|
||||
|
||||
case arp: AkkaRemoteProtocol if arp.hasMessage ⇒
|
||||
client.netty.receiveMessage(new RemoteMessage(arp.getMessage, client.netty.system))
|
||||
|
||||
case other ⇒
|
||||
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.netty, client.remoteAddress)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
|
||||
if (client.isWithinReconnectionTimeWindow) {
|
||||
timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) =
|
||||
if (client.isRunning) {
|
||||
client.openChannels.remove(event.getChannel)
|
||||
client.connect(reconnectIfAlreadyConnected = true)
|
||||
}
|
||||
}, client.netty.settings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS)
|
||||
} else runOnceNow {
|
||||
client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
}
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
try {
|
||||
client.notifyListeners(RemoteClientConnected(client.netty, client.remoteAddress))
|
||||
client.resetReconnectionTimeWindow
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
client.notifyListeners(RemoteClientDisconnected(client.netty, client.remoteAddress))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
val cause = event.getCause
|
||||
if (cause ne null) {
|
||||
client.notifyListeners(RemoteClientError(cause, client.netty, client.remoteAddress))
|
||||
cause match {
|
||||
case e: ReadTimeoutException ⇒
|
||||
runOnceNow {
|
||||
client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
case e: Exception ⇒ event.getChannel.close()
|
||||
}
|
||||
|
||||
} else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.netty, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
class ActiveRemoteClientPipelineFactory(
|
||||
name: String,
|
||||
bootstrap: ClientBootstrap,
|
||||
executionHandler: ExecutionHandler,
|
||||
remoteAddress: Address,
|
||||
client: ActiveRemoteClient) extends ChannelPipelineFactory {
|
||||
|
||||
import client.netty.settings
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val timeout = new ReadTimeoutHandler(client.netty.timer, settings.ReadTimeout.length, settings.ReadTimeout.unit)
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(client.netty)
|
||||
val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.netty.timer, client)
|
||||
|
||||
new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient)
|
||||
}
|
||||
}
|
||||
|
||||
class PassiveRemoteClient(val currentChannel: Channel,
|
||||
netty: NettyRemoteTransport,
|
||||
remoteAddress: Address)
|
||||
extends RemoteClient(netty, remoteAddress) {
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn {
|
||||
netty.notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
}
|
||||
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
netty.notifyListeners(RemoteClientShutdown(netty, remoteAddress))
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1,412 +1,113 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.remote.netty
|
||||
|
||||
import akka.actor.{ ActorRef, simpleName }
|
||||
import akka.remote._
|
||||
import RemoteProtocol._
|
||||
import akka.util._
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.{ NioServerSocketChannelFactory, NioClientSocketChannelFactory }
|
||||
import org.jboss.netty.bootstrap.{ ServerBootstrap, ClientBootstrap }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
|
||||
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
|
||||
import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
|
||||
import scala.collection.mutable.HashMap
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic._
|
||||
import akka.AkkaException
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
import java.util.concurrent.Executors
|
||||
|
||||
import scala.collection.mutable.HashMap
|
||||
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder }
|
||||
import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor
|
||||
import org.jboss.netty.util.HashedWheelTimer
|
||||
|
||||
import akka.actor.{ Address, ActorSystemImpl, ActorRef }
|
||||
import akka.dispatch.MonitorableThreadFactory
|
||||
import akka.event.Logging
|
||||
import org.jboss.netty.channel._
|
||||
import akka.actor.ActorSystemImpl
|
||||
import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor }
|
||||
import java.util.concurrent._
|
||||
import locks.ReentrantReadWriteLock
|
||||
|
||||
class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* This is the abstract baseclass for netty remote clients, currently there's only an
|
||||
* ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that
|
||||
* reuses an already established connection.
|
||||
*/
|
||||
abstract class RemoteClient private[akka] (
|
||||
val remoteSupport: NettyRemoteSupport,
|
||||
val remoteAddress: RemoteNettyAddress) {
|
||||
|
||||
val log = Logging(remoteSupport.system, "RemoteClient")
|
||||
|
||||
val name = simpleName(this) + "@" + remoteAddress
|
||||
|
||||
private[remote] val runSwitch = new Switch()
|
||||
|
||||
private[remote] def isRunning = runSwitch.isOn
|
||||
|
||||
protected def currentChannel: Channel
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean
|
||||
|
||||
def shutdown(): Boolean
|
||||
|
||||
def isBoundTo(address: RemoteNettyAddress): Boolean = remoteAddress == address
|
||||
|
||||
/**
|
||||
* Converts the message to the wireprotocol and sends the message across the wire
|
||||
*/
|
||||
def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) {
|
||||
if (remoteSupport.remote.remoteSettings.LogSentMessages)
|
||||
log.debug("Sending message [{}] from [{}] to [{}]", message, senderOption, recipient)
|
||||
|
||||
send((message, senderOption, recipient))
|
||||
} else {
|
||||
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress)
|
||||
remoteSupport.notifyListeners(RemoteClientError(exception, remoteSupport, remoteAddress))
|
||||
throw exception
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends the message across the wire
|
||||
*/
|
||||
private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = {
|
||||
try {
|
||||
val channel = currentChannel
|
||||
val f = channel.write(request)
|
||||
f.addListener(
|
||||
new ChannelFutureListener {
|
||||
def operationComplete(future: ChannelFuture) {
|
||||
if (future.isCancelled || !future.isSuccess) {
|
||||
remoteSupport.notifyListeners(RemoteClientWriteFailed(request, future.getCause, remoteSupport, remoteAddress))
|
||||
}
|
||||
}
|
||||
})
|
||||
// Check if we should back off
|
||||
if (!channel.isWritable) {
|
||||
val backoff = remoteSupport.remote.remoteSettings.BackoffTimeout
|
||||
if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteClientError(e, remoteSupport, remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = name
|
||||
}
|
||||
|
||||
class PassiveRemoteClient(val currentChannel: Channel,
|
||||
remoteSupport: NettyRemoteSupport,
|
||||
remoteAddress: RemoteNettyAddress)
|
||||
extends RemoteClient(remoteSupport, remoteAddress) {
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn {
|
||||
remoteSupport.notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress))
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
}
|
||||
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
remoteSupport.notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress))
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node.
|
||||
*/
|
||||
class ActiveRemoteClient private[akka] (
|
||||
remoteSupport: NettyRemoteSupport,
|
||||
remoteAddress: RemoteNettyAddress,
|
||||
localAddress: RemoteSystemAddress[ParsedTransportAddress],
|
||||
val loader: Option[ClassLoader] = None)
|
||||
extends RemoteClient(remoteSupport, remoteAddress) {
|
||||
|
||||
if (remoteAddress.ip.isEmpty) throw new java.net.UnknownHostException(remoteAddress.host)
|
||||
|
||||
import remoteSupport.clientSettings._
|
||||
|
||||
//TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
|
||||
@volatile
|
||||
private var bootstrap: ClientBootstrap = _
|
||||
@volatile
|
||||
private var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var executionHandler: ExecutionHandler = _
|
||||
|
||||
@volatile
|
||||
private var reconnectionTimeWindowStart = 0L
|
||||
|
||||
def notifyListeners(msg: RemoteLifeCycleEvent): Unit = remoteSupport.notifyListeners(msg)
|
||||
|
||||
def currentChannel = connection.getChannel
|
||||
|
||||
/**
|
||||
* Connect to remote server.
|
||||
*/
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
|
||||
|
||||
def sendSecureCookie(connection: ChannelFuture) {
|
||||
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
|
||||
if (SecureCookie.nonEmpty) handshake.setCookie(SecureCookie.get)
|
||||
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(localAddress.system)
|
||||
.setHostname(localAddress.transport.host)
|
||||
.setPort(localAddress.transport.port)
|
||||
.build)
|
||||
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
|
||||
}
|
||||
|
||||
def attemptReconnect(): Boolean = {
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port))
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
runSwitch switchOn {
|
||||
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
|
||||
|
||||
executionHandler = new ExecutionHandler(remoteSupport.executor)
|
||||
|
||||
bootstrap = new ClientBootstrap(remoteSupport.clientChannelFactory)
|
||||
bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this))
|
||||
bootstrap.setOption("tcpNoDelay", true)
|
||||
bootstrap.setOption("keepAlive", true)
|
||||
bootstrap.setOption("connectTimeoutMillis", ConnectionTimeout.toMillis)
|
||||
|
||||
log.debug("Starting remote client connection to [{}]", remoteAddress)
|
||||
|
||||
connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port))
|
||||
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
sendSecureCookie(connection)
|
||||
notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress))
|
||||
true
|
||||
}
|
||||
} match {
|
||||
case true ⇒ true
|
||||
case false if reconnectIfAlreadyConnected ⇒
|
||||
connection.getChannel.close()
|
||||
openChannels.remove(connection.getChannel)
|
||||
|
||||
log.debug("Remote client reconnecting to [{}]", remoteAddress)
|
||||
attemptReconnect()
|
||||
|
||||
case false ⇒ false
|
||||
}
|
||||
}
|
||||
|
||||
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
|
||||
def shutdown() = runSwitch switchOff {
|
||||
log.debug("Shutting down remote client [{}]", name)
|
||||
|
||||
notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress))
|
||||
try {
|
||||
if ((connection ne null) && (connection.getChannel ne null))
|
||||
connection.getChannel.close()
|
||||
} finally {
|
||||
try {
|
||||
if (openChannels ne null) openChannels.close.awaitUninterruptibly()
|
||||
} finally {
|
||||
connection = null
|
||||
executionHandler = null
|
||||
}
|
||||
}
|
||||
|
||||
log.debug("[{}] has been shut down", name)
|
||||
}
|
||||
|
||||
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
|
||||
if (reconnectionTimeWindowStart == 0L) {
|
||||
reconnectionTimeWindowStart = System.currentTimeMillis
|
||||
true
|
||||
} else {
|
||||
val timeLeft = (ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
|
||||
if (timeLeft)
|
||||
log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
|
||||
|
||||
timeLeft
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
|
||||
}
|
||||
|
||||
class ActiveRemoteClientPipelineFactory(
|
||||
name: String,
|
||||
bootstrap: ClientBootstrap,
|
||||
executionHandler: ExecutionHandler,
|
||||
remoteAddress: RemoteNettyAddress,
|
||||
client: ActiveRemoteClient) extends ChannelPipelineFactory {
|
||||
|
||||
import client.remoteSupport.clientSettings._
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val timeout = new ReadTimeoutHandler(client.remoteSupport.timer, ReadTimeout.length, ReadTimeout.unit)
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(client.remoteSupport)
|
||||
val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.remoteSupport.timer, client)
|
||||
|
||||
new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient)
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteMessageEncoder(remoteSupport: NettyRemoteSupport) extends ProtobufEncoder {
|
||||
override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = {
|
||||
msg match {
|
||||
case (message: Any, sender: Option[_], recipient: ActorRef) ⇒
|
||||
super.encode(ctx, channel,
|
||||
remoteSupport.createMessageSendEnvelope(
|
||||
remoteSupport.createRemoteMessageProtocolBuilder(
|
||||
recipient,
|
||||
message,
|
||||
sender.asInstanceOf[Option[ActorRef]]).build))
|
||||
case _ ⇒ super.encode(ctx, channel, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class ActiveRemoteClientHandler(
|
||||
val name: String,
|
||||
val bootstrap: ClientBootstrap,
|
||||
val remoteAddress: RemoteNettyAddress,
|
||||
val timer: HashedWheelTimer,
|
||||
val client: ActiveRemoteClient)
|
||||
extends SimpleChannelUpstreamHandler {
|
||||
|
||||
def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() }
|
||||
}, 0, TimeUnit.MILLISECONDS)
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
|
||||
try {
|
||||
event.getMessage match {
|
||||
case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒
|
||||
val rcp = arp.getInstruction
|
||||
rcp.getCommandType match {
|
||||
case CommandType.SHUTDOWN ⇒ runOnceNow { client.remoteSupport.shutdownClientConnection(remoteAddress) }
|
||||
case _ ⇒ //Ignore others
|
||||
}
|
||||
|
||||
case arp: AkkaRemoteProtocol if arp.hasMessage ⇒
|
||||
client.remoteSupport.receiveMessage(new RemoteMessage(arp.getMessage, client.remoteSupport.system, client.loader))
|
||||
|
||||
case other ⇒
|
||||
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.remoteSupport, client.remoteAddress)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
|
||||
if (client.isWithinReconnectionTimeWindow) {
|
||||
timer.newTimeout(new TimerTask() {
|
||||
def run(timeout: Timeout) =
|
||||
if (client.isRunning) {
|
||||
client.openChannels.remove(event.getChannel)
|
||||
client.connect(reconnectIfAlreadyConnected = true)
|
||||
}
|
||||
}, client.remoteSupport.clientSettings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS)
|
||||
} else runOnceNow {
|
||||
client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
}
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
try {
|
||||
client.notifyListeners(RemoteClientConnected(client.remoteSupport, client.remoteAddress))
|
||||
client.resetReconnectionTimeWindow
|
||||
} catch {
|
||||
case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
client.notifyListeners(RemoteClientDisconnected(client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
val cause = event.getCause
|
||||
if (cause ne null) {
|
||||
client.notifyListeners(RemoteClientError(cause, client.remoteSupport, client.remoteAddress))
|
||||
cause match {
|
||||
case e: ReadTimeoutException ⇒
|
||||
runOnceNow {
|
||||
client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread
|
||||
}
|
||||
case e: Exception ⇒ event.getChannel.close()
|
||||
}
|
||||
|
||||
} else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.remoteSupport, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
import akka.remote.RemoteProtocol.AkkaRemoteProtocol
|
||||
import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef }
|
||||
|
||||
/**
|
||||
* Provides the implementation of the Netty remote support
|
||||
*/
|
||||
class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val address: RemoteSystemAddress[RemoteNettyAddress])
|
||||
extends RemoteSupport[RemoteNettyAddress](_system) with RemoteMarshallingOps {
|
||||
val log = Logging(system, "NettyRemoteSupport")
|
||||
class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: ActorSystemImpl, val provider: RemoteActorRefProvider)
|
||||
extends RemoteTransport with RemoteMarshallingOps {
|
||||
|
||||
val serverSettings = remote.remoteSettings.serverSettings
|
||||
val clientSettings = remote.remoteSettings.clientSettings
|
||||
val threadFactory = _system.threadFactory.copy(_system.threadFactory.name + "-remote")
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory)
|
||||
val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName)
|
||||
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory)
|
||||
|
||||
val executor = new OrderedMemoryAwareThreadPoolExecutor(
|
||||
serverSettings.ExecutionPoolSize,
|
||||
serverSettings.MaxChannelMemorySize,
|
||||
serverSettings.MaxTotalMemorySize,
|
||||
serverSettings.ExecutionPoolKeepAlive.length,
|
||||
serverSettings.ExecutionPoolKeepAlive.unit,
|
||||
threadFactory)
|
||||
settings.ExecutionPoolSize,
|
||||
settings.MaxChannelMemorySize,
|
||||
settings.MaxTotalMemorySize,
|
||||
settings.ExecutionPoolKeepalive.length,
|
||||
settings.ExecutionPoolKeepalive.unit,
|
||||
system.threadFactory)
|
||||
|
||||
val clientChannelFactory = new NioClientSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(threadFactory),
|
||||
Executors.newCachedThreadPool(threadFactory))
|
||||
Executors.newCachedThreadPool(system.threadFactory),
|
||||
Executors.newCachedThreadPool(system.threadFactory))
|
||||
|
||||
private val remoteClients = new HashMap[RemoteNettyAddress, RemoteClient]
|
||||
private val remoteClients = new HashMap[Address, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
||||
override protected def useUntrustedMode = serverSettings.UntrustedMode
|
||||
override protected def useUntrustedMode = remoteSettings.UntrustedMode
|
||||
|
||||
val server = try new NettyRemoteServer(this) catch {
|
||||
case ex ⇒ shutdown(); throw ex
|
||||
}
|
||||
|
||||
// the address is set in start() or from the RemoteServerHandler, whichever comes first
|
||||
private val _address = new AtomicReference[Address]
|
||||
private[akka] def setAddressFromChannel(ch: Channel) = {
|
||||
val addr = ch.getLocalAddress match {
|
||||
case sa: InetSocketAddress ⇒ sa
|
||||
case x ⇒ throw new RemoteTransportException("unknown local address type " + x.getClass, null)
|
||||
}
|
||||
_address.compareAndSet(null, Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(addr.getPort)))
|
||||
}
|
||||
|
||||
def address = _address.get
|
||||
|
||||
val log = Logging(system.eventStream, "NettyRemoteTransport(" + address + ")")
|
||||
|
||||
def start(): Unit = {
|
||||
server.start()
|
||||
setAddressFromChannel(server.channel)
|
||||
}
|
||||
|
||||
def shutdown(): Unit = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients foreach {
|
||||
case (_, client) ⇒ try client.shutdown() catch {
|
||||
case e ⇒ log.error(e, "failure while shutting down [{}]", client)
|
||||
}
|
||||
}
|
||||
remoteClients.clear()
|
||||
} finally {
|
||||
clientsLock.writeLock().unlock()
|
||||
try {
|
||||
if (server != null) server.shutdown()
|
||||
} finally {
|
||||
try {
|
||||
timer.stop()
|
||||
} finally {
|
||||
try {
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
} finally {
|
||||
executor.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected[akka] def send(
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
recipient: RemoteActorRef,
|
||||
loader: Option[ClassLoader]): Unit = {
|
||||
recipient: RemoteActorRef): Unit = {
|
||||
|
||||
val recipientAddress = recipient.path.address match {
|
||||
case RemoteSystemAddress(sys, transport) ⇒
|
||||
transport match {
|
||||
case x: RemoteNettyAddress ⇒ x
|
||||
case _ ⇒ throw new IllegalArgumentException("invoking NettyRemoteSupport.send with foreign target address " + transport)
|
||||
}
|
||||
}
|
||||
val recipientAddress = recipient.path.address
|
||||
|
||||
clientsLock.readLock.lock
|
||||
try {
|
||||
|
|
@ -421,7 +122,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
//Recheck for addition, race between upgrades
|
||||
case Some(client) ⇒ client //If already populated by other writer
|
||||
case None ⇒ //Populate map
|
||||
val client = new ActiveRemoteClient(this, recipientAddress, remote.remoteAddress, loader)
|
||||
val client = new ActiveRemoteClient(this, recipientAddress, address)
|
||||
client.connect()
|
||||
remoteClients += recipientAddress -> client
|
||||
client
|
||||
|
|
@ -439,7 +140,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def bindClient(remoteAddress: RemoteNettyAddress, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = {
|
||||
def bindClient(remoteAddress: Address, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
if (putIfAbsent && remoteClients.contains(remoteAddress)) false
|
||||
|
|
@ -453,7 +154,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def unbindClient(remoteAddress: RemoteNettyAddress): Unit = {
|
||||
def unbindClient(remoteAddress: Address): Unit = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients.foreach { case (k, v) ⇒ if (v.isBoundTo(remoteAddress)) { v.shutdown(); remoteClients.remove(k) } }
|
||||
|
|
@ -462,7 +163,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def shutdownClientConnection(remoteAddress: RemoteNettyAddress): Boolean = {
|
||||
def shutdownClientConnection(remoteAddress: Address): Boolean = {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients.remove(remoteAddress) match {
|
||||
|
|
@ -474,7 +175,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
def restartClientConnection(remoteAddress: RemoteNettyAddress): Boolean = {
|
||||
def restartClientConnection(remoteAddress: Address): Boolean = {
|
||||
clientsLock.readLock().lock()
|
||||
try {
|
||||
remoteClients.get(remoteAddress) match {
|
||||
|
|
@ -486,229 +187,24 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Server section
|
||||
*/
|
||||
@volatile
|
||||
private var currentServer: NettyRemoteServer = _
|
||||
}
|
||||
|
||||
def name = currentServer match {
|
||||
case null ⇒ remote.remoteAddress.toString
|
||||
case server ⇒ server.name
|
||||
}
|
||||
|
||||
private val _isRunning = new Switch(false)
|
||||
|
||||
def isRunning = _isRunning.isOn
|
||||
|
||||
def start(loader: Option[ClassLoader] = None): Unit =
|
||||
_isRunning switchOn { currentServer = new NettyRemoteServer(this, loader, address) }
|
||||
|
||||
/**
|
||||
* Common section
|
||||
*/
|
||||
|
||||
def shutdown(): Unit = _isRunning switchOff {
|
||||
clientsLock.writeLock().lock()
|
||||
try {
|
||||
remoteClients foreach { case (_, client) ⇒ client.shutdown() }
|
||||
remoteClients.clear()
|
||||
} finally {
|
||||
clientsLock.writeLock().unlock()
|
||||
try {
|
||||
val s = currentServer
|
||||
currentServer = null
|
||||
s.shutdown()
|
||||
} finally {
|
||||
try {
|
||||
timer.stop()
|
||||
} finally {
|
||||
try {
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
} finally {
|
||||
executor.shutdown()
|
||||
}
|
||||
}
|
||||
}
|
||||
class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends ProtobufEncoder {
|
||||
override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = {
|
||||
msg match {
|
||||
case (message: Any, sender: Option[_], recipient: ActorRef) ⇒
|
||||
super.encode(ctx, channel,
|
||||
remoteSupport.createMessageSendEnvelope(
|
||||
remoteSupport.createRemoteMessageProtocolBuilder(
|
||||
recipient,
|
||||
message,
|
||||
sender.asInstanceOf[Option[ActorRef]]).build))
|
||||
case _ ⇒ super.encode(ctx, channel, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class NettyRemoteServer(
|
||||
val remoteSupport: NettyRemoteSupport,
|
||||
val loader: Option[ClassLoader],
|
||||
val address: RemoteSystemAddress[RemoteNettyAddress]) {
|
||||
val log = Logging(remoteSupport.system, "NettyRemoteServer")
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
if (address.transport.ip.isEmpty) throw new java.net.UnknownHostException(address.transport.host)
|
||||
|
||||
val name = "NettyRemoteServer@" + address
|
||||
|
||||
private val factory = new NioServerSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(remoteSupport.threadFactory),
|
||||
Executors.newCachedThreadPool(remoteSupport.threadFactory))
|
||||
|
||||
private val executionHandler = new ExecutionHandler(remoteSupport.executor)
|
||||
|
||||
// group of open channels, used for clean-up
|
||||
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
|
||||
|
||||
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executionHandler, loader, remoteSupport)
|
||||
private val bootstrap: ServerBootstrap = {
|
||||
val b = new ServerBootstrap(factory)
|
||||
b.setPipelineFactory(pipelineFactory)
|
||||
b.setOption("backlog", Backlog)
|
||||
b.setOption("child.tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
b.setOption("child.reuseAddress", true)
|
||||
b
|
||||
}
|
||||
|
||||
openChannels.add(bootstrap.bind(new InetSocketAddress(address.transport.ip.get, address.transport.port)))
|
||||
|
||||
def shutdown() {
|
||||
try {
|
||||
val shutdownSignal = {
|
||||
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
|
||||
b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(address.system)
|
||||
.setHostname(address.transport.host)
|
||||
.setPort(address.transport.port)
|
||||
.build)
|
||||
if (SecureCookie.nonEmpty)
|
||||
b.setCookie(SecureCookie.get)
|
||||
b.build
|
||||
}
|
||||
openChannels.write(remoteSupport.createControlEnvelope(shutdownSignal)).awaitUninterruptibly
|
||||
openChannels.disconnect
|
||||
openChannels.close.awaitUninterruptibly
|
||||
bootstrap.releaseExternalResources()
|
||||
remoteSupport.notifyListeners(RemoteServerShutdown(remoteSupport))
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteServerPipelineFactory(
|
||||
val name: String,
|
||||
val openChannels: ChannelGroup,
|
||||
val executionHandler: ExecutionHandler,
|
||||
val loader: Option[ClassLoader],
|
||||
val remoteSupport: NettyRemoteSupport) extends ChannelPipelineFactory {
|
||||
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(remoteSupport)
|
||||
|
||||
val authenticator = if (RequireCookie) new RemoteServerAuthenticationHandler(SecureCookie) :: Nil else Nil
|
||||
val remoteServer = new RemoteServerHandler(name, openChannels, loader, remoteSupport)
|
||||
val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil
|
||||
new StaticChannelPipeline(stages: _*)
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler {
|
||||
val authenticated = new AnyRef
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match {
|
||||
case None ⇒ ctx.sendUpstream(event)
|
||||
case Some(cookie) ⇒
|
||||
ctx.getAttachment match {
|
||||
case `authenticated` ⇒ ctx.sendUpstream(event)
|
||||
case null ⇒ event.getMessage match {
|
||||
case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒
|
||||
val instruction = remoteProtocol.getInstruction
|
||||
instruction.getCookie match {
|
||||
case `cookie` ⇒
|
||||
ctx.setAttachment(authenticated)
|
||||
ctx.sendUpstream(event)
|
||||
case _ ⇒
|
||||
throw new SecurityException(
|
||||
"The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie")
|
||||
}
|
||||
case _ ⇒
|
||||
throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerHandler(
|
||||
val name: String,
|
||||
val openChannels: ChannelGroup,
|
||||
val applicationLoader: Option[ClassLoader],
|
||||
val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
val log = Logging(remoteSupport.system, "RemoteServerHandler")
|
||||
|
||||
import remoteSupport.serverSettings._
|
||||
|
||||
/**
|
||||
* ChannelOpen overridden to store open channels for a clean postStop of a node.
|
||||
* If a channel is closed before, it is automatically removed from the open channels group.
|
||||
*/
|
||||
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
remoteSupport.notifyListeners(RemoteServerClientConnected(remoteSupport, clientAddress))
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
remoteSupport.notifyListeners(RemoteServerClientDisconnected(remoteSupport, clientAddress))
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match {
|
||||
case s @ Some(address) ⇒
|
||||
if (UsePassiveConnections)
|
||||
remoteSupport.unbindClient(address)
|
||||
remoteSupport.notifyListeners(RemoteServerClientClosed(remoteSupport, s))
|
||||
case None ⇒
|
||||
remoteSupport.notifyListeners(RemoteServerClientClosed[RemoteNettyAddress](remoteSupport, None))
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
|
||||
event.getMessage match {
|
||||
case remote: AkkaRemoteProtocol if remote.hasMessage ⇒
|
||||
remoteSupport.receiveMessage(new RemoteMessage(remote.getMessage, remoteSupport.system, applicationLoader))
|
||||
|
||||
case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒
|
||||
val instruction = remote.getInstruction
|
||||
instruction.getCommandType match {
|
||||
case CommandType.CONNECT if UsePassiveConnections ⇒
|
||||
val origin = instruction.getOrigin
|
||||
val inbound = RemoteNettyAddress(origin.getHostname, origin.getPort)
|
||||
val client = new PassiveRemoteClient(event.getChannel, remoteSupport, inbound)
|
||||
remoteSupport.bindClient(inbound, client)
|
||||
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
||||
case _ ⇒ //Unknown command
|
||||
}
|
||||
case _ ⇒ //ignore
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
remoteSupport.notifyListeners(RemoteServerError(event.getCause, remoteSupport))
|
||||
event.getChannel.close()
|
||||
}
|
||||
|
||||
private def getClientAddress(c: Channel): Option[RemoteNettyAddress] =
|
||||
c.getRemoteAddress match {
|
||||
case inet: InetSocketAddress ⇒ Some(RemoteNettyAddress(inet.getHostName, Some(inet.getAddress), inet.getPort))
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
|
||||
|
||||
class DefaultDisposableChannelGroup(name: String) extends DefaultChannelGroup(name) {
|
||||
protected val guard = new ReentrantReadWriteLock
|
||||
|
|
|
|||
210
akka-remote/src/main/scala/akka/remote/netty/Server.scala
Normal file
210
akka-remote/src/main/scala/akka/remote/netty/Server.scala
Normal file
|
|
@ -0,0 +1,210 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.Executors
|
||||
import scala.Option.option2Iterable
|
||||
import org.jboss.netty.bootstrap.ServerBootstrap
|
||||
import org.jboss.netty.channel.ChannelHandler.Sharable
|
||||
import org.jboss.netty.channel.group.ChannelGroup
|
||||
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelHandler, Channel }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.execution.ExecutionHandler
|
||||
import akka.event.Logging
|
||||
import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol }
|
||||
import akka.remote.{ RemoteServerStarted, RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage }
|
||||
import akka.actor.Address
|
||||
import java.net.InetAddress
|
||||
import akka.actor.ActorSystemImpl
|
||||
import org.jboss.netty.channel.ChannelLocal
|
||||
import org.jboss.netty.channel.ChannelEvent
|
||||
|
||||
class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
||||
|
||||
import netty.settings
|
||||
|
||||
val ip = InetAddress.getByName(settings.Hostname)
|
||||
|
||||
private val factory = new NioServerSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(netty.system.threadFactory),
|
||||
Executors.newCachedThreadPool(netty.system.threadFactory))
|
||||
|
||||
private val executionHandler = new ExecutionHandler(netty.executor)
|
||||
|
||||
// group of open channels, used for clean-up
|
||||
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
|
||||
|
||||
private val bootstrap = {
|
||||
val b = new ServerBootstrap(factory)
|
||||
b.setPipelineFactory(new RemoteServerPipelineFactory(openChannels, executionHandler, netty))
|
||||
b.setOption("backlog", settings.Backlog)
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
b.setOption("reuseAddress", true)
|
||||
b
|
||||
}
|
||||
|
||||
@volatile
|
||||
private[akka] var channel: Channel = _
|
||||
|
||||
def start(): Unit = {
|
||||
channel = bootstrap.bind(new InetSocketAddress(ip, settings.DesiredPortFromConfig))
|
||||
openChannels.add(channel)
|
||||
netty.notifyListeners(RemoteServerStarted(netty))
|
||||
}
|
||||
|
||||
def shutdown() {
|
||||
try {
|
||||
val shutdownSignal = {
|
||||
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
|
||||
b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
|
||||
.setSystem(settings.systemName)
|
||||
.setHostname(settings.Hostname)
|
||||
.setPort(settings.DesiredPortFromConfig)
|
||||
.build)
|
||||
if (settings.SecureCookie.nonEmpty)
|
||||
b.setCookie(settings.SecureCookie.get)
|
||||
b.build
|
||||
}
|
||||
openChannels.write(netty.createControlEnvelope(shutdownSignal)).awaitUninterruptibly
|
||||
openChannels.disconnect
|
||||
openChannels.close.awaitUninterruptibly
|
||||
bootstrap.releaseExternalResources()
|
||||
netty.notifyListeners(RemoteServerShutdown(netty))
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RemoteServerPipelineFactory(
|
||||
val openChannels: ChannelGroup,
|
||||
val executionHandler: ExecutionHandler,
|
||||
val netty: NettyRemoteTransport) extends ChannelPipelineFactory {
|
||||
|
||||
import netty.settings
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val lenDec = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4)
|
||||
val lenPrep = new LengthFieldPrepender(4)
|
||||
val messageDec = new RemoteMessageDecoder
|
||||
val messageEnc = new RemoteMessageEncoder(netty)
|
||||
|
||||
val authenticator = if (settings.RequireCookie) new RemoteServerAuthenticationHandler(settings.SecureCookie) :: Nil else Nil
|
||||
val remoteServer = new RemoteServerHandler(openChannels, netty)
|
||||
val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil
|
||||
new StaticChannelPipeline(stages: _*)
|
||||
}
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler {
|
||||
val authenticated = new AnyRef
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match {
|
||||
case None ⇒ ctx.sendUpstream(event)
|
||||
case Some(cookie) ⇒
|
||||
ctx.getAttachment match {
|
||||
case `authenticated` ⇒ ctx.sendUpstream(event)
|
||||
case null ⇒ event.getMessage match {
|
||||
case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒
|
||||
val instruction = remoteProtocol.getInstruction
|
||||
instruction.getCookie match {
|
||||
case `cookie` ⇒
|
||||
ctx.setAttachment(authenticated)
|
||||
ctx.sendUpstream(event)
|
||||
case _ ⇒
|
||||
throw new SecurityException(
|
||||
"The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie")
|
||||
}
|
||||
case _ ⇒
|
||||
throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ChannelLocalSystem extends ChannelLocal[ActorSystemImpl] {
|
||||
override def initialValue(ch: Channel): ActorSystemImpl = null
|
||||
}
|
||||
|
||||
@ChannelHandler.Sharable
|
||||
class RemoteServerHandler(
|
||||
val openChannels: ChannelGroup,
|
||||
val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
import netty.settings
|
||||
|
||||
private var addressToSet = true
|
||||
|
||||
// TODO look into moving that into onBind or similar, but verify that that is guaranteed to be the first to be called
|
||||
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
|
||||
if (addressToSet) {
|
||||
netty.setAddressFromChannel(event.getChannel)
|
||||
addressToSet = false
|
||||
}
|
||||
super.handleUpstream(ctx, event)
|
||||
}
|
||||
|
||||
/**
|
||||
* ChannelOpen overridden to store open channels for a clean postStop of a node.
|
||||
* If a channel is closed before, it is automatically removed from the open channels group.
|
||||
*/
|
||||
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
|
||||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
netty.notifyListeners(RemoteServerClientConnected(netty, clientAddress))
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx.getChannel)
|
||||
netty.notifyListeners(RemoteServerClientDisconnected(netty, clientAddress))
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match {
|
||||
case s @ Some(address) ⇒
|
||||
if (settings.UsePassiveConnections)
|
||||
netty.unbindClient(address)
|
||||
netty.notifyListeners(RemoteServerClientClosed(netty, s))
|
||||
case None ⇒
|
||||
netty.notifyListeners(RemoteServerClientClosed(netty, None))
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
|
||||
event.getMessage match {
|
||||
case remote: AkkaRemoteProtocol if remote.hasMessage ⇒
|
||||
netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system))
|
||||
|
||||
case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒
|
||||
val instruction = remote.getInstruction
|
||||
instruction.getCommandType match {
|
||||
case CommandType.CONNECT if settings.UsePassiveConnections ⇒
|
||||
val origin = instruction.getOrigin
|
||||
val inbound = Address("akka", origin.getSystem, Some(origin.getHostname), Some(origin.getPort))
|
||||
val client = new PassiveRemoteClient(event.getChannel, netty, inbound)
|
||||
netty.bindClient(inbound, client)
|
||||
case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed
|
||||
case _ ⇒ //Unknown command
|
||||
}
|
||||
case _ ⇒ //ignore
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
netty.notifyListeners(RemoteServerError(event.getCause, netty))
|
||||
event.getChannel.close()
|
||||
}
|
||||
|
||||
private def getClientAddress(c: Channel): Option[Address] =
|
||||
c.getRemoteAddress match {
|
||||
case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", Some(inet.getAddress.toString), Some(inet.getPort)))
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
63
akka-remote/src/main/scala/akka/remote/netty/Settings.scala
Normal file
63
akka-remote/src/main/scala/akka/remote/netty/Settings.scala
Normal file
|
|
@ -0,0 +1,63 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.netty
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent.TimeUnit._
|
||||
import java.net.InetAddress
|
||||
import akka.config.ConfigurationException
|
||||
|
||||
class NettySettings(config: Config, val systemName: String) {
|
||||
|
||||
import config._
|
||||
|
||||
val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS)
|
||||
|
||||
val SecureCookie: Option[String] = getString("secure-cookie") match {
|
||||
case "" ⇒ None
|
||||
case cookie ⇒ Some(cookie)
|
||||
}
|
||||
val RequireCookie = {
|
||||
val requireCookie = getBoolean("require-cookie")
|
||||
if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException(
|
||||
"Configuration option 'akka.remote.netty.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.netty.secure-cookie'.")
|
||||
requireCookie
|
||||
}
|
||||
|
||||
val UsePassiveConnections = getBoolean("use-passive-connections")
|
||||
|
||||
val ReconnectionTimeWindow = Duration(getMilliseconds("reconnection-time-window"), MILLISECONDS)
|
||||
val ReadTimeout = Duration(getMilliseconds("read-timeout"), MILLISECONDS)
|
||||
val ReconnectDelay = Duration(getMilliseconds("reconnect-delay"), MILLISECONDS)
|
||||
val MessageFrameSize = getBytes("message-frame-size").toInt
|
||||
|
||||
val Hostname = getString("hostname") match {
|
||||
case "" ⇒ InetAddress.getLocalHost.getHostAddress
|
||||
case value ⇒ value
|
||||
}
|
||||
val DesiredPortFromConfig = getInt("port")
|
||||
|
||||
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
|
||||
|
||||
val Backlog = getInt("backlog")
|
||||
|
||||
val ExecutionPoolKeepalive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS)
|
||||
|
||||
val ExecutionPoolSize = getInt("execution-pool-size") match {
|
||||
case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val MaxChannelMemorySize = getBytes("max-channel-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-channel-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val MaxTotalMemorySize = getBytes("max-total-memory-size") match {
|
||||
case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-total-memory-size is less than 0 bytes")
|
||||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,8 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka
|
||||
|
||||
package object remote {
|
||||
type TransportsMap = Map[String, (String, Int) ⇒ Either[String, RemoteTransportAddress]]
|
||||
}
|
||||
|
|
@ -12,7 +12,7 @@ import akka.actor.InternalActorRef
|
|||
import akka.actor.Props
|
||||
import akka.config.ConfigurationException
|
||||
import akka.remote.RemoteScope
|
||||
import akka.remote.RemoteAddressExtractor
|
||||
import akka.actor.AddressExtractor
|
||||
|
||||
/**
|
||||
* [[akka.routing.RouterConfig]] implementation for remote deployment on defined
|
||||
|
|
@ -46,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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,8 +21,8 @@ trait AbstractRemoteActorMultiJvmSpec {
|
|||
case (idx, host) =>
|
||||
ConfigFactory.parseString("""
|
||||
akka {
|
||||
remote.server.hostname="%s"
|
||||
remote.server.port = "%d"
|
||||
remote.netty.hostname="%s"
|
||||
remote.netty.port = "%d"
|
||||
}""".format(host, 9990+idx, idx)) withFallback commonConfig
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -32,14 +32,4 @@ abstract class AkkaRemoteSpec(config: Config)
|
|||
extends AkkaSpec(config.withFallback(AkkaRemoteSpec.testConf))
|
||||
with MultiJvmSync {
|
||||
|
||||
/**
|
||||
* Helper function for accessing the underlying remoting.
|
||||
*/
|
||||
def remote: Remote = {
|
||||
system.asInstanceOf[ActorSystemImpl].provider match {
|
||||
case r: RemoteActorRefProvider ⇒ r.remote
|
||||
case _ ⇒ throw new Exception("Remoting is not enabled")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.testkit._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
|
|
@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! self.path.address.hostPort
|
||||
case "identify" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -53,7 +53,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(nodeConfigs(1)
|
|||
val actor = system.actorOf(Props[SomeActor], "service-hello")
|
||||
actor.isInstanceOf[RemoteActorRef] must be(true)
|
||||
|
||||
Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0))
|
||||
Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0))
|
||||
|
||||
barrier("done")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.testkit._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
|
|
@ -10,7 +10,7 @@ object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec {
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "identify" ⇒ sender ! self.path.address.hostPort
|
||||
case "identify" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -53,7 +53,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(NewRemoteActorMultiJvmS
|
|||
barrier("start")
|
||||
|
||||
val actor = system.actorOf(Props[SomeActor], "service-hello")
|
||||
Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0))
|
||||
Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0))
|
||||
|
||||
barrier("done")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.dispatch.Await
|
||||
|
|
@ -10,7 +10,7 @@ object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp
|
|||
override def NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -89,7 +89,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RandomRoutedRe
|
|||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).toString
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort
|
||||
replies = replies + (nodeName -> (replies(nodeName) + 1))
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.dispatch.Await
|
||||
|
|
@ -11,7 +11,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJ
|
|||
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -90,7 +90,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RoundRobin
|
|||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).toString
|
||||
val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort
|
||||
|
||||
replies = replies + (nodeName -> (replies(nodeName) + 1))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +1,6 @@
|
|||
package akka.remote
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.routing._
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
|
|
@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMul
|
|||
override def NrOfNodes = 4
|
||||
class SomeActor extends Actor with Serializable {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.address.hostPort
|
||||
case "hit" ⇒ sender ! self
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
}
|
||||
|
|
@ -89,7 +89,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter
|
|||
}
|
||||
|
||||
val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) {
|
||||
case name: String ⇒ (name, 1)
|
||||
case ref: ActorRef ⇒ (ref.path.address.hostPort, 1)
|
||||
}).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) {
|
||||
case (m, (n, c)) ⇒ m + (n -> (m(n) + c))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -2,11 +2,12 @@ package akka.remote
|
|||
|
||||
import java.net.InetSocketAddress
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Address
|
||||
|
||||
class AccrualFailureDetectorSpec extends AkkaSpec {
|
||||
|
||||
"An AccrualFailureDetector" must {
|
||||
val conn = RemoteNettyAddress("localhost", 2552)
|
||||
val conn = Address("akka", "", Some("localhost"), Some(2552))
|
||||
|
||||
"mark node as available after a series of successful heartbeats" in {
|
||||
val fd = new AccrualFailureDetector()
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.remote
|
|||
|
||||
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
||||
|
||||
import akka.remote.netty.NettyRemoteSupport
|
||||
import akka.remote.netty.NettyRemoteTransport
|
||||
import akka.actor.Actor
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.DefaultTimeout
|
||||
|
|
|
|||
|
|
@ -34,7 +34,7 @@ object RemoteCommunicationSpec {
|
|||
class RemoteCommunicationSpec extends AkkaSpec("""
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
port = 12345
|
||||
}
|
||||
|
|
@ -48,7 +48,7 @@ akka {
|
|||
|
||||
import RemoteCommunicationSpec._
|
||||
|
||||
val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config)
|
||||
val conf = ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config)
|
||||
val other = ActorSystem("remote_sys", conf)
|
||||
|
||||
val remote = other.actorOf(Props(new Actor {
|
||||
|
|
|
|||
|
|
@ -1,8 +1,13 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.util.duration._
|
||||
import akka.util.Duration
|
||||
import akka.remote.netty.NettyRemoteTransport
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RemoteConfigSpec extends AkkaSpec(
|
||||
|
|
@ -14,51 +19,49 @@ class RemoteConfigSpec extends AkkaSpec(
|
|||
}
|
||||
""") {
|
||||
|
||||
"RemoteExtension" must {
|
||||
"be able to parse remote and cluster config elements" in {
|
||||
"Remoting" must {
|
||||
|
||||
"be able to parse generic remote config elements" in {
|
||||
val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings
|
||||
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)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,14 +16,14 @@ akka {
|
|||
/watchers.remote = "akka://other@127.0.0.1:2666"
|
||||
}
|
||||
}
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = "127.0.0.1"
|
||||
port = 2665
|
||||
port = 0
|
||||
}
|
||||
}
|
||||
""")) with ImplicitSender with DefaultTimeout with DeathWatchSpec {
|
||||
|
||||
val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.server.port=2666").withFallback(system.settings.config))
|
||||
val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.port=2666").withFallback(system.settings.config))
|
||||
|
||||
override def atTermination() {
|
||||
other.shutdown()
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) {
|
|||
service,
|
||||
deployment.get.config,
|
||||
RoundRobinRouter(3),
|
||||
RemoteScope(UnparsedSystemAddress(Some("sys"), UnparsedTransportAddress("akka", "wallace", 2552))))))
|
||||
RemoteScope(Address("akka", "sys", Some("wallace"), Some(2552))))))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import com.typesafe.config._
|
|||
object RemoteRouterSpec {
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self.path
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -20,15 +20,15 @@ object RemoteRouterSpec {
|
|||
class RemoteRouterSpec extends AkkaSpec("""
|
||||
akka {
|
||||
actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
remote.server {
|
||||
remote.netty {
|
||||
hostname = localhost
|
||||
port = 12345
|
||||
port = 0
|
||||
}
|
||||
actor.deployment {
|
||||
/blub {
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka://remote_sys@localhost:12346"]
|
||||
target.nodes = ["akka://remote_sys@localhost:12347"]
|
||||
}
|
||||
/elastic-blub {
|
||||
router = round-robin
|
||||
|
|
@ -36,7 +36,7 @@ akka {
|
|||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
}
|
||||
target.nodes = ["akka://remote_sys@localhost:12346"]
|
||||
target.nodes = ["akka://remote_sys@localhost:12347"]
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -44,7 +44,7 @@ akka {
|
|||
|
||||
import RemoteRouterSpec._
|
||||
|
||||
val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config)
|
||||
val conf = ConfigFactory.parseString("akka.remote.netty.port=12347").withFallback(system.settings.config)
|
||||
val other = ActorSystem("remote_sys", conf)
|
||||
|
||||
override def atTermination() {
|
||||
|
|
@ -56,26 +56,26 @@ akka {
|
|||
"deploy its children on remote host driven by configuration" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
"deploy its children on remote host driven by programatic definition" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2),
|
||||
Seq("akka://remote_sys@localhost:12346"))), "blub2")
|
||||
Seq("akka://remote_sys@localhost:12347"))), "blub2")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
"deploy dynamic resizable number of children on remote host driven by configuration" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub")
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c1"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
router ! ""
|
||||
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c2"
|
||||
expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347"
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue