remote cleanup: include feedback from Viktor and Patrik

- re-label Client/Server to Inbound/Outbound for netty settings
  description
- move to just using exactly one class loader for all reflective
  activities of the ActorSystem, which is either the context class
  loader or the one which loaded the ActorSystem’s class; document that
  putting Akka on boot class path will not work
- be more careful with initializing the Client- and ServerBootstrap
- rename Port to DesiredPortFromConfig to discourage misuse
- write test for NettySettings
- various small fixes
This commit is contained in:
Roland 2012-01-30 11:48:02 +01:00
parent 2bebf29c1c
commit 4fb0858e55
19 changed files with 141 additions and 83 deletions

View file

@ -298,12 +298,13 @@ class LocalActorRefProvider(
def this(_systemName: String, def this(_systemName: String,
settings: ActorSystem.Settings, settings: ActorSystem.Settings,
eventStream: EventStream, eventStream: EventStream,
scheduler: Scheduler) = scheduler: Scheduler,
classloader: ClassLoader) =
this(_systemName, this(_systemName,
settings, settings,
eventStream, eventStream,
scheduler, scheduler,
new Deployer(settings)) new Deployer(settings, classloader))
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))

View file

@ -322,6 +322,14 @@ abstract class ExtendedActorSystem extends ActorSystem {
*/ */
def deathWatch: DeathWatch def deathWatch: DeathWatch
/**
* ClassLoader which is used for reflective accesses internally. This is set
* to the context class loader, if one is set, or the class loader which
* loaded the ActorSystem implementation. The context class loader is also
* set on all threads created by the ActorSystem, if one was set during
* creation.
*/
def internalClassLoader: ClassLoader
} }
class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem { class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem {
@ -386,16 +394,17 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
val scheduler: Scheduler = createScheduler() val scheduler: Scheduler = createScheduler()
val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader
val provider: ActorRefProvider = { val provider: ActorRefProvider = {
val arguments = Seq( val arguments = Seq(
classOf[String] -> name, classOf[String] -> name,
classOf[Settings] -> settings, classOf[Settings] -> settings,
classOf[EventStream] -> eventStream, classOf[EventStream] -> eventStream,
classOf[Scheduler] -> scheduler) classOf[Scheduler] -> scheduler,
classOf[ClassLoader] -> internalClassLoader)
val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match {
ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, loader) match {
case Left(e) throw e case Left(e) throw e
case Right(p) p case Right(p) p
} }
@ -416,7 +425,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
def locker: Locker = provider.locker def locker: Locker = provider.locker
val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler)) val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(
threadFactory, eventStream, deadLetterMailbox, scheduler, internalClassLoader))
val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher
def terminationFuture: Future[Unit] = provider.terminationFuture def terminationFuture: Future[Unit] = provider.terminationFuture
@ -533,10 +544,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
private def loadExtensions() { private def loadExtensions() {
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse this.getClass.getClassLoader
settings.config.getStringList("akka.extensions") foreach { fqcn settings.config.getStringList("akka.extensions") foreach { fqcn
import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs } import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs }
getObjectFor[AnyRef](fqcn, loader).fold(_ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match { getObjectFor[AnyRef](fqcn, internalClassLoader).fold(_ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match {
case Right(p: ExtensionIdProvider) registerExtension(p.lookup()); case Right(p: ExtensionIdProvider) registerExtension(p.lookup());
case Right(p: ExtensionId[_]) registerExtension(p); case Right(p: ExtensionId[_]) registerExtension(p);
case Right(other) log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) case Right(other) log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)

View file

@ -23,7 +23,7 @@ case object LocalScope extends Scope
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Deployer(val settings: ActorSystem.Settings) { class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -41,7 +41,6 @@ class Deployer(val settings: ActorSystem.Settings) {
def deploy(d: Deploy): Unit = deployments.put(d.path, d) def deploy(d: Deploy): Unit = deployments.put(d.path, d)
protected def parseConfig(key: String, config: Config): Option[Deploy] = { protected def parseConfig(key: String, config: Config): Option[Deploy] = {
import akka.util.ReflectiveAccess.getClassFor
val deployment = config.withFallback(default) val deployment = config.withFallback(default)
@ -65,8 +64,8 @@ class Deployer(val settings: ActorSystem.Settings) {
case "scatter-gather" ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) case "scatter-gather" ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer)
case "broadcast" BroadcastRouter(nrOfInstances, routees, resizer) case "broadcast" BroadcastRouter(nrOfInstances, routees, resizer)
case fqn case fqn
val constructorSignature = Array[Class[_]](classOf[Config]) val args = Seq(classOf[Config] -> deployment)
ReflectiveAccess.createInstance[RouterConfig](fqn, constructorSignature, Array[AnyRef](deployment)) match { ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match {
case Right(router) router case Right(router) router
case Left(exception) case Left(exception)
throw new IllegalArgumentException( throw new IllegalArgumentException(

View file

@ -322,8 +322,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
} }
case fqcn case fqcn
val args = Seq(classOf[Config] -> config) val args = Seq(classOf[Config] -> config)
val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match {
ReflectiveAccess.createInstance[MailboxType](fqcn, args, loader) match {
case Right(instance) instance case Right(instance) instance
case Left(exception) case Left(exception)
throw new IllegalArgumentException( throw new IllegalArgumentException(

View file

@ -19,13 +19,15 @@ trait DispatcherPrerequisites {
def eventStream: EventStream def eventStream: EventStream
def deadLetterMailbox: Mailbox def deadLetterMailbox: Mailbox
def scheduler: Scheduler def scheduler: Scheduler
def classloader: ClassLoader
} }
case class DefaultDispatcherPrerequisites( case class DefaultDispatcherPrerequisites(
val threadFactory: ThreadFactory, val threadFactory: ThreadFactory,
val eventStream: EventStream, val eventStream: EventStream,
val deadLetterMailbox: Mailbox, val deadLetterMailbox: Mailbox,
val scheduler: Scheduler) extends DispatcherPrerequisites val scheduler: Scheduler,
val classloader: ClassLoader) extends DispatcherPrerequisites
object Dispatchers { object Dispatchers {
/** /**
@ -134,8 +136,8 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
case "BalancingDispatcher" new BalancingDispatcherConfigurator(cfg, prerequisites) case "BalancingDispatcher" new BalancingDispatcherConfigurator(cfg, prerequisites)
case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites) case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites)
case fqn case fqn
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites]) val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, constructorSignature, Array[AnyRef](cfg, prerequisites)) match { ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match {
case Right(configurator) configurator case Right(configurator) configurator
case Left(exception) case Left(exception)
throw new IllegalArgumentException( throw new IllegalArgumentException(

View file

@ -101,7 +101,7 @@ trait LoggingBus extends ActorEventBus {
if loggerName != StandardOutLoggerName if loggerName != StandardOutLoggerName
} yield { } yield {
try { try {
ReflectiveAccess.getClassFor[Actor](loggerName) match { ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match {
case Right(actorClass) addLogger(system, actorClass, level, logName) case Right(actorClass) addLogger(system, actorClass, level, logName)
case Left(exception) throw exception case Left(exception) throw exception
} }

View file

@ -217,6 +217,20 @@ and parsed by the actor system can be displayed like this:
println(system.settings()); println(system.settings());
// this is a shortcut for system.settings().config().root().render() // this is a shortcut for system.settings().config().root().render()
A Word About ClassLoaders
-------------------------
In several places of the configuration file it is possible to specify the
fully-qualified class name of something to be instantiated by Akka. This is
done using Java reflection, which in turn uses a :class:`ClassLoader`. Getting
the right one in challenging environments like application containers or OSGi
bundles is not always trivial, the current approach of Akka is that each
:class:`ActorSystem` implementation stores the current threads context class
loader (if available, otherwise just its own loader as in
``this.getClass.getClassLoader``) and uses that for all reflective accesses.
This implies that putting Akka on the boot class path will yield
:class:`NullPointerException` from strange places: this is simply not
supported.
Application specific settings Application specific settings
----------------------------- -----------------------------

View file

@ -3,7 +3,7 @@
*/ */
package akka.actor.mailbox package akka.actor.mailbox
import akka.actor.{ ActorContext, ActorRef } import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem }
import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox } import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox }
import akka.remote.MessageSerializer import akka.remote.MessageSerializer
import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol } import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol }
@ -15,7 +15,7 @@ private[akka] object DurableExecutableMailboxConfig {
abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue { abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue {
import DurableExecutableMailboxConfig._ import DurableExecutableMailboxConfig._
def system = owner.system def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
def ownerPath = owner.self.path def ownerPath = owner.self.path
val ownerPathString = ownerPath.elements.mkString("/") val ownerPathString = ownerPath.elements.mkString("/")
val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_") val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")

View file

@ -16,11 +16,9 @@ import org.bson.DefaultBSONSerializer
import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.RemoteProtocol.MessageProtocol
import akka.remote.MessageSerializer import akka.remote.MessageSerializer
import akka.actor.{ ActorSystem, ActorSystemImpl } import akka.actor.ExtendedActorSystem
class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging {
val systemImpl = system.asInstanceOf[ActorSystemImpl]
protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = { protected[akka] def serializeDurableMsg(msg: MongoDurableMessage)(implicit serializer: BSONSerializer) = {
@ -67,10 +65,10 @@ class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObjec
val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument]
system.log.debug("Deserializing a durable message from MongoDB: {}", doc) system.log.debug("Deserializing a durable message from MongoDB: {}", doc)
val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData)
val msg = MessageSerializer.deserialize(system, msgData, getClass.getClassLoader) val msg = MessageSerializer.deserialize(system, msgData, system.internalClassLoader)
val ownerPath = doc.as[String]("ownerPath") val ownerPath = doc.as[String]("ownerPath")
val senderPath = doc.as[String]("senderPath") val senderPath = doc.as[String]("senderPath")
val sender = systemImpl.actorFor(senderPath) val sender = system.actorFor(senderPath)
MongoDurableMessage(ownerPath, msg, sender) MongoDurableMessage(ownerPath, msg, sender)
} }

View file

@ -55,69 +55,66 @@ akka {
# If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged
log-sent-messages = off log-sent-messages = off
# Each property is annotated with (C) or (S) or (C&S), where C stands for “client” and S for “server” role. # Each property is annotated with (I) or (O) or (I&O), where I stands for “inbound” and O for “outbound” connections.
# The NettyRemoteTransport always starts the server role to allow inbound connections, and it starts # The NettyRemoteTransport always starts the server role to allow inbound connections, and it starts
# active client connections whenever sending to a destination which is not yet connected; if configured # active client connections whenever sending to a destination which is not yet connected; if configured
# it reuses inbound connections for replies, which is called a passive client connection (i.e. from server # it reuses inbound connections for replies, which is called a passive client connection (i.e. from server
# to client). # to client).
netty { netty {
# (C) In case of increased latency / overflow how long # (O) In case of increased latency / overflow how long
# should we wait (blocking the sender) until we deem the send to be cancelled? # should we wait (blocking the sender) until we deem the send to be cancelled?
# 0 means "never backoff", any positive number will indicate time to block at most. # 0 means "never backoff", any positive number will indicate time to block at most.
backoff-timeout = 0ms backoff-timeout = 0ms
# (C&S) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' # (I&O) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh'
# or using 'akka.util.Crypt.generateSecureCookie' # or using 'akka.util.Crypt.generateSecureCookie'
secure-cookie = "" secure-cookie = ""
# (S) Should the remote server require that it peers share the same secure-cookie # (I) Should the remote server require that it peers share the same secure-cookie
# (defined in the 'remote' section)? # (defined in the 'remote' section)?
require-cookie = off require-cookie = off
# (S) Reuse inbound connections for outbound messages # (I) Reuse inbound connections for outbound messages
use-passive-connections = on use-passive-connections = on
# (C&S) Whether any Threds created by the remoting should be daemons or not # (I) The hostname or ip to bind the remoting to,
daemonic = on
# (S) The hostname or ip to bind the remoting to,
# InetAddress.getLocalHost.getHostAddress is used if empty # InetAddress.getLocalHost.getHostAddress is used if empty
hostname = "" hostname = ""
# (S) The default remote server port clients should connect to. # (I) The default remote server port clients should connect to.
# Default is 2552 (AKKA), use 0 if you want a random available port # Default is 2552 (AKKA), use 0 if you want a random available port
port = 2552 port = 2552
# (C&S) Increase this if you want to be able to send messages with large payloads # (I&O) Increase this if you want to be able to send messages with large payloads
message-frame-size = 1 MiB message-frame-size = 1 MiB
# (C) Timeout duration # (O) Timeout duration
connection-timeout = 120s connection-timeout = 120s
# (S) Sets the size of the connection backlog # (I) Sets the size of the connection backlog
backlog = 4096 backlog = 4096
# (S) Length in akka.time-unit how long core threads will be kept alive if idling # (I) Length in akka.time-unit how long core threads will be kept alive if idling
execution-pool-keepalive = 60s execution-pool-keepalive = 60s
# (S) Size of the core pool of the remote execution unit # (I) Size of the core pool of the remote execution unit
execution-pool-size = 4 execution-pool-size = 4
# (S) Maximum channel size, 0 for off # (I) Maximum channel size, 0 for off
max-channel-memory-size = 0b max-channel-memory-size = 0b
# (S) Maximum total size of all channels, 0 for off # (I) Maximum total size of all channels, 0 for off
max-total-memory-size = 0b max-total-memory-size = 0b
# (C) Time between reconnect attempts for active clients # (O) Time between reconnect attempts for active clients
reconnect-delay = 5s reconnect-delay = 5s
# (C) Inactivity period after which active client connection is shutdown; will be # (O) Inactivity period after which active client connection is shutdown; will be
# re-established in case of new communication requests # re-established in case of new communication requests
read-timeout = 3600s read-timeout = 3600s
# (C) Maximum time window that a client should try to reconnect for # (O) Maximum time window that a client should try to reconnect for
reconnection-time-window = 600s reconnection-time-window = 600s
} }

View file

@ -22,11 +22,12 @@ class RemoteActorRefProvider(
val systemName: String, val systemName: String,
val settings: ActorSystem.Settings, val settings: ActorSystem.Settings,
val eventStream: EventStream, val eventStream: EventStream,
val scheduler: Scheduler) extends ActorRefProvider { val scheduler: Scheduler,
val classloader: ClassLoader) extends ActorRefProvider {
val remoteSettings = new RemoteSettings(settings.config, systemName) val remoteSettings = new RemoteSettings(settings.config, systemName)
val deployer = new RemoteDeployer(settings) val deployer = new RemoteDeployer(settings, classloader)
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer)
@ -87,7 +88,7 @@ class RemoteActorRefProvider(
classOf[ActorSystemImpl] -> system, classOf[ActorSystemImpl] -> system,
classOf[RemoteActorRefProvider] -> this) classOf[RemoteActorRefProvider] -> this)
ReflectiveAccess.createInstance[RemoteTransport](fqn, args, getClass.getClassLoader) match { ReflectiveAccess.createInstance[RemoteTransport](fqn, args, system.internalClassLoader) match {
case Left(problem) throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) case Left(problem) throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
case Right(remote) remote case Right(remote) remote
} }

View file

@ -17,8 +17,10 @@ case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMs
* Internal system "daemon" actor for remote internal communication. * Internal system "daemon" actor for remote internal communication.
* *
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action. * It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
*
* INTERNAL USE ONLY!
*/ */
class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter)
extends VirtualPathContainer(system.provider, _path, _parent, _log) { extends VirtualPathContainer(system.provider, _path, _parent, _log) {
/** /**

View file

@ -10,7 +10,7 @@ import akka.config.ConfigurationException
case class RemoteScope(node: Address) extends Scope case class RemoteScope(node: Address) extends Scope
class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings) { class RemoteDeployer(_settings: ActorSystem.Settings, _classloader: ClassLoader) extends Deployer(_settings, _classloader) {
override protected def parseConfig(path: String, config: Config): Option[Deploy] = { override protected def parseConfig(path: String, config: Config): Option[Deploy] = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._

View file

@ -158,11 +158,12 @@ class ActiveRemoteClient private[akka] (
executionHandler = new ExecutionHandler(netty.executor) executionHandler = new ExecutionHandler(netty.executor)
bootstrap = new ClientBootstrap(netty.clientChannelFactory) val b = new ClientBootstrap(netty.clientChannelFactory)
bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this)) b.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, b, executionHandler, remoteAddress, this))
bootstrap.setOption("tcpNoDelay", true) b.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true) b.setOption("keepAlive", true)
bootstrap.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis) b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis)
bootstrap = b
val remoteIP = InetAddress.getByName(remoteAddress.host.get) val remoteIP = InetAddress.getByName(remoteAddress.host.get)
log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP) log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP)

View file

@ -33,20 +33,19 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor
val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName) val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName)
val threadFactory = new MonitorableThreadFactory("NettyRemoteTransport", settings.Daemonic, Some(getClass.getClassLoader)) val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory)
val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory)
val executor = new OrderedMemoryAwareThreadPoolExecutor( val executor = new OrderedMemoryAwareThreadPoolExecutor(
settings.ExecutionPoolSize, settings.ExecutionPoolSize,
settings.MaxChannelMemorySize, settings.MaxChannelMemorySize,
settings.MaxTotalMemorySize, settings.MaxTotalMemorySize,
settings.ExecutionPoolKeepAlive.length, settings.ExecutionPoolKeepalive.length,
settings.ExecutionPoolKeepAlive.unit, settings.ExecutionPoolKeepalive.unit,
threadFactory) system.threadFactory)
val clientChannelFactory = new NioClientSocketChannelFactory( val clientChannelFactory = new NioClientSocketChannelFactory(
Executors.newCachedThreadPool(threadFactory), Executors.newCachedThreadPool(system.threadFactory),
Executors.newCachedThreadPool(threadFactory)) Executors.newCachedThreadPool(system.threadFactory))
private val remoteClients = new HashMap[Address, RemoteClient] private val remoteClients = new HashMap[Address, RemoteClient]
private val clientsLock = new ReentrantReadWriteLock private val clientsLock = new ReentrantReadWriteLock
@ -79,7 +78,11 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor
def shutdown(): Unit = { def shutdown(): Unit = {
clientsLock.writeLock().lock() clientsLock.writeLock().lock()
try { try {
remoteClients foreach { case (_, client) client.shutdown() } remoteClients foreach {
case (_, client) try client.shutdown() catch {
case e log.error(e, "failure while shutting down [{}]", client)
}
}
remoteClients.clear() remoteClients.clear()
} finally { } finally {
clientsLock.writeLock().unlock() clientsLock.writeLock().unlock()

View file

@ -29,28 +29,29 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) {
val ip = InetAddress.getByName(settings.Hostname) val ip = InetAddress.getByName(settings.Hostname)
private val factory = new NioServerSocketChannelFactory( private val factory = new NioServerSocketChannelFactory(
Executors.newCachedThreadPool(netty.threadFactory), Executors.newCachedThreadPool(netty.system.threadFactory),
Executors.newCachedThreadPool(netty.threadFactory)) Executors.newCachedThreadPool(netty.system.threadFactory))
private val bootstrap = new ServerBootstrap(factory)
private val executionHandler = new ExecutionHandler(netty.executor) private val executionHandler = new ExecutionHandler(netty.executor)
// group of open channels, used for clean-up // group of open channels, used for clean-up
private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server")
val pipelineFactory = new RemoteServerPipelineFactory(openChannels, executionHandler, netty) private val bootstrap = {
bootstrap.setPipelineFactory(pipelineFactory) val b = new ServerBootstrap(factory)
bootstrap.setOption("backlog", settings.Backlog) b.setPipelineFactory(new RemoteServerPipelineFactory(openChannels, executionHandler, netty))
bootstrap.setOption("tcpNoDelay", true) b.setOption("backlog", settings.Backlog)
bootstrap.setOption("child.keepAlive", true) b.setOption("tcpNoDelay", true)
bootstrap.setOption("reuseAddress", true) b.setOption("child.keepAlive", true)
b.setOption("reuseAddress", true)
b
}
@volatile @volatile
private[akka] var channel: Channel = _ private[akka] var channel: Channel = _
def start(): Unit = { def start(): Unit = {
channel = bootstrap.bind(new InetSocketAddress(ip, settings.Port)) channel = bootstrap.bind(new InetSocketAddress(ip, settings.DesiredPortFromConfig))
openChannels.add(channel) openChannels.add(channel)
netty.notifyListeners(RemoteServerStarted(netty)) netty.notifyListeners(RemoteServerStarted(netty))
} }
@ -62,7 +63,7 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) {
b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder
.setSystem(settings.systemName) .setSystem(settings.systemName)
.setHostname(settings.Hostname) .setHostname(settings.Hostname)
.setPort(settings.Port) .setPort(settings.DesiredPortFromConfig)
.build) .build)
if (settings.SecureCookie.nonEmpty) if (settings.SecureCookie.nonEmpty)
b.setCookie(settings.SecureCookie.get) b.setCookie(settings.SecureCookie.get)
@ -139,6 +140,7 @@ class RemoteServerHandler(
private var addressToSet = true private var addressToSet = true
// TODO look into moving that into onBind or similar, but verify that that is guaranteed to be the first to be called
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
if (addressToSet) { if (addressToSet) {
netty.setAddressFromChannel(event.getChannel) netty.setAddressFromChannel(event.getChannel)

View file

@ -13,7 +13,6 @@ class NettySettings(config: Config, val systemName: String) {
import config._ import config._
val Daemonic = getBoolean("daemonic")
val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS) val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS)
val SecureCookie: Option[String] = getString("secure-cookie") match { val SecureCookie: Option[String] = getString("secure-cookie") match {
@ -38,13 +37,13 @@ class NettySettings(config: Config, val systemName: String) {
case "" InetAddress.getLocalHost.getHostAddress case "" InetAddress.getLocalHost.getHostAddress
case value value case value value
} }
val Port = getInt("port") val DesiredPortFromConfig = getInt("port")
val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS)
val Backlog = getInt("backlog") val Backlog = getInt("backlog")
val ExecutionPoolKeepAlive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS) val ExecutionPoolKeepalive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS)
val ExecutionPoolSize = getInt("execution-pool-size") match { val ExecutionPoolSize = getInt("execution-pool-size") match {
case sz if sz < 1 throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1") case sz if sz < 1 throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1")

View file

@ -89,7 +89,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter
} }
val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) {
case ref: ActorRef (ref.asInstanceOf[ActorRef].path.address.hostPort, 1) case ref: ActorRef (ref.path.address.hostPort, 1)
}).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) { }).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) {
case (m, (n, c)) m + (n -> (m(n) + c)) case (m, (n, c)) m + (n -> (m(n) + c))
} }

View file

@ -6,6 +6,8 @@ package akka.remote
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.util.duration._ import akka.util.duration._
import akka.util.Duration
import akka.remote.netty.NettyRemoteTransport
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class RemoteConfigSpec extends AkkaSpec( class RemoteConfigSpec extends AkkaSpec(
@ -17,8 +19,9 @@ class RemoteConfigSpec extends AkkaSpec(
} }
""") { """) {
"RemoteExtension" must { "Remoting" must {
"be able to parse remote and cluster config elements" in {
"be able to parse generic remote config elements" in {
val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings
import settings._ import settings._
@ -33,5 +36,32 @@ class RemoteConfigSpec extends AkkaSpec(
GossipFrequency must be(1 second) GossipFrequency must be(1 second)
SeedNodes must be(Set()) SeedNodes must be(Set())
} }
"be able to parse Netty config elements" in {
val settings =
system.asInstanceOf[ExtendedActorSystem]
.provider.asInstanceOf[RemoteActorRefProvider]
.transport.asInstanceOf[NettyRemoteTransport]
.settings
import settings._
BackoffTimeout must be(Duration.Zero)
SecureCookie must be(None)
RequireCookie must be(false)
UsePassiveConnections must be(true)
Hostname must not be "" // will be set to the local IP
DesiredPortFromConfig must be(2552)
MessageFrameSize must be(1048576)
ConnectionTimeout must be(2 minutes)
Backlog must be(4096)
ExecutionPoolKeepalive must be(1 minute)
ExecutionPoolSize must be(4)
MaxChannelMemorySize must be(0)
MaxTotalMemorySize must be(0)
ReconnectDelay must be(5 seconds)
ReadTimeout must be(1 hour)
ReconnectionTimeWindow must be(10 minutes)
}
} }
} }