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:
Patrik Nordwall 2011-11-22 13:04:10 +01:00
parent c56341b3a6
commit 179399296e
85 changed files with 1233 additions and 643 deletions

View file

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