Merging with latest master
This commit is contained in:
commit
ea1817b6d8
39 changed files with 7022 additions and 218 deletions
|
|
@ -100,8 +100,6 @@ private[akka] class ActiveRemoteClient private[akka] (
|
|||
private var connection: ChannelFuture = _
|
||||
@volatile
|
||||
private[remote] var openChannels: DefaultChannelGroup = _
|
||||
@volatile
|
||||
private var executionHandler: ExecutionHandler = _
|
||||
|
||||
@volatile
|
||||
private var reconnectionTimeWindowStart = 0L
|
||||
|
|
@ -144,9 +142,8 @@ private[akka] class ActiveRemoteClient private[akka] (
|
|||
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, localAddress, this))
|
||||
b.setPipelineFactory(netty.createPipeline(new ActiveRemoteClientHandler(name, b, remoteAddress, localAddress, netty.timer, this), true))
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("keepAlive", true)
|
||||
b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis)
|
||||
|
|
@ -164,6 +161,7 @@ private[akka] class ActiveRemoteClient private[akka] (
|
|||
notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress))
|
||||
false
|
||||
} else {
|
||||
ChannelAddress.set(connection.getChannel, Some(remoteAddress))
|
||||
sendSecureCookie(connection)
|
||||
notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
true
|
||||
|
|
@ -187,14 +185,15 @@ private[akka] class ActiveRemoteClient private[akka] (
|
|||
|
||||
notifyListeners(RemoteClientShutdown(netty, remoteAddress))
|
||||
try {
|
||||
if ((connection ne null) && (connection.getChannel ne null))
|
||||
if ((connection ne null) && (connection.getChannel ne null)) {
|
||||
ChannelAddress.remove(connection.getChannel)
|
||||
connection.getChannel.close()
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
if (openChannels ne null) openChannels.close.awaitUninterruptibly()
|
||||
} finally {
|
||||
connection = null
|
||||
executionHandler = null
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -307,35 +306,9 @@ private[akka] class ActiveRemoteClientHandler(
|
|||
}
|
||||
}
|
||||
|
||||
private[akka] class ActiveRemoteClientPipelineFactory(
|
||||
name: String,
|
||||
bootstrap: ClientBootstrap,
|
||||
executionHandler: ExecutionHandler,
|
||||
remoteAddress: Address,
|
||||
localAddress: Address,
|
||||
client: ActiveRemoteClient) extends ChannelPipelineFactory {
|
||||
|
||||
import client.netty.settings
|
||||
|
||||
def getPipeline: ChannelPipeline = {
|
||||
val timeout = new IdleStateHandler(client.netty.timer,
|
||||
settings.ReadTimeout.toSeconds.toInt,
|
||||
settings.WriteTimeout.toSeconds.toInt,
|
||||
settings.AllTimeout.toSeconds.toInt)
|
||||
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, localAddress, client.netty.timer, client)
|
||||
|
||||
new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient)
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] class PassiveRemoteClient(val currentChannel: Channel,
|
||||
netty: NettyRemoteTransport,
|
||||
remoteAddress: Address)
|
||||
extends RemoteClient(netty, remoteAddress) {
|
||||
remoteAddress: Address) extends RemoteClient(netty, remoteAddress) {
|
||||
|
||||
def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn {
|
||||
netty.notifyListeners(RemoteClientStarted(netty, remoteAddress))
|
||||
|
|
|
|||
|
|
@ -12,9 +12,11 @@ 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.channel.{ ChannelHandlerContext, Channel, StaticChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal }
|
||||
import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder }
|
||||
import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder }
|
||||
import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor
|
||||
import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor }
|
||||
import org.jboss.netty.handler.timeout.IdleStateHandler
|
||||
import org.jboss.netty.util.HashedWheelTimer
|
||||
import akka.event.Logging
|
||||
import akka.remote.RemoteProtocol.AkkaRemoteProtocol
|
||||
|
|
@ -22,6 +24,10 @@ import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteActorRefPr
|
|||
import akka.util.NonFatal
|
||||
import akka.actor.{ ExtendedActorSystem, Address, ActorRef }
|
||||
|
||||
object ChannelAddress extends ChannelLocal[Option[Address]] {
|
||||
override def initialValue(ch: Channel): Option[Address] = None
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides the implementation of the Netty remote support
|
||||
*/
|
||||
|
|
@ -31,28 +37,111 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
|
||||
val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName)
|
||||
|
||||
// TODO replace by system.scheduler
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory)
|
||||
|
||||
val executor = new OrderedMemoryAwareThreadPoolExecutor(
|
||||
settings.ExecutionPoolSize,
|
||||
settings.MaxChannelMemorySize,
|
||||
settings.MaxTotalMemorySize,
|
||||
settings.ExecutionPoolKeepalive.length,
|
||||
settings.ExecutionPoolKeepalive.unit,
|
||||
system.threadFactory)
|
||||
|
||||
// TODO make configurable/shareable with server socket factory
|
||||
val clientChannelFactory = new NioClientSocketChannelFactory(
|
||||
Executors.newCachedThreadPool(system.threadFactory),
|
||||
Executors.newCachedThreadPool(system.threadFactory))
|
||||
|
||||
/**
|
||||
* Backing scaffolding for the default implementation of NettyRemoteSupport.createPipeline.
|
||||
*/
|
||||
object PipelineFactory {
|
||||
/**
|
||||
* Construct a StaticChannelPipeline from a sequence of handlers; to be used
|
||||
* in implementations of ChannelPipelineFactory.
|
||||
*/
|
||||
def apply(handlers: Seq[ChannelHandler]): StaticChannelPipeline = new StaticChannelPipeline(handlers: _*)
|
||||
|
||||
/**
|
||||
* Constructs the NettyRemoteTransport default pipeline with the give “head” handler, which
|
||||
* is taken by-name to allow it not to be shared across pipelines.
|
||||
*
|
||||
* @param withTimeout determines whether an IdleStateHandler shall be included
|
||||
*/
|
||||
def apply(endpoint: ⇒ Seq[ChannelHandler], withTimeout: Boolean): ChannelPipelineFactory =
|
||||
new ChannelPipelineFactory {
|
||||
def getPipeline = apply(defaultStack(withTimeout) ++ endpoint)
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct a default protocol stack, excluding the “head” handler (i.e. the one which
|
||||
* actually dispatches the received messages to the local target actors).
|
||||
*/
|
||||
def defaultStack(withTimeout: Boolean): Seq[ChannelHandler] =
|
||||
(if (withTimeout) timeout :: Nil else Nil) :::
|
||||
msgFormat :::
|
||||
authenticator :::
|
||||
executionHandler ::
|
||||
Nil
|
||||
|
||||
/**
|
||||
* Construct an IdleStateHandler which uses [[akka.remote.netty.NettyRemoteTransport]].timer.
|
||||
*/
|
||||
def timeout = new IdleStateHandler(timer,
|
||||
settings.ReadTimeout.toSeconds.toInt,
|
||||
settings.WriteTimeout.toSeconds.toInt,
|
||||
settings.AllTimeout.toSeconds.toInt)
|
||||
|
||||
/**
|
||||
* Construct frame&protobuf encoder/decoder.
|
||||
*/
|
||||
def msgFormat = new LengthFieldBasedFrameDecoder(settings.MessageFrameSize, 0, 4, 0, 4) ::
|
||||
new LengthFieldPrepender(4) ::
|
||||
new RemoteMessageDecoder ::
|
||||
new RemoteMessageEncoder(NettyRemoteTransport.this) ::
|
||||
Nil
|
||||
|
||||
/**
|
||||
* Construct an ExecutionHandler which is used to ensure that message dispatch does not
|
||||
* happen on a netty thread (that could be bad if re-sending over the network for
|
||||
* remote-deployed actors).
|
||||
*/
|
||||
val executionHandler = new ExecutionHandler(new OrderedMemoryAwareThreadPoolExecutor(
|
||||
settings.ExecutionPoolSize,
|
||||
settings.MaxChannelMemorySize,
|
||||
settings.MaxTotalMemorySize,
|
||||
settings.ExecutionPoolKeepalive.length,
|
||||
settings.ExecutionPoolKeepalive.unit,
|
||||
system.threadFactory))
|
||||
|
||||
/**
|
||||
* Construct and authentication handler which uses the SecureCookie to somewhat
|
||||
* protect the TCP port from unauthorized use (don’t rely on it too much, though,
|
||||
* as this is NOT a cryptographic feature).
|
||||
*/
|
||||
def authenticator = if (settings.RequireCookie) new RemoteServerAuthenticationHandler(settings.SecureCookie) :: Nil else Nil
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is factored out to provide an extension point in case the
|
||||
* pipeline shall be changed. It is recommended to use
|
||||
*/
|
||||
def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean): ChannelPipelineFactory =
|
||||
PipelineFactory(Seq(endpoint), withTimeout)
|
||||
|
||||
private val remoteClients = new HashMap[Address, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
||||
override protected def useUntrustedMode = remoteSettings.UntrustedMode
|
||||
|
||||
val server = try new NettyRemoteServer(this) catch {
|
||||
case ex ⇒ shutdown(); throw ex
|
||||
}
|
||||
val server: NettyRemoteServer = try createServer() catch { case NonFatal(ex) ⇒ shutdown(); throw ex }
|
||||
|
||||
/**
|
||||
* Override this method to inject a subclass of NettyRemoteServer instead of
|
||||
* the normal one, e.g. for inserting security hooks. If this method throws
|
||||
* an exception, the transport will shut itself down and re-throw.
|
||||
*/
|
||||
protected def createServer(): NettyRemoteServer = new NettyRemoteServer(this)
|
||||
|
||||
/**
|
||||
* Override this method to inject a subclass of RemoteClient instead of
|
||||
* the normal one, e.g. for inserting security hooks. Get this transport’s
|
||||
* address from `this.address`.
|
||||
*/
|
||||
protected def createClient(recipient: Address): RemoteClient = new ActiveRemoteClient(this, recipient, address)
|
||||
|
||||
// the address is set in start() or from the RemoteServerHandler, whichever comes first
|
||||
private val _address = new AtomicReference[Address]
|
||||
|
|
@ -91,11 +180,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
try {
|
||||
timer.stop()
|
||||
} finally {
|
||||
try {
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
} finally {
|
||||
executor.shutdown()
|
||||
}
|
||||
clientChannelFactory.releaseExternalResources()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -121,7 +206,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
//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, address)
|
||||
val client = createClient(recipientAddress)
|
||||
remoteClients += recipientAddress -> client
|
||||
client
|
||||
}
|
||||
|
|
|
|||
|
|
@ -35,14 +35,12 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
|||
new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool())
|
||||
}
|
||||
|
||||
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.setPipelineFactory(netty.createPipeline(new RemoteServerHandler(openChannels, netty), false))
|
||||
b.setOption("backlog", settings.Backlog)
|
||||
b.setOption("tcpNoDelay", true)
|
||||
b.setOption("child.keepAlive", true)
|
||||
|
|
@ -82,26 +80,6 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
|||
}
|
||||
}
|
||||
|
||||
private[akka] 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
|
||||
private[akka] class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler {
|
||||
val authenticated = new AnyRef
|
||||
|
|
@ -134,10 +112,6 @@ private[akka] class RemoteServerHandler(
|
|||
val openChannels: ChannelGroup,
|
||||
val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler {
|
||||
|
||||
val channelAddress = new ChannelLocal[Option[Address]](false) {
|
||||
override def initialValue(channel: Channel) = None
|
||||
}
|
||||
|
||||
import netty.settings
|
||||
|
||||
private var addressToSet = true
|
||||
|
|
@ -161,16 +135,16 @@ private[akka] class RemoteServerHandler(
|
|||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = ()
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
netty.notifyListeners(RemoteServerClientDisconnected(netty, channelAddress.get(ctx.getChannel)))
|
||||
netty.notifyListeners(RemoteServerClientDisconnected(netty, ChannelAddress.get(ctx.getChannel)))
|
||||
}
|
||||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val address = channelAddress.get(ctx.getChannel)
|
||||
val address = ChannelAddress.get(ctx.getChannel)
|
||||
if (address.isDefined && settings.UsePassiveConnections)
|
||||
netty.unbindClient(address.get)
|
||||
|
||||
netty.notifyListeners(RemoteServerClientClosed(netty, address))
|
||||
channelAddress.remove(ctx.getChannel)
|
||||
ChannelAddress.remove(ctx.getChannel)
|
||||
}
|
||||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try {
|
||||
|
|
@ -184,7 +158,7 @@ private[akka] class RemoteServerHandler(
|
|||
case CommandType.CONNECT ⇒
|
||||
val origin = instruction.getOrigin
|
||||
val inbound = Address("akka", origin.getSystem, origin.getHostname, origin.getPort)
|
||||
channelAddress.set(event.getChannel, Option(inbound))
|
||||
ChannelAddress.set(event.getChannel, Option(inbound))
|
||||
|
||||
//If we want to reuse the inbound connections as outbound we need to get busy
|
||||
if (settings.UsePassiveConnections)
|
||||
|
|
|
|||
|
|
@ -1,6 +1,7 @@
|
|||
package akka.remote
|
||||
|
||||
import com.typesafe.config.{Config, ConfigFactory}
|
||||
import akka.actor.Address
|
||||
|
||||
trait AbstractRemoteActorMultiJvmSpec {
|
||||
def NrOfNodes: Int
|
||||
|
|
@ -8,7 +9,6 @@ trait AbstractRemoteActorMultiJvmSpec {
|
|||
|
||||
def PortRangeStart = 1990
|
||||
def NodeRange = 1 to NrOfNodes
|
||||
def PortRange = PortRangeStart to NrOfNodes
|
||||
|
||||
private[this] val remotes: IndexedSeq[String] = {
|
||||
val nodesOpt = Option(AkkaRemoteSpec.testNodes).map(_.split(",").toIndexedSeq)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue