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
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue