Switching to using OrderedMemoryAware executor same for all of the remoting, added more remote config tests and switched to mem size conf

This commit is contained in:
Viktor Klang 2011-12-30 12:04:20 +01:00
parent 88d2427b7f
commit 5d2bd2492f
7 changed files with 47 additions and 31 deletions

View file

@ -16,7 +16,6 @@ import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, Lengt
import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
import org.jboss.netty.handler.execution.{ OrderedMemoryAwareThreadPoolExecutor, ExecutionHandler }
import scala.collection.mutable.HashMap
import java.net.InetSocketAddress
import java.util.concurrent._
@ -26,6 +25,7 @@ import akka.event.Logging
import locks.ReentrantReadWriteLock
import org.jboss.netty.channel._
import akka.actor.ActorSystemImpl
import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor }
class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
@ -139,6 +139,8 @@ class ActiveRemoteClient private[akka] (
def currentChannel = connection.getChannel
private val senderRemoteAddress = remoteSupport.remote.remoteAddress
@volatile
private var executionHandler: ExecutionHandler = _
/**
* Connect to remote server.
@ -179,8 +181,10 @@ class ActiveRemoteClient private[akka] (
runSwitch switchOn {
openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName)
executionHandler = new ExecutionHandler(remoteSupport.executor)
bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool))
bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, remoteAddress, this))
bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this))
bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true)
@ -218,6 +222,8 @@ class ActiveRemoteClient private[akka] (
notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress))
openChannels.close.awaitUninterruptibly
openChannels = null
executionHandler.releaseExternalResources()
executionHandler = null
bootstrap.releaseExternalResources()
bootstrap = null
connection = null
@ -244,6 +250,7 @@ class ActiveRemoteClient private[akka] (
class ActiveRemoteClientPipelineFactory(
name: String,
bootstrap: ClientBootstrap,
executionHandler: ExecutionHandler,
remoteAddress: RemoteNettyAddress,
client: ActiveRemoteClient) extends ChannelPipelineFactory {
@ -257,7 +264,7 @@ class ActiveRemoteClientPipelineFactory(
val protobufEnc = new ProtobufEncoder
val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.remoteSupport.timer, client)
new StaticChannelPipeline(timeout, lenDec, protobufDec, lenPrep, protobufEnc, remoteClient)
new StaticChannelPipeline(timeout, lenDec, protobufDec, lenPrep, protobufEnc, executionHandler, remoteClient)
}
}
@ -351,8 +358,12 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
val clientSettings = remote.remoteSettings.clientSettings
val timer: HashedWheelTimer = new HashedWheelTimer
_system.registerOnTermination(timer.stop()) //Shut this guy down at the end
val executor = new OrderedMemoryAwareThreadPoolExecutor(
serverSettings.ExecutionPoolSize,
serverSettings.MaxChannelMemorySize,
serverSettings.MaxTotalMemorySize,
serverSettings.ExecutionPoolKeepAlive.length,
serverSettings.ExecutionPoolKeepAlive.unit)
private val remoteClients = new HashMap[RemoteNettyAddress, RemoteClient]
private val clientsLock = new ReentrantReadWriteLock
@ -486,7 +497,11 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre
remoteClients.clear()
} finally {
clientsLock.writeLock().unlock()
currentServer.getAndSet(None) foreach { _.shutdown() }
try {
currentServer.getAndSet(None) foreach { _.shutdown() }
} finally {
try { timer.stop() } finally { executor.shutdown() }
}
}
}
}
@ -506,18 +521,12 @@ class NettyRemoteServer(
private val bootstrap = new ServerBootstrap(factory)
private val executor = new ExecutionHandler(
new OrderedMemoryAwareThreadPoolExecutor(
ExecutionPoolSize,
MaxChannelMemorySize,
MaxTotalMemorySize,
ExecutionPoolKeepAlive.length,
ExecutionPoolKeepAlive.unit))
private val executionHandler = new ExecutionHandler(remoteSupport.executor)
// group of open channels, used for clean-up
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executor, loader, remoteSupport)
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executionHandler, loader, remoteSupport)
bootstrap.setPipelineFactory(pipelineFactory)
bootstrap.setOption("backlog", Backlog)
bootstrap.setOption("child.tcpNoDelay", true)
@ -545,7 +554,7 @@ class NettyRemoteServer(
openChannels.disconnect
openChannels.close.awaitUninterruptibly
bootstrap.releaseExternalResources()
executor.releaseExternalResources()
executionHandler.releaseExternalResources()
remoteSupport.notifyListeners(RemoteServerShutdown(remoteSupport))
} catch {
case e: Exception remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport))
@ -556,7 +565,7 @@ class NettyRemoteServer(
class RemoteServerPipelineFactory(
val name: String,
val openChannels: ChannelGroup,
val executor: ExecutionHandler,
val executionHandler: ExecutionHandler,
val loader: Option[ClassLoader],
val remoteSupport: NettyRemoteSupport) extends ChannelPipelineFactory {
@ -570,7 +579,7 @@ class RemoteServerPipelineFactory(
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 :: executor :: authenticator ::: remoteServer :: Nil
val stages: List[ChannelHandler] = lenDec :: protobufDec :: lenPrep :: protobufEnc :: executionHandler :: authenticator ::: remoteServer :: Nil
new StaticChannelPipeline(stages: _*)
}
}