Merging with latest master

This commit is contained in:
Viktor Klang 2012-05-24 11:59:36 +02:00
commit ea1817b6d8
39 changed files with 7022 additions and 218 deletions

View file

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

View file

@ -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 (dont 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 transports
* 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
}

View file

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

View file

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