Modularize configuration. See #1361
* Split config reference to one for each module/extension. * Adjusted signature of registerExtension to avoid race of extension init * Moved Duration.dilated to testkit * TestKitExtension * RemoteExtension * SerializationExtension * Durable mailboxes extensions * Fixed broken serialization bindings and added test * Updated configuration documentation * System properties akka.remote.hostname akka.remote.port replaced with akka.remote.server.hostname and akka.remote.server.port * Adjustments of ActorSystem initialization. Still don't like the two-phase constructor/init flow. Very fragile for changes. Review fixes. SerializationExtension
This commit is contained in:
parent
c56341b3a6
commit
179399296e
85 changed files with 1233 additions and 643 deletions
|
|
@ -21,6 +21,8 @@ import java.util.concurrent.atomic.AtomicBoolean
|
|||
import akka.event.EventStream
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.dispatch.Promise
|
||||
import java.net.InetAddress
|
||||
import akka.serialization.SerializationExtension
|
||||
|
||||
/**
|
||||
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
|
||||
|
|
@ -29,33 +31,43 @@ import akka.dispatch.Promise
|
|||
*/
|
||||
class RemoteActorRefProvider(
|
||||
val settings: ActorSystem.Settings,
|
||||
val rootPath: ActorPath,
|
||||
val eventStream: EventStream,
|
||||
val dispatcher: MessageDispatcher,
|
||||
val scheduler: Scheduler) extends ActorRefProvider {
|
||||
|
||||
val log = Logging(eventStream, "RemoteActorRefProvider")
|
||||
|
||||
val local = new LocalActorRefProvider(settings, rootPath, eventStream, dispatcher, scheduler)
|
||||
|
||||
def deathWatch = local.deathWatch
|
||||
def guardian = local.guardian
|
||||
def systemGuardian = local.systemGuardian
|
||||
def nodename = local.nodename
|
||||
def clustername = local.clustername
|
||||
def tempName = local.tempName
|
||||
|
||||
@volatile
|
||||
var remote: Remote = _
|
||||
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
/*
|
||||
* The problem is that ActorRefs need a reference to the ActorSystem to
|
||||
* provide their service. Hence they cannot be created while the
|
||||
* constructors of ActorSystem and ActorRefProvider are still running.
|
||||
* The solution is to split out that last part into an init() method,
|
||||
* but it also requires these references to be @volatile and lazy.
|
||||
*/
|
||||
@volatile
|
||||
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
|
||||
private var system: ActorSystemImpl = _
|
||||
private lazy val remoteExtension = RemoteExtension(system)
|
||||
private lazy val serializationExtension = SerializationExtension(system)
|
||||
lazy val rootPath: ActorPath = {
|
||||
val remoteAddress = RemoteAddress(remoteExtension.settings.serverSettings.Hostname, remoteExtension.settings.serverSettings.Port)
|
||||
new RootActorPath(remoteAddress)
|
||||
}
|
||||
private lazy val local = new LocalActorRefProvider(settings, eventStream, scheduler, rootPath,
|
||||
remoteExtension.settings.NodeName, remoteExtension.settings.ClusterName)
|
||||
private[akka] lazy val remote = new Remote(system, nodename)
|
||||
private lazy val remoteDaemonConnectionManager = new RemoteConnectionManager(system, remote)
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
local.init(system)
|
||||
remote = new Remote(system, nodename)
|
||||
remoteDaemonConnectionManager = new RemoteConnectionManager(system, remote)
|
||||
def init(_system: ActorSystemImpl) {
|
||||
system = _system
|
||||
local.init(_system)
|
||||
terminationFuture.onComplete(_ ⇒ remote.server.shutdown())
|
||||
}
|
||||
|
||||
|
|
@ -64,7 +76,7 @@ class RemoteActorRefProvider(
|
|||
|
||||
private[akka] def deployer: Deployer = local.deployer
|
||||
|
||||
def defaultDispatcher = dispatcher
|
||||
def dispatcher = local.dispatcher
|
||||
def defaultTimeout = settings.ActorTimeout
|
||||
|
||||
private[akka] def actorOf(system: ActorSystemImpl, props: Props, supervisor: ActorRef, name: String, systemService: Boolean): ActorRef =
|
||||
|
|
@ -74,7 +86,7 @@ class RemoteActorRefProvider(
|
|||
if (systemService) local.actorOf(system, props, supervisor, path, systemService)
|
||||
else {
|
||||
val name = path.name
|
||||
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
|
||||
val newFuture = Promise[ActorRef](5000)(dispatcher) // FIXME is this proper timeout?
|
||||
|
||||
actors.putIfAbsent(path.toString, newFuture) match { // we won the race -- create the actor and resolve the future
|
||||
case null ⇒
|
||||
|
|
@ -90,7 +102,7 @@ class RemoteActorRefProvider(
|
|||
// case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass)
|
||||
// }
|
||||
|
||||
def isReplicaNode: Boolean = remoteAddresses exists { _ == system.address }
|
||||
def isReplicaNode: Boolean = remoteAddresses exists { _ == rootPath.remoteAddress }
|
||||
|
||||
//system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||
|
||||
|
|
@ -123,7 +135,7 @@ class RemoteActorRefProvider(
|
|||
if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout)
|
||||
() ⇒ new ScatterGatherFirstCompletedRouter()(dispatcher, defaultTimeout)
|
||||
|
||||
case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
|
||||
case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
|
||||
|
|
@ -207,9 +219,9 @@ class RemoteActorRefProvider(
|
|||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress)
|
||||
|
||||
val actorFactoryBytes =
|
||||
system.serialization.serialize(actorFactory) match {
|
||||
serializationExtension.serialization.serialize(actorFactory) match {
|
||||
case Left(error) ⇒ throw error
|
||||
case Right(bytes) ⇒ if (remote.shouldCompressData) LZF.compress(bytes) else bytes
|
||||
case Right(bytes) ⇒ if (remoteExtension.settings.ShouldCompressData) LZF.compress(bytes) else bytes
|
||||
}
|
||||
|
||||
val command = RemoteSystemDaemonMessageProtocol.newBuilder
|
||||
|
|
@ -229,7 +241,7 @@ class RemoteActorRefProvider(
|
|||
private def sendCommandToRemoteNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, withACK: Boolean) {
|
||||
if (withACK) {
|
||||
try {
|
||||
val f = connection ? (command, remote.remoteSystemDaemonAckTimeout)
|
||||
val f = connection ? (command, remoteExtension.settings.RemoteSystemDaemonAckTimeout)
|
||||
(try f.await.value catch { case _: FutureTimeoutException ⇒ None }) match {
|
||||
case Some(Right(receiver)) ⇒
|
||||
log.debug("Remote system command sent to [{}] successfully received", receiver)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue