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

@ -25,6 +25,7 @@ import akka.actor.ActorSystem
import akka.event.Logging
import locks.ReentrantReadWriteLock
import org.jboss.netty.channel._
import akka.actor.ActorSystemImpl
class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
@ -140,6 +141,8 @@ class ActiveRemoteClient private[akka] (
def currentChannel = connection.getChannel
private val senderRemoteAddress = remoteSupport.system.asInstanceOf[ActorSystemImpl].provider.rootPath.remoteAddress
/**
* Connect to remote server.
*/
@ -147,9 +150,11 @@ class ActiveRemoteClient private[akka] (
def sendSecureCookie(connection: ChannelFuture) {
val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT)
if (SECURE_COOKIE.nonEmpty) handshake.setCookie(SECURE_COOKIE.get)
val addr = remoteSupport.system.rootPath.remoteAddress
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder.setHostname(addr.hostname).setPort(addr.port).build)
if (SecureCookie.nonEmpty) handshake.setCookie(SecureCookie.get)
handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
.setHostname(senderRemoteAddress.hostname)
.setPort(senderRemoteAddress.port)
.build)
connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build))
}
@ -230,7 +235,7 @@ class ActiveRemoteClient private[akka] (
reconnectionTimeWindowStart = System.currentTimeMillis
true
} else {
val timeLeft = (RECONNECTION_TIME_WINDOW - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
val timeLeft = (ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
if (timeLeft)
log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
@ -254,8 +259,8 @@ class ActiveRemoteClientPipelineFactory(
import client.remoteSupport.clientSettings._
def getPipeline: ChannelPipeline = {
val timeout = new ReadTimeoutHandler(timer, READ_TIMEOUT.length, READ_TIMEOUT.unit)
val lenDec = new LengthFieldBasedFrameDecoder(MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
val timeout = new ReadTimeoutHandler(timer, ReadTimeout.length, ReadTimeout.unit)
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
val lenPrep = new LengthFieldPrepender(4)
val protobufDec = new ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
val protobufEnc = new ProtobufEncoder
@ -311,7 +316,7 @@ class ActiveRemoteClientHandler(
client.connect(reconnectIfAlreadyConnected = true)
}
}
}, client.remoteSupport.clientSettings.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS)
}, client.remoteSupport.clientSettings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS)
} else runOnceNow {
client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread
}
@ -353,8 +358,8 @@ class ActiveRemoteClientHandler(
class NettyRemoteSupport(_system: ActorSystem) extends RemoteSupport(_system) with RemoteMarshallingOps {
val log = Logging(system, "NettyRemoteSupport")
val serverSettings = new RemoteServerSettings(system.settings.config)
val clientSettings = new RemoteClientSettings(system.settings.config)
val serverSettings = RemoteExtension(system).settings.serverSettings
val clientSettings = RemoteExtension(system).settings.clientSettings
private val remoteClients = new HashMap[RemoteAddress, RemoteClient]
private val clientsLock = new ReentrantReadWriteLock
@ -449,7 +454,7 @@ class NettyRemoteSupport(_system: ActorSystem) extends RemoteSupport(_system) wi
def name = currentServer.get match {
case Some(server) server.name
case None "Non-running NettyRemoteServer@" + system.rootPath.remoteAddress
case None "Non-running NettyRemoteServer@" + system.asInstanceOf[ActorSystemImpl].provider.rootPath.remoteAddress
}
private val _isRunning = new Switch(false)
@ -484,7 +489,7 @@ class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Optio
val log = Logging(remoteSupport.system, "NettyRemoteServer")
import remoteSupport.serverSettings._
val address = remoteSupport.system.rootPath.remoteAddress
val address = remoteSupport.system.asInstanceOf[ActorSystemImpl].provider.rootPath.remoteAddress
val name = "NettyRemoteServer@" + address
@ -497,11 +502,11 @@ class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Optio
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, remoteSupport)
bootstrap.setPipelineFactory(pipelineFactory)
bootstrap.setOption("backlog", BACKLOG)
bootstrap.setOption("backlog", Backlog)
bootstrap.setOption("child.tcpNoDelay", true)
bootstrap.setOption("child.keepAlive", true)
bootstrap.setOption("child.reuseAddress", true)
bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT.toMillis)
bootstrap.setOption("child.connectTimeoutMillis", ConnectionTimeout.toMillis)
openChannels.add(bootstrap.bind(new InetSocketAddress(address.hostname, address.port)))
remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport))
@ -510,8 +515,8 @@ class NettyRemoteServer(val remoteSupport: NettyRemoteSupport, val loader: Optio
try {
val shutdownSignal = {
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
if (SECURE_COOKIE.nonEmpty)
b.setCookie(SECURE_COOKIE.get)
if (SecureCookie.nonEmpty)
b.setCookie(SecureCookie.get)
b.build
}
openChannels.write(remoteSupport.createControlEnvelope(shutdownSignal)).awaitUninterruptibly
@ -537,12 +542,12 @@ class RemoteServerPipelineFactory(
import remoteSupport.serverSettings._
def getPipeline: ChannelPipeline = {
val lenDec = new LengthFieldBasedFrameDecoder(MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4)
val lenPrep = new LengthFieldPrepender(4)
val protobufDec = new ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance)
val protobufEnc = new ProtobufEncoder
val authenticator = if (REQUIRE_COOKIE) new RemoteServerAuthenticationHandler(SECURE_COOKIE) :: Nil else Nil
val authenticator = if (RequireCookie) new RemoteServerAuthenticationHandler(SecureCookie) :: Nil else Nil
val remoteServer = new RemoteServerHandler(name, openChannels, loader, remoteSupport)
val stages: List[ChannelHandler] = lenDec :: protobufDec :: lenPrep :: protobufEnc :: authenticator ::: remoteServer :: Nil
new StaticChannelPipeline(stages: _*)
@ -619,7 +624,7 @@ class RemoteServerHandler(
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match {
case s @ Some(address)
if (USE_PASSIVE_CONNECTIONS)
if (UsePassiveConnections)
remoteSupport.unbindClient(address)
remoteSupport.notifyListeners(RemoteServerClientClosed(remoteSupport, s))
case None
@ -629,12 +634,12 @@ class RemoteServerHandler(
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
event.getMessage match {
case remote: AkkaRemoteProtocol if remote.hasMessage
remoteSupport.receiveMessage(new RemoteMessage(remote.getMessage, remoteSupport, applicationLoader), UNTRUSTED_MODE)
remoteSupport.receiveMessage(new RemoteMessage(remote.getMessage, remoteSupport, applicationLoader), UntrustedMode)
case remote: AkkaRemoteProtocol if remote.hasInstruction
val instruction = remote.getInstruction
instruction.getCommandType match {
case CommandType.CONNECT if USE_PASSIVE_CONNECTIONS
case CommandType.CONNECT if UsePassiveConnections
val origin = instruction.getOrigin
val inbound = RemoteAddress(origin.getHostname, origin.getPort)
val client = new PassiveRemoteClient(event.getChannel, remoteSupport, inbound)