From edceda8edf29fb64c2a0ab285c31b7465278e8d0 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 20 Jan 2012 14:29:50 +0100 Subject: [PATCH 01/13] restructure remoting - remove Remote, incorporate its few fields into RemoteActorRefProvider - rename RemoteSupport to RemoteTransport to sync up with conf and the way we talk about it - remove LocalAddress/RemoteAddress etc. and just have a final case class Address(protocol, system, host, port) - split netty settings out or RemoteSettings into NettySettings - split out from NettyRemoteSupport.scala: Server.scala, Client.scala, Settings.scala plus a few fixes, including using the contextClassLoader when loading the provider for ActorSystemImpl --- .../src/main/scala/akka/actor/ActorRef.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 4 +- .../main/scala/akka/actor/ActorSystem.scala | 13 +- .../src/main/scala/akka/actor/Address.scala | 68 +- .../src/main/scala/akka/event/Logging.scala | 2 +- .../scala/akka/util/ReflectiveAccess.scala | 9 + akka-remote/src/main/resources/reference.conf | 131 ++-- .../akka/remote/AccrualFailureDetector.scala | 18 +- .../src/main/scala/akka/remote/Gossiper.scala | 40 +- .../akka/remote/NetworkEventStream.scala | 13 +- .../src/main/scala/akka/remote/Remote.scala | 275 ------- .../akka/remote/RemoteActorRefProvider.scala | 86 ++- .../scala/akka/remote/RemoteAddress.scala | 5 + .../akka/remote/RemoteConnectionManager.scala | 20 +- .../main/scala/akka/remote/RemoteDaemon.scala | 80 ++ .../scala/akka/remote/RemoteDeployer.scala | 4 +- .../scala/akka/remote/RemoteInterface.scala | 343 --------- .../scala/akka/remote/RemoteSettings.scala | 77 +- .../scala/akka/remote/RemoteTransport.scala | 321 ++++++++ .../main/scala/akka/remote/netty/Client.scala | 347 +++++++++ .../remote/netty/NettyRemoteSupport.scala | 707 +++--------------- .../main/scala/akka/remote/netty/Server.scala | 202 +++++ .../scala/akka/remote/netty/Settings.scala | 63 ++ .../src/main/scala/akka/remote/package.scala | 8 - .../akka/routing/RemoteRouterConfig.scala | 6 +- .../AbstractRemoteActorMultiJvmSpec.scala | 4 +- .../scala/akka/remote/AkkaRemoteSpec.scala | 10 - .../remote/AccrualFailureDetectorSpec.scala | 3 +- .../akka/remote/NetworkFailureSpec.scala | 2 +- .../akka/remote/RemoteCommunicationSpec.scala | 4 +- .../scala/akka/remote/RemoteConfigSpec.scala | 58 +- .../akka/remote/RemoteDeathWatchSpec.scala | 4 +- .../akka/remote/RemoteDeployerSpec.scala | 2 +- .../scala/akka/remote/RemoteRouterSpec.scala | 4 +- 34 files changed, 1415 insertions(+), 1520 deletions(-) delete mode 100644 akka-remote/src/main/scala/akka/remote/Remote.scala create mode 100644 akka-remote/src/main/scala/akka/remote/RemoteAddress.scala create mode 100644 akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/RemoteInterface.scala create mode 100644 akka-remote/src/main/scala/akka/remote/RemoteTransport.scala create mode 100644 akka-remote/src/main/scala/akka/remote/netty/Client.scala create mode 100644 akka-remote/src/main/scala/akka/remote/netty/Server.scala create mode 100644 akka-remote/src/main/scala/akka/remote/netty/Settings.scala delete mode 100644 akka-remote/src/main/scala/akka/remote/package.scala diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 94ec966468..2bc8e77885 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -253,7 +253,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe } private[akka] case object Nobody extends MinimalActorRef { - val path = new RootActorPath(new LocalAddress("all-systems"), "/Nobody") + val path = new RootActorPath(Address("akka", "all-systems"), "/Nobody") } /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 7bc9f502d1..a41f4572ff 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -280,7 +280,7 @@ class LocalActorRefProvider( eventStream, scheduler, deadLetters, - new RootActorPath(LocalAddress(_systemName)), + new RootActorPath(Address("akka", _systemName)), new Deployer(settings)) val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")") @@ -443,7 +443,7 @@ class LocalActorRefProvider( deadLetters } else if (elems.head.isEmpty) actorFor(rootGuardian, elems.tail) else actorFor(ref, elems) - case LocalActorPath(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems) + case ActorPathExtractor(address, elems) if address == rootPath.address ⇒ actorFor(rootGuardian, elems) case _ ⇒ log.debug("look-up of unknown path '{}' failed", path) deadLetters diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index a84d080536..db807df939 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -341,20 +341,19 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor } val provider: ActorRefProvider = { - val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match { - case Left(e) ⇒ throw e - case Right(b) ⇒ b - } val arguments = Seq( classOf[String] -> name, classOf[Settings] -> settings, classOf[EventStream] -> eventStream, classOf[Scheduler] -> scheduler, classOf[InternalActorRef] -> deadLetters) - val types: Array[Class[_]] = arguments map (_._1) toArray - val values: Array[AnyRef] = arguments map (_._2) toArray - ReflectiveAccess.createInstance[ActorRefProvider](providerClass, types, values) match { + val loader = Thread.currentThread.getContextClassLoader match { + case null ⇒ getClass.getClassLoader + case l ⇒ l + } + + ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, loader) match { case Left(e) ⇒ throw e case Right(p) ⇒ p } diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index e9097d72d6..e9dc24dad8 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -9,17 +9,41 @@ import java.net.URISyntaxException * The address specifies the physical location under which an Actor can be * reached. Examples are local addresses, identified by the ActorSystem’s * name, and remote addresses, identified by protocol, host and port. + * + * This class is final to allow use as a case class (copy method etc.); if + * for example a remote transport would want to associate additional + * information with an address, then this must be done externally. */ -abstract class Address { - def protocol: String - def hostPort: String +final case class Address(protocol: String, system: String, host: Option[String], port: Option[Int]) { + + def this(protocol: String, system: String) = this(protocol, system, None, None) + def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port)) + @transient - override lazy val toString = protocol + "://" + hostPort + override lazy val toString = { + val sb = new StringBuilder(protocol) + sb.append("://") + sb.append(hostPort) + sb.toString + } + + @transient + lazy val hostPort = { + val sb = new StringBuilder(system) + if (host.isDefined) { + sb.append('@') + sb.append(host.get) + } + if (port.isDefined) { + sb.append(':') + sb.append(port.get) + } + sb.toString + } } -case class LocalAddress(systemName: String) extends Address { - def protocol = "akka" - def hostPort = systemName +object Address { + def apply(protocol: String, system: String) = new Address(protocol, system) } object RelativeActorPath { @@ -32,12 +56,34 @@ object RelativeActorPath { } } -object LocalActorPath { - def unapply(addr: String): Option[(LocalAddress, Iterable[String])] = { +object AddressExtractor { + def unapply(addr: String): Option[Address] = { try { val uri = new URI(addr) - if (uri.getScheme != "akka" || uri.getUserInfo != null || uri.getHost == null || uri.getPath == null) None - else Some(LocalAddress(uri.getHost), ActorPath.split(uri.getPath).drop(1)) + if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null)) None + else { + val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost, + if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost), + if (uri.getPort < 0) None else Some(uri.getPort)) + Some(addr) + } + } catch { + case _: URISyntaxException ⇒ None + } + } +} + +object ActorPathExtractor { + def unapply(addr: String): Option[(Address, Iterable[String])] = { + try { + val uri = new URI(addr) + if (uri.getScheme == null || (uri.getUserInfo == null && uri.getHost == null) || uri.getPath == null) None + else { + val addr = Address(uri.getScheme, if (uri.getUserInfo != null) uri.getUserInfo else uri.getHost, + if (uri.getUserInfo == null || uri.getHost == null) None else Some(uri.getHost), + if (uri.getPort < 0) None else Some(uri.getPort)) + Some((addr, ActorPath.split(uri.getPath).drop(1))) + } } catch { case _: URISyntaxException ⇒ None } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 07a1da1da5..16396bd62e 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -647,7 +647,7 @@ object Logging { * akka.stdout-loglevel in akka.conf. */ class StandardOutLogger extends MinimalActorRef with StdOutLogger { - val path: ActorPath = new RootActorPath(LocalAddress("all-systems"), "/StandardOutLogger") + val path: ActorPath = new RootActorPath(Address("akka", "all-systems"), "/StandardOutLogger") override val toString = "StandardOutLogger" override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message) } diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index 70b6fa5a03..0c8721db2c 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -39,6 +39,15 @@ object ReflectiveAccess { } } + def createInstance[T](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Either[Exception, T] = + createInstance(clazz, args.map(_._1).toArray, args.map(_._2).toArray) + + def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)], classloader: ClassLoader): Either[Exception, T] = + createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, classloader) + + def createInstance[T](fqcn: String, args: Seq[(Class[_], AnyRef)]): Either[Exception, T] = + createInstance(fqcn, args.map(_._1).toArray, args.map(_._2).toArray, loader) + //Obtains a reference to fqn.MODULE$ def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Either[Exception, T] = try { getClassFor(fqn, classloader) match { diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a89508982c..e519aae027 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -38,29 +38,82 @@ akka { remote { - # Which implementation of akka.remote.RemoteSupport to use + # Which implementation of akka.remote.RemoteTransport to use # default is a TCP-based remote transport based on Netty - transport = "akka.remote.netty.NettyRemoteSupport" + transport = "akka.remote.netty.NettyRemoteTransport" - # In case of increased latency / overflow how long - # 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. - backoff-timeout = 0ms - - use-compression = off - - # Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' - # or using 'akka.util.Crypt.generateSecureCookie' - secure-cookie = "" + # Enable untrusted mode for full security of server managed actors, allows + # untrusted clients to connect. + untrusted-mode = off # Timeout for ACK of cluster operations, lik checking actor out etc. remote-daemon-ack-timeout = 30s - # Reuse inbound connections for outbound messages - use-passive-connections = on + # Each property is annotated with (C) or (S) or (C&S), where C stands for “client” and S for “server” role. + # 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 + # it reuses inbound connections for replies, which is called a passive client connection (i.e. from server + # to client). + netty { + + # (C) In case of increased latency / overflow how long + # 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. + backoff-timeout = 0ms + + # (C&S) Generate your own with '$AKKA_HOME/scripts/generate_config_with_secure_cookie.sh' + # or using 'akka.util.Crypt.generateSecureCookie' + secure-cookie = "" + + # (S) Should the remote server require that it peers share the same secure-cookie + # (defined in the 'remote' section)? + require-cookie = off - # Whether any Threds created by the remoting should be daemons or not - daemonic = on + # (S) Reuse inbound connections for outbound messages + use-passive-connections = on + + # (C&S) Whether any Threds created by the remoting should be daemons or not + daemonic = on + + # (S) The hostname or ip to bind the remoting to, + # InetAddress.getLocalHost.getHostAddress is used if empty + hostname = "" + + # (S) The default remote server port clients should connect to. + # Default is 2552 (AKKA), use 0 if you want a random available port + port = 2552 + + # (C&S) Increase this if you want to be able to send messages with large payloads + message-frame-size = 1 MiB + + # (C) Timeout duration + connection-timeout = 120s + + # (S) Sets the size of the connection backlog + backlog = 4096 + + # (S) Length in akka.time-unit how long core threads will be kept alive if idling + execution-pool-keepalive = 60s + + # (S) Size of the core pool of the remote execution unit + execution-pool-size = 4 + + # (S) Maximum channel size, 0 for off + max-channel-memory-size = 0b + + # (S) Maximum total size of all channels, 0 for off + max-total-memory-size = 0b + + # (C) Time between reconnect attempts for active clients + reconnect-delay = 5s + + # (C) Inactivity period after which active client connection is shutdown; will be + # re-established in case of new communication requests + read-timeout = 3600s + + # (C) Maximum time window that a client should try to reconnect for + reconnection-time-window = 600s + } # accrual failure detection config failure-detector { @@ -93,52 +146,6 @@ akka { daemonic = on } - server { - # The hostname or ip to bind the remoting to, - # InetAddress.getLocalHost.getHostAddress is used if empty - hostname = "" - - # The default remote server port clients should connect to. - # Default is 2552 (AKKA), use 0 if you want a random available port - port = 2552 - - # Increase this if you want to be able to send messages with large payloads - message-frame-size = 1 MiB - - # Timeout duration - connection-timeout = 120s - - # Should the remote server require that it peers share the same secure-cookie - # (defined in the 'remote' section)? - require-cookie = off - - # Enable untrusted mode for full security of server managed actors, allows - # untrusted clients to connect. - untrusted-mode = off - - # Sets the size of the connection backlog - backlog = 4096 - - # Length in akka.time-unit how long core threads will be kept alive if idling - execution-pool-keepalive = 60s - - # Size of the core pool of the remote execution unit - execution-pool-size = 4 - - # Maximum channel size, 0 for off - max-channel-memory-size = 0b - - # Maximum total size of all channels, 0 for off - max-total-memory-size = 0b - } - - client { - reconnect-delay = 5s - read-timeout = 3600s - message-frame-size = 1 MiB - # Maximum time window that a client should try to reconnect for - reconnection-time-window = 600s - } } cluster { diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 23043c5303..0384358557 100644 --- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala +++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala @@ -5,13 +5,11 @@ package akka.remote import java.util.concurrent.atomic.AtomicReference - import scala.collection.immutable.Map import scala.annotation.tailrec - import System.{ currentTimeMillis ⇒ newTimestamp } - import akka.actor.ActorSystem +import akka.actor.Address /** * Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper: @@ -35,9 +33,9 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10 */ private case class State( version: Long = 0L, - failureStats: Map[ParsedTransportAddress, FailureStats] = Map.empty[ParsedTransportAddress, FailureStats], - intervalHistory: Map[ParsedTransportAddress, Vector[Long]] = Map.empty[ParsedTransportAddress, Vector[Long]], - timestamps: Map[ParsedTransportAddress, Long] = Map.empty[ParsedTransportAddress, Long]) + failureStats: Map[Address, FailureStats] = Map.empty[Address, FailureStats], + intervalHistory: Map[Address, Vector[Long]] = Map.empty[Address, Vector[Long]], + timestamps: Map[Address, Long] = Map.empty[Address, Long]) private val state = new AtomicReference[State](State()) @@ -45,13 +43,13 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10 * Returns true if the connection is considered to be up and healthy * and returns false otherwise. */ - def isAvailable(connection: ParsedTransportAddress): Boolean = phi(connection) < threshold + def isAvailable(connection: Address): Boolean = phi(connection) < threshold /** * Records a heartbeat for a connection. */ @tailrec - final def heartbeat(connection: ParsedTransportAddress) { + final def heartbeat(connection: Address) { val oldState = state.get val latestTimestamp = oldState.timestamps.get(connection) @@ -132,7 +130,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10 * Implementations of 'Cumulative Distribution Function' for Exponential Distribution. * For a discussion on the math read [https://issues.apache.org/jira/browse/CASSANDRA-2597]. */ - def phi(connection: ParsedTransportAddress): Double = { + def phi(connection: Address): Double = { val oldState = state.get val oldTimestamp = oldState.timestamps.get(connection) if (oldTimestamp.isEmpty) 0.0D // treat unmanaged connections, e.g. with zero heartbeats, as healthy connections @@ -147,7 +145,7 @@ class AccrualFailureDetector(val threshold: Int = 8, val maxSampleSize: Int = 10 * Removes the heartbeat management for a connection. */ @tailrec - final def remove(connection: ParsedTransportAddress) { + final def remove(connection: Address) { val oldState = state.get if (oldState.failureStats.contains(connection)) { diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index a12e5ecab1..9af6f584ae 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -27,8 +27,8 @@ import akka.dispatch.Await * Interface for node membership change listener. */ trait NodeMembershipChangeListener { - def nodeConnected(node: ParsedTransportAddress) - def nodeDisconnected(node: ParsedTransportAddress) + def nodeConnected(node: Address) + def nodeDisconnected(node: Address) } /** @@ -36,9 +36,9 @@ trait NodeMembershipChangeListener { */ case class Gossip( version: VectorClock, - node: ParsedTransportAddress, - availableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress], - unavailableNodes: Set[ParsedTransportAddress] = Set.empty[ParsedTransportAddress]) + node: Address, + availableNodes: Set[Address] = Set.empty[Address], + unavailableNodes: Set[Address] = Set.empty[Address]) // ====== START - NEW GOSSIP IMPLEMENTATION ====== /* @@ -94,7 +94,7 @@ case class Gossip( * gossip to random seed with certain probability depending on number of unreachable, seed and live nodes. * */ -class Gossiper(remote: Remote, system: ActorSystemImpl) { +class Gossiper(remote: RemoteActorRefProvider, system: ActorSystemImpl) { /** * Represents the state for this Gossiper. Implemented using optimistic lockless concurrency, @@ -108,27 +108,19 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { private val serialization = remote.serialization private val log = Logging(system, "Gossiper") private val failureDetector = remote.failureDetector - private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[ParsedTransportAddress, ActorRef]) + private val connectionManager = new RemoteConnectionManager(system, remote, Map.empty[Address, ActorRef]) private val seeds = { - val seeds = remoteSettings.SeedNodes flatMap { - case x: UnparsedTransportAddress ⇒ - x.parse(remote.transports) match { - case y: ParsedTransportAddress ⇒ Some(y) - case _ ⇒ None - } - case _ ⇒ None - } - if (seeds.isEmpty) throw new ConfigurationException( + if (remoteSettings.SeedNodes.isEmpty) throw new ConfigurationException( "At least one seed node must be defined in the configuration [akka.cluster.seed-nodes]") - else seeds + else remoteSettings.SeedNodes } - private val address = remote.remoteAddress + private val address = remote.transport.address private val nodeFingerprint = address.## private val random = SecureRandom.getInstance("SHA1PRNG") - private val initalDelayForGossip = remoteSettings.InitalDelayForGossip + private val initalDelayForGossip = remoteSettings.InitialDelayForGossip private val gossipFrequency = remoteSettings.GossipFrequency private val state = new AtomicReference[State](State(currentGossip = newGossip())) @@ -166,7 +158,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { node ← oldAvailableNodes if connectionManager.connectionFor(node).isEmpty } { - val connectionFactory = () ⇒ system.actorFor(RootActorPath(RemoteSystemAddress(system.name, gossipingNode)) / "remote") + val connectionFactory = () ⇒ system.actorFor(RootActorPath(gossipingNode) / "remote") connectionManager.putIfAbsent(node, connectionFactory) // create a new remote connection to the new node oldState.nodeMembershipChangeListeners foreach (_ nodeConnected node) // notify listeners about the new nodes } @@ -240,7 +232,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { /** * Gossips set of nodes passed in as argument. Returns 'true' if it gossiped to a "seed" node. */ - private def gossipTo(nodes: Set[ParsedTransportAddress]): Boolean = { + private def gossipTo(nodes: Set[Address]): Boolean = { val peers = nodes filter (_ != address) // filter out myself val peer = selectRandomNode(peers) val oldState = state.get @@ -298,8 +290,8 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { private def newGossip(): Gossip = Gossip( version = VectorClock(), - node = address.transport, - availableNodes = Set(address.transport)) + node = address, + availableNodes = Set(address)) private def incrementVersionForGossip(from: Gossip): Gossip = { val newVersion = from.version.increment(nodeFingerprint, newTimestamp) @@ -314,7 +306,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { } } - private def selectRandomNode(nodes: Set[ParsedTransportAddress]): ParsedTransportAddress = { + private def selectRandomNode(nodes: Set[Address]): Address = { nodes.toList(random.nextInt(nodes.size)) } } diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index 6d3f340cb5..763b22a6fa 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -8,6 +8,7 @@ import scala.collection.mutable import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid } import akka.actor.Actor._ import akka.actor.ActorSystemImpl +import akka.actor.Address /** * Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc. @@ -17,10 +18,10 @@ object NetworkEventStream { private sealed trait NetworkEventStreamEvent - private case class Register(listener: Listener, connectionAddress: ParsedTransportAddress) + private case class Register(listener: Listener, connectionAddress: Address) extends NetworkEventStreamEvent - private case class Unregister(listener: Listener, connectionAddress: ParsedTransportAddress) + private case class Unregister(listener: Listener, connectionAddress: Address) extends NetworkEventStreamEvent /** @@ -35,8 +36,8 @@ object NetworkEventStream { */ private class Channel extends Actor { - val listeners = new mutable.HashMap[ParsedTransportAddress, mutable.Set[Listener]]() { - override def default(k: ParsedTransportAddress) = mutable.Set.empty[Listener] + val listeners = new mutable.HashMap[Address, mutable.Set[Listener]]() { + override def default(k: Address) = mutable.Set.empty[Listener] } def receive = { @@ -67,12 +68,12 @@ class NetworkEventStream(system: ActorSystemImpl) { /** * Registers a network event stream listener (asyncronously). */ - def register(listener: Listener, connectionAddress: ParsedTransportAddress) = + def register(listener: Listener, connectionAddress: Address) = sender ! Register(listener, connectionAddress) /** * Unregisters a network event stream listener (asyncronously) . */ - def unregister(listener: Listener, connectionAddress: ParsedTransportAddress) = + def unregister(listener: Listener, connectionAddress: Address) = sender ! Unregister(listener, connectionAddress) } diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala deleted file mode 100644 index 02d6d682b0..0000000000 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ /dev/null @@ -1,275 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ - -package akka.remote - -import akka.actor._ -import akka.event._ -import akka.util._ -import akka.util.duration._ -import akka.util.Helpers._ -import akka.serialization.{ JavaSerializer, Serialization, SerializationExtension } -import akka.dispatch.MessageDispatcher -import akka.dispatch.SystemMessage -import scala.annotation.tailrec -import akka.remote.RemoteProtocol.{ ActorRefProtocol, AkkaRemoteProtocol, RemoteControlProtocol, RemoteMessageProtocol } - -/** - * Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc. - */ -class Remote(val settings: ActorSystem.Settings, val remoteSettings: RemoteSettings) { - - import settings._ - - // TODO make this really pluggable - val transports: TransportsMap = Map("akka" -> ((h, p) ⇒ Right(RemoteNettyAddress(h, p)))) - val remoteAddress: RemoteSystemAddress[ParsedTransportAddress] = { - val unparsedAddress = remoteSettings.serverSettings.URI match { - case RemoteAddressExtractor(a) ⇒ a - case x ⇒ throw new IllegalArgumentException("cannot parse URI " + x) - } - val parsed = unparsedAddress.parse(transports) match { - case Left(x) ⇒ throw new IllegalArgumentException(x.transport.error) - case Right(x) ⇒ x - } - parsed.copy(system = settings.name) - } - - val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize) - - @volatile - private var _serialization: Serialization = _ - def serialization = _serialization - - @volatile - private var _computeGridDispatcher: MessageDispatcher = _ - def computeGridDispatcher = _computeGridDispatcher - - @volatile - private var _remoteDaemon: InternalActorRef = _ - def remoteDaemon = _remoteDaemon - - @volatile - private var _eventStream: NetworkEventStream = _ - def eventStream = _eventStream - - @volatile - private var _transport: RemoteSupport[ParsedTransportAddress] = _ - def transport = _transport - - @volatile - private var _provider: RemoteActorRefProvider = _ - def provider = _provider - - def init(system: ActorSystemImpl, provider: RemoteActorRefProvider) = { - - val log = Logging(system, "Remote") - - _provider = provider - _serialization = SerializationExtension(system) - _computeGridDispatcher = system.dispatchers.lookup("akka.remote.compute-grid-dispatcher") - _remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log) - _eventStream = new NetworkEventStream(system) - _transport = { - val arguments = Seq( - classOf[ActorSystemImpl] -> system, - classOf[Remote] -> this, - classOf[RemoteSystemAddress[_ <: ParsedTransportAddress]] -> remoteAddress) - val types: Array[Class[_]] = arguments map (_._1) toArray - val values: Array[AnyRef] = arguments map (_._2) toArray - - ReflectiveAccess.createInstance[RemoteSupport[ParsedTransportAddress]](remoteSettings.RemoteTransport, types, values) match { - case Left(problem) ⇒ - - log.error(problem, "Could not load remote transport layer") - throw problem - - case Right(remote) ⇒ - - remote.start(Option(Thread.currentThread().getContextClassLoader)) //TODO Any application loader here? - - val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor { - def receive = { - case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address) - case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address) - case _ ⇒ //ignore other - } - }), "RemoteClientLifeCycleListener") - - system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent]) - system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) - - remote - } - } - - log.info("Starting remote server on [{}@{}]", system.name, remoteAddress) - } -} - -sealed trait DaemonMsg -case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg -case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg - -/** - * 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. - */ -class RemoteSystemDaemon(system: ActorSystemImpl, remote: Remote, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) - extends VirtualPathContainer(_path, _parent, _log) { - - /** - * Find the longest matching path which we know about and return that ref - * (or ask that ref to continue searching if elements are left). - */ - override def getChild(names: Iterator[String]): InternalActorRef = { - - @tailrec - def rec(s: String, n: Int): (InternalActorRef, Int) = { - getChild(s) match { - case null ⇒ - val last = s.lastIndexOf('/') - if (last == -1) (Nobody, n) - else rec(s.substring(0, last), n + 1) - case ref ⇒ (ref, n) - } - } - - val full = Vector() ++ names - rec(full.mkString("/"), 0) match { - case (Nobody, _) ⇒ Nobody - case (ref, 0) ⇒ ref - case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator) - } - } - - override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match { - case message: DaemonMsg ⇒ - log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address.hostPort) - message match { - case DaemonMsgCreate(factory, path, supervisor) ⇒ - import remote.remoteAddress - implicit val t = remote.transports - - path match { - case ParsedActorPath(`remoteAddress`, elems) if elems.nonEmpty && elems.head == "remote" ⇒ - // TODO RK canonicalize path so as not to duplicate it always #1446 - val subpath = elems.drop(1) - val path = remote.remoteDaemon.path / subpath - val actor = system.provider.actorOf(system, - Props(creator = factory), - supervisor.asInstanceOf[InternalActorRef], - path, true, None) - addChild(subpath.mkString("/"), actor) - system.deathWatch.subscribe(this, actor) - case _ ⇒ - log.error("remote path does not match path from message [{}]", message) - } - case DaemonMsgWatch(watcher, watched) ⇒ - val other = system.actorFor(watcher.path.root / "remote") - system.deathWatch.subscribe(other, watched) - } - - case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/")) - - case t: Terminated ⇒ system.deathWatch.publish(t) - - case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this) - } - -} - -class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, classLoader: Option[ClassLoader] = None) { - - def originalReceiver = input.getRecipient.getPath - - lazy val sender: ActorRef = - if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath) - else system.deadLetters - - lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) - - lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, classLoader) - - override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender -} - -trait RemoteMarshallingOps { - - def log: LoggingAdapter - - def system: ActorSystem - - def remote: Remote - - protected def useUntrustedMode: Boolean - - def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { - val arp = AkkaRemoteProtocol.newBuilder - arp.setMessage(rmp) - arp.build - } - - def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = { - val arp = AkkaRemoteProtocol.newBuilder - arp.setInstruction(rcp) - arp.build - } - - /** - * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. - */ - def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { - ActorRefProtocol.newBuilder.setPath(actor.path.toString).build - } - - def createRemoteMessageProtocolBuilder( - recipient: ActorRef, - message: Any, - senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = { - - val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient)) - messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef])) - - if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get)) - - messageBuilder - } - - def receiveMessage(remoteMessage: RemoteMessage) { - log.debug("received message {}", remoteMessage) - - val remoteDaemon = remote.remoteDaemon - - remoteMessage.recipient match { - case `remoteDaemon` ⇒ - remoteMessage.payload match { - case m @ (_: DaemonMsg | _: Terminated) ⇒ - try remoteDaemon ! m catch { - case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender) - } - case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) - } - case l: LocalRef ⇒ - remoteMessage.payload match { - case msg: SystemMessage ⇒ - if (useUntrustedMode) - throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message") - else l.sendSystemMessage(msg) - case _: AutoReceivedMessage if (useUntrustedMode) ⇒ - throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor") - case m ⇒ l.!(m)(remoteMessage.sender) - } - case r: RemoteActorRef ⇒ - implicit val t = remote.transports - remoteMessage.originalReceiver match { - case ParsedActorPath(address, _) if address == remote.remoteDaemon.path.address ⇒ - r.!(remoteMessage.payload)(remoteMessage.sender) - case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) - } - case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) - } - } -} diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 397f110783..21b7089708 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -16,6 +16,10 @@ import akka.event.EventStream import akka.dispatch.Promise import akka.config.ConfigurationException import java.util.concurrent.{ TimeoutException } +import com.typesafe.config.Config +import akka.util.ReflectiveAccess +import akka.serialization.Serialization +import akka.serialization.SerializationExtension /** * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. @@ -37,22 +41,63 @@ class RemoteActorRefProvider( val deployer = new RemoteDeployer(settings) - val remote = new Remote(settings, remoteSettings) - implicit val transports = remote.transports + val transport: RemoteTransport = { + val fqn = remoteSettings.RemoteTransport + // TODO check if this classloader is the right one + ReflectiveAccess.createInstance[RemoteTransport]( + fqn, + Seq(classOf[RemoteSettings] -> remoteSettings), + getClass.getClassLoader) match { + case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) + case Right(remote) ⇒ remote + } + } - val log = Logging(eventStream, "RemoteActorRefProvider(" + remote.remoteAddress + ")") + val log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")") - val rootPath: ActorPath = RootActorPath(remote.remoteAddress) + val rootPath: ActorPath = RootActorPath(transport.address) private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath, deployer) + val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize) + + @volatile + private var _serialization: Serialization = _ + def serialization = _serialization + + @volatile + private var _remoteDaemon: InternalActorRef = _ + def remoteDaemon = _remoteDaemon + + @volatile + private var _networkEventStream: NetworkEventStream = _ + def networkEventStream = _networkEventStream + val deathWatch = new RemoteDeathWatch(local.deathWatch, this) def init(system: ActorSystemImpl) { local.init(system) - remote.init(system, this) - local.registerExtraNames(Map(("remote", remote.remoteDaemon))) - terminationFuture.onComplete(_ ⇒ remote.transport.shutdown()) + + _remoteDaemon = new RemoteSystemDaemon(system, transport.address, rootPath / "remote", rootGuardian, log) + _serialization = SerializationExtension(system) + + transport.start(system, this) + + val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor { + def receive = { + case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address) + case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address) + case _ ⇒ //ignore other + } + }), "RemoteClientLifeCycleListener") + + _networkEventStream = new NetworkEventStream(system) + + system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent]) + system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) + + local.registerExtraNames(Map(("remote", remoteDaemon))) + terminationFuture.onComplete(_ ⇒ transport.shutdown()) } def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy]): InternalActorRef = { @@ -102,18 +147,12 @@ class RemoteActorRefProvider( }) deployment match { - case Some(Deploy(_, _, _, RemoteScope(address))) ⇒ - // FIXME RK this should be done within the deployer, i.e. the whole parsing business - address.parse(remote.transports) match { - case Left(x) ⇒ - throw new ConfigurationException("cannot parse remote address: " + x) - case Right(addr) ⇒ - if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment) - else { - val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements - useActorOnNode(rpath, props.creator, supervisor) - new RemoteActorRef(this, remote.transport, rpath, supervisor, None) - } + case Some(Deploy(_, _, _, RemoteScope(addr))) ⇒ + if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment) + else { + val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements + useActorOnNode(rpath, props.creator, supervisor) + new RemoteActorRef(this, transport, rpath, supervisor, None) } case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment) @@ -123,14 +162,13 @@ class RemoteActorRefProvider( def actorFor(path: ActorPath): InternalActorRef = path.root match { case `rootPath` ⇒ actorFor(rootGuardian, path.elements) - case RootActorPath(_: RemoteSystemAddress[_], _) ⇒ new RemoteActorRef(this, remote.transport, path, Nobody, None) - case _ ⇒ local.actorFor(path) + case _ ⇒ new RemoteActorRef(this, transport, path, Nobody, None) } def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match { - case ParsedActorPath(address, elems) ⇒ + case ActorPathExtractor(address, elems) ⇒ if (address == rootPath.address) actorFor(rootGuardian, elems) - else new RemoteActorRef(this, remote.transport, new RootActorPath(address) / elems, Nobody, None) + else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody, None) case _ ⇒ local.actorFor(ref, path) } @@ -159,7 +197,7 @@ trait RemoteRef extends ActorRefScope { */ private[akka] class RemoteActorRef private[akka] ( provider: RemoteActorRefProvider, - remote: RemoteSupport[ParsedTransportAddress], + remote: RemoteTransport, val path: ActorPath, val getParent: InternalActorRef, loader: Option[ClassLoader]) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteAddress.scala b/akka-remote/src/main/scala/akka/remote/RemoteAddress.scala new file mode 100644 index 0000000000..f7274c2356 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/RemoteAddress.scala @@ -0,0 +1,5 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.remote + diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index 404bf98c4c..fc65734822 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -19,14 +19,14 @@ import java.util.concurrent.atomic.AtomicReference */ class RemoteConnectionManager( system: ActorSystemImpl, - remote: Remote, - initialConnections: Map[ParsedTransportAddress, ActorRef] = Map.empty[ParsedTransportAddress, ActorRef]) + remote: RemoteActorRefProvider, + initialConnections: Map[Address, ActorRef] = Map.empty[Address, ActorRef]) extends ConnectionManager { val log = Logging(system, "RemoteConnectionManager") // FIXME is this VersionedIterable really needed? It is not used I think. Complicates API. See 'def connections' etc. - case class State(version: Long, connections: Map[ParsedTransportAddress, ActorRef]) + case class State(version: Long, connections: Map[Address, ActorRef]) extends VersionedIterable[ActorRef] { def iterable: Iterable[ActorRef] = connections.values } @@ -52,7 +52,7 @@ class RemoteConnectionManager( def size: Int = connections.connections.size - def connectionFor(address: ParsedTransportAddress): Option[ActorRef] = connections.connections.get(address) + def connectionFor(address: Address): Option[ActorRef] = connections.connections.get(address) def isEmpty: Boolean = connections.connections.isEmpty @@ -61,7 +61,7 @@ class RemoteConnectionManager( } @tailrec - final def failOver(from: ParsedTransportAddress, to: ParsedTransportAddress) { + final def failOver(from: Address, to: Address) { log.debug("Failing over connection from [{}] to [{}]", from, to) val oldState = state.get @@ -92,8 +92,8 @@ class RemoteConnectionManager( val oldState = state.get() var changed = false - var faultyAddress: ParsedTransportAddress = null - var newConnections = Map.empty[ParsedTransportAddress, ActorRef] + var faultyAddress: Address = null + var newConnections = Map.empty[Address, ActorRef] oldState.connections.keys foreach { address ⇒ val actorRef: ActorRef = oldState.connections.get(address).get @@ -119,7 +119,7 @@ class RemoteConnectionManager( } @tailrec - final def putIfAbsent(address: ParsedTransportAddress, newConnectionFactory: () ⇒ ActorRef): ActorRef = { + final def putIfAbsent(address: Address, newConnectionFactory: () ⇒ ActorRef): ActorRef = { val oldState = state.get() val oldConnections = oldState.connections @@ -146,6 +146,6 @@ class RemoteConnectionManager( } } - private[remote] def newConnection(remoteAddress: ParsedTransportAddress, actorPath: ActorPath) = - new RemoteActorRef(remote.provider, remote.transport, actorPath, Nobody, None) + private[remote] def newConnection(remoteAddress: Address, actorPath: ActorPath) = + new RemoteActorRef(remote, remote.transport, actorPath, Nobody, None) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala new file mode 100644 index 0000000000..81088b5000 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -0,0 +1,80 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package akka.remote + +import scala.annotation.tailrec + +import akka.actor.{ VirtualPathContainer, Terminated, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor } +import akka.event.LoggingAdapter + +sealed trait DaemonMsg +case class DaemonMsgCreate(factory: () ⇒ Actor, path: String, supervisor: ActorRef) extends DaemonMsg +case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMsg + +/** + * 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. + */ +class RemoteSystemDaemon(system: ActorSystemImpl, address: Address, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) + extends VirtualPathContainer(_path, _parent, _log) { + + /** + * Find the longest matching path which we know about and return that ref + * (or ask that ref to continue searching if elements are left). + */ + override def getChild(names: Iterator[String]): InternalActorRef = { + + @tailrec + def rec(s: String, n: Int): (InternalActorRef, Int) = { + getChild(s) match { + case null ⇒ + val last = s.lastIndexOf('/') + if (last == -1) (Nobody, n) + else rec(s.substring(0, last), n + 1) + case ref ⇒ (ref, n) + } + } + + val full = Vector() ++ names + rec(full.mkString("/"), 0) match { + case (Nobody, _) ⇒ Nobody + case (ref, 0) ⇒ ref + case (ref, n) ⇒ ref.getChild(full.takeRight(n).iterator) + } + } + + override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match { + case message: DaemonMsg ⇒ + log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, path.address) + message match { + case DaemonMsgCreate(factory, path, supervisor) ⇒ + path match { + case ActorPathExtractor(`address`, elems) if elems.nonEmpty && elems.head == "remote" ⇒ + // TODO RK canonicalize path so as not to duplicate it always #1446 + val subpath = elems.drop(1) + val path = this.path / subpath + val actor = system.provider.actorOf(system, + Props(creator = factory), + supervisor.asInstanceOf[InternalActorRef], + path, true, None) + addChild(subpath.mkString("/"), actor) + system.deathWatch.subscribe(this, actor) + case _ ⇒ + log.error("remote path does not match path from message [{}]", message) + } + case DaemonMsgWatch(watcher, watched) ⇒ + val other = system.actorFor(watcher.path.root / "remote") + system.deathWatch.subscribe(other, watched) + } + + case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/")) + + case t: Terminated ⇒ system.deathWatch.publish(t) + + case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this) + } + +} diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index de3e0825ff..63eaae0632 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -8,7 +8,7 @@ import akka.routing._ import com.typesafe.config._ import akka.config.ConfigurationException -case class RemoteScope(node: UnparsedSystemAddress[UnparsedTransportAddress]) extends Scope +case class RemoteScope(node: Address) extends Scope class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings) { @@ -18,7 +18,7 @@ class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings super.parseConfig(path, config) match { case d @ Some(deploy) ⇒ deploy.config.getString("remote") match { - case RemoteAddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r))) + case AddressExtractor(r) ⇒ Some(deploy.copy(scope = RemoteScope(r))) case str ⇒ if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str) val nodes = deploy.config.getStringList("target.nodes").asScala diff --git a/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala b/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala deleted file mode 100644 index a5474ce427..0000000000 --- a/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala +++ /dev/null @@ -1,343 +0,0 @@ -/** - * Copyright (C) 2009-2010 Typesafe Inc. - */ - -package akka.remote - -import akka.actor._ -import akka.AkkaException -import scala.reflect.BeanProperty -import java.io.{ PrintWriter, PrintStream } -import java.net.InetSocketAddress -import java.net.URI -import java.net.URISyntaxException -import java.net.InetAddress -import java.net.UnknownHostException -import java.net.UnknownServiceException -import akka.event.Logging - -/** - * Interface for remote transports to encode their addresses. The three parts - * are named according to the URI spec (precisely java.net.URI) which is used - * for parsing. That means that the address’ parts must conform to what an - * URI expects, but otherwise each transport may assign a different meaning - * to these parts. - */ -trait RemoteTransportAddress { - def protocol: String - def host: String - def port: Int -} - -trait ParsedTransportAddress extends RemoteTransportAddress - -case class RemoteNettyAddress(host: String, ip: Option[InetAddress], port: Int) extends ParsedTransportAddress { - def protocol = "akka" - - override def toString(): String = "akka://" + host + ":" + port -} - -object RemoteNettyAddress { - def apply(host: String, port: Int): RemoteNettyAddress = { - // TODO ticket #1639 - val ip = try Some(InetAddress.getByName(host)) catch { case _: UnknownHostException ⇒ None } - new RemoteNettyAddress(host, ip, port) - } - def apply(s: String): RemoteNettyAddress = { - val RE = """([^:]+):(\d+)""".r - s match { - case RE(h, p) ⇒ apply(h, Integer.parseInt(p)) - case _ ⇒ throw new IllegalArgumentException("cannot parse " + s + " as ") - } - } -} - -case class UnparsedTransportAddress(protocol: String, host: String, port: Int) extends RemoteTransportAddress { - def parse(transports: TransportsMap): RemoteTransportAddress = - transports.get(protocol) - .map(_(host, port)) - .toRight("protocol " + protocol + " not known") - .joinRight.fold(UnparseableTransportAddress(protocol, host, port, _), identity) -} - -case class UnparseableTransportAddress(protocol: String, host: String, port: Int, error: String) extends RemoteTransportAddress - -case class RemoteSystemAddress[+T <: ParsedTransportAddress](system: String, transport: T) extends Address { - def protocol = transport.protocol - @transient - lazy val hostPort = system + "@" + transport.host + ":" + transport.port -} - -case class UnparsedSystemAddress[+T <: RemoteTransportAddress](system: Option[String], transport: T) { - def parse(transports: TransportsMap): Either[UnparsedSystemAddress[UnparseableTransportAddress], RemoteSystemAddress[ParsedTransportAddress]] = - system match { - case Some(sys) ⇒ - transport match { - case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x)) - case y: UnparsedTransportAddress ⇒ - y.parse(transports) match { - case x: ParsedTransportAddress ⇒ Right(RemoteSystemAddress(sys, x)) - case y: UnparseableTransportAddress ⇒ Left(UnparsedSystemAddress(system, y)) - case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z))) - } - case z ⇒ Left(UnparsedSystemAddress(system, UnparseableTransportAddress(z.protocol, z.host, z.port, "cannot parse " + z))) - } - case None ⇒ Left(UnparsedSystemAddress(None, UnparseableTransportAddress(transport.protocol, transport.host, transport.port, "no system name specified"))) - } -} - -object RemoteAddressExtractor { - def unapply(s: String): Option[UnparsedSystemAddress[UnparsedTransportAddress]] = { - try { - val uri = new URI(s) - if (uri.getScheme == null || uri.getHost == null || uri.getPort == -1) None - else Some(UnparsedSystemAddress(Option(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort))) - } catch { - case _: URISyntaxException ⇒ None - } - } -} - -object RemoteActorPath { - def unapply(addr: String): Option[(UnparsedSystemAddress[UnparsedTransportAddress], Iterable[String])] = { - try { - val uri = new URI(addr) - if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None - else Some(UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)), - ActorPath.split(uri.getPath).drop(1)) - } catch { - case _: URISyntaxException ⇒ None - } - } -} - -object ParsedActorPath { - def unapply(addr: String)(implicit transports: TransportsMap): Option[(RemoteSystemAddress[ParsedTransportAddress], Iterable[String])] = { - try { - val uri = new URI(addr) - if (uri.getScheme == null || uri.getUserInfo == null || uri.getHost == null || uri.getPort == -1 || uri.getPath == null) None - else - UnparsedSystemAddress(Some(uri.getUserInfo), UnparsedTransportAddress(uri.getScheme, uri.getHost, uri.getPort)).parse(transports) match { - case Left(_) ⇒ None - case Right(x) ⇒ Some(x, ActorPath.split(uri.getPath).drop(1)) - } - } catch { - case _: URISyntaxException ⇒ None - } - } -} - -class RemoteException(message: String) extends AkkaException(message) - -trait RemoteModule { - protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit -} - -/** - * Remote life-cycle events. - */ -sealed trait RemoteLifeCycleEvent { - def logLevel: Logging.LogLevel -} - -/** - * Life-cycle events for RemoteClient. - */ -trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent { - def remoteAddress: ParsedTransportAddress -} - -case class RemoteClientError[T <: ParsedTransportAddress]( - @BeanProperty cause: Throwable, - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.ErrorLevel - override def toString = - "RemoteClientError@" + - remoteAddress + - ": Error[" + - (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + - "]" -} - -case class RemoteClientDisconnected[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteClientDisconnected@" + remoteAddress -} - -case class RemoteClientConnected[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteClientConnected@" + remoteAddress -} - -case class RemoteClientStarted[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteClientStarted@" + remoteAddress -} - -case class RemoteClientShutdown[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteClientShutdown@" + remoteAddress -} - -case class RemoteClientWriteFailed[T <: ParsedTransportAddress]( - @BeanProperty request: AnyRef, - @BeanProperty cause: Throwable, - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: T) extends RemoteClientLifeCycleEvent { - override def logLevel = Logging.WarningLevel - override def toString = - "RemoteClientWriteFailed@" + - remoteAddress + - ": MessageClass[" + - (if (request ne null) request.getClass.getName else "no message") + - "] Error[" + - (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + - "]" -} - -/** - * Life-cycle events for RemoteServer. - */ -trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent - -case class RemoteServerStarted[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteServerStarted@" + remote.name -} - -case class RemoteServerShutdown[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.InfoLevel - override def toString = - "RemoteServerShutdown@" + remote.name -} - -case class RemoteServerError[T <: ParsedTransportAddress]( - @BeanProperty val cause: Throwable, - @BeanProperty remote: RemoteSupport[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.ErrorLevel - override def toString = - "RemoteServerError@" + - remote.name + - ": Error[" + - (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + - "]" -} - -case class RemoteServerClientConnected[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientConnected@" + - remote.name + - ": Client[" + - (if (clientAddress.isDefined) clientAddress.get else "no address") + - "]" -} - -case class RemoteServerClientDisconnected[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientDisconnected@" + - remote.name + - ": Client[" + - (if (clientAddress.isDefined) clientAddress.get else "no address") + - "]" -} - -case class RemoteServerClientClosed[T <: ParsedTransportAddress]( - @BeanProperty remote: RemoteSupport[T], - @BeanProperty val clientAddress: Option[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.DebugLevel - override def toString = - "RemoteServerClientClosed@" + - remote.name + - ": Client[" + - (if (clientAddress.isDefined) clientAddress.get else "no address") + - "]" -} - -case class RemoteServerWriteFailed[T <: ParsedTransportAddress]( - @BeanProperty request: AnyRef, - @BeanProperty cause: Throwable, - @BeanProperty remote: RemoteSupport[T], - @BeanProperty remoteAddress: Option[T]) extends RemoteServerLifeCycleEvent { - override def logLevel = Logging.WarningLevel - override def toString = - "RemoteServerWriteFailed@" + - remote + - ": ClientAddress[" + - remoteAddress + - "] MessageClass[" + - (if (request ne null) request.getClass.getName else "no message") + - "] Error[" + - (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + - "]" -} - -/** - * Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down. - */ -class RemoteClientException[T <: ParsedTransportAddress] private[akka] ( - message: String, - @BeanProperty val client: RemoteSupport[T], - val remoteAddress: T, cause: Throwable = null) extends AkkaException(message, cause) - -abstract class RemoteSupport[-T <: ParsedTransportAddress](val system: ActorSystemImpl) { - /** - * Shuts down the remoting - */ - def shutdown(): Unit - - /** - * Gets the name of the server instance - */ - def name: String - - /** - * Starts up the remoting - */ - def start(loader: Option[ClassLoader]): Unit - - /** - * Shuts down a specific client connected to the supplied remote address returns true if successful - */ - def shutdownClientConnection(address: T): Boolean - - /** - * Restarts a specific client connected to the supplied remote address, but only if the client is not shut down - */ - def restartClientConnection(address: T): Boolean - - /** Methods that needs to be implemented by a transport **/ - - protected[akka] def send(message: Any, - senderOption: Option[ActorRef], - recipient: RemoteActorRef, - loader: Option[ClassLoader]): Unit - - protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = { - system.eventStream.publish(message) - system.log.log(message.logLevel, "REMOTE: {}", message) - } - - override def toString = name -} diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index eae0741844..b2e09b5e13 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -10,85 +10,28 @@ import java.net.InetAddress import akka.config.ConfigurationException import com.eaio.uuid.UUID import scala.collection.JavaConverters._ +import akka.actor.Address +import akka.actor.AddressExtractor class RemoteSettings(val config: Config, val systemName: String) { import config._ val RemoteTransport = getString("akka.remote.transport") - val Daemonic = getBoolean("akka.remote.daemonic") + + // AccrualFailureDetector val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold") val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size") - val ShouldCompressData = getBoolean("akka.remote.use-compression") + + // Gossiper val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS) - val InitalDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS) + val InitialDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS) val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS) - val BackoffTimeout = Duration(getMilliseconds("akka.remote.backoff-timeout"), MILLISECONDS) - // TODO cluster config will go into akka-cluster/reference.conf when we enable that module - val SeedNodes = Set.empty[RemoteNettyAddress] ++ getStringList("akka.cluster.seed-nodes").asScala.collect { - case RemoteAddressExtractor(addr) ⇒ addr.transport + val SeedNodes = Set.empty[Address] ++ getStringList("akka.cluster.seed-nodes").asScala.collect { + case AddressExtractor(addr) ⇒ addr } - val serverSettings = new RemoteServerSettings - val clientSettings = new RemoteClientSettings + val UntrustedMode = getBoolean("akka.remote.untrusted-mode") - class RemoteClientSettings { - val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match { - case "" ⇒ None - case cookie ⇒ Some(cookie) - } - - val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS) - val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS) - val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS) - val MessageFrameSize = getBytes("akka.remote.client.message-frame-size").toInt - } - - class RemoteServerSettings { - import scala.collection.JavaConverters._ - val MessageFrameSize = getBytes("akka.remote.server.message-frame-size").toInt - val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match { - case "" ⇒ None - case cookie ⇒ Some(cookie) - } - val RequireCookie = { - val requireCookie = getBoolean("akka.remote.server.require-cookie") - if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException( - "Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.") - requireCookie - } - - val UsePassiveConnections = getBoolean("akka.remote.use-passive-connections") - - val UntrustedMode = getBoolean("akka.remote.server.untrusted-mode") - val Hostname = getString("akka.remote.server.hostname") match { - case "" ⇒ InetAddress.getLocalHost.getHostAddress - case value ⇒ value - } - val Port = getInt("akka.remote.server.port") - val ConnectionTimeout = Duration(getMilliseconds("akka.remote.server.connection-timeout"), MILLISECONDS) - - val Backlog = getInt("akka.remote.server.backlog") - - val ExecutionPoolKeepAlive = Duration(getMilliseconds("akka.remote.server.execution-pool-keepalive"), MILLISECONDS) - - val ExecutionPoolSize = getInt("akka.remote.server.execution-pool-size") match { - case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.server.execution-pool-size is less than 1") - case sz ⇒ sz - } - - val MaxChannelMemorySize = getBytes("akka.remote.server.max-channel-memory-size") match { - case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-channel-memory-size is less than 0 bytes") - case sz ⇒ sz - } - - val MaxTotalMemorySize = getBytes("akka.remote.server.max-total-memory-size") match { - case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-total-memory-size is less than 0 bytes") - case sz ⇒ sz - } - - // TODO handle the system name right and move this to config file syntax - val URI = "akka://sys@" + Hostname + ":" + Port - } } \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala new file mode 100644 index 0000000000..19fd6c1c43 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -0,0 +1,321 @@ +/** + * Copyright (C) 2009-2010 Typesafe Inc. + */ + +package akka.remote + +import scala.reflect.BeanProperty + +import akka.actor.{ Terminated, LocalRef, InternalActorRef, AutoReceivedMessage, AddressExtractor, Address, ActorSystemImpl, ActorSystem, ActorRef } +import akka.dispatch.SystemMessage +import akka.event.{ LoggingAdapter, Logging } +import akka.remote.RemoteProtocol.{ RemoteMessageProtocol, RemoteControlProtocol, AkkaRemoteProtocol, ActorRefProtocol } +import akka.AkkaException + +/** + * Remote life-cycle events. + */ +sealed trait RemoteLifeCycleEvent { + def logLevel: Logging.LogLevel +} + +/** + * Life-cycle events for RemoteClient. + */ +trait RemoteClientLifeCycleEvent extends RemoteLifeCycleEvent { + def remoteAddress: Address +} + +case class RemoteClientError( + @BeanProperty cause: Throwable, + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.ErrorLevel + override def toString = + "RemoteClientError@" + + remoteAddress + + ": Error[" + + (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + + "]" +} + +case class RemoteClientDisconnected( + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.DebugLevel + override def toString = + "RemoteClientDisconnected@" + remoteAddress +} + +case class RemoteClientConnected( + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.DebugLevel + override def toString = + "RemoteClientConnected@" + remoteAddress +} + +case class RemoteClientStarted( + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.InfoLevel + override def toString = + "RemoteClientStarted@" + remoteAddress +} + +case class RemoteClientShutdown( + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.InfoLevel + override def toString = + "RemoteClientShutdown@" + remoteAddress +} + +case class RemoteClientWriteFailed( + @BeanProperty request: AnyRef, + @BeanProperty cause: Throwable, + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Address) extends RemoteClientLifeCycleEvent { + override def logLevel = Logging.WarningLevel + override def toString = + "RemoteClientWriteFailed@" + + remoteAddress + + ": MessageClass[" + + (if (request ne null) request.getClass.getName else "no message") + + "] Error[" + + (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + + "]" +} + +/** + * Life-cycle events for RemoteServer. + */ +trait RemoteServerLifeCycleEvent extends RemoteLifeCycleEvent + +case class RemoteServerStarted( + @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.InfoLevel + override def toString = + "RemoteServerStarted@" + remote +} + +case class RemoteServerShutdown( + @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.InfoLevel + override def toString = + "RemoteServerShutdown@" + remote +} + +case class RemoteServerError( + @BeanProperty val cause: Throwable, + @BeanProperty remote: RemoteTransport) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.ErrorLevel + override def toString = + "RemoteServerError@" + + remote + + ": Error[" + + (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + + "]" +} + +case class RemoteServerClientConnected( + @BeanProperty remote: RemoteTransport, + @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.DebugLevel + override def toString = + "RemoteServerClientConnected@" + + remote + + ": Client[" + + (if (clientAddress.isDefined) clientAddress.get else "no address") + + "]" +} + +case class RemoteServerClientDisconnected( + @BeanProperty remote: RemoteTransport, + @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.DebugLevel + override def toString = + "RemoteServerClientDisconnected@" + + remote + + ": Client[" + + (if (clientAddress.isDefined) clientAddress.get else "no address") + + "]" +} + +case class RemoteServerClientClosed( + @BeanProperty remote: RemoteTransport, + @BeanProperty val clientAddress: Option[Address]) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.DebugLevel + override def toString = + "RemoteServerClientClosed@" + + remote + + ": Client[" + + (if (clientAddress.isDefined) clientAddress.get else "no address") + + "]" +} + +case class RemoteServerWriteFailed( + @BeanProperty request: AnyRef, + @BeanProperty cause: Throwable, + @BeanProperty remote: RemoteTransport, + @BeanProperty remoteAddress: Option[Address]) extends RemoteServerLifeCycleEvent { + override def logLevel = Logging.WarningLevel + override def toString = + "RemoteServerWriteFailed@" + + remote + + ": ClientAddress[" + + remoteAddress + + "] MessageClass[" + + (if (request ne null) request.getClass.getName else "no message") + + "] Error[" + + (if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") + + "]" +} + +/** + * Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down. + */ +class RemoteClientException private[akka] ( + message: String, + @BeanProperty val client: RemoteTransport, + val remoteAddress: Address, cause: Throwable = null) extends AkkaException(message, cause) + +class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause) + +abstract class RemoteTransport { + /** + * Shuts down the remoting + */ + def shutdown(): Unit + + /** + * Address to be used in RootActorPath of refs generated for this transport. + */ + def address: Address + + /** + * The actor system, for which this transport is instantiated. Will publish to its eventStream. + */ + def system: ActorSystem + + /** + * Starts up the remoting + */ + def start(system: ActorSystemImpl, provider: RemoteActorRefProvider): Unit + + /** + * Shuts down a specific client connected to the supplied remote address returns true if successful + */ + def shutdownClientConnection(address: Address): Boolean + + /** + * Restarts a specific client connected to the supplied remote address, but only if the client is not shut down + */ + def restartClientConnection(address: Address): Boolean + + /** Methods that needs to be implemented by a transport **/ + + protected[akka] def send(message: Any, + senderOption: Option[ActorRef], + recipient: RemoteActorRef, + loader: Option[ClassLoader]): Unit + + protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = { + system.eventStream.publish(message) + system.log.log(message.logLevel, "REMOTE: {}", message) + } + + override def toString = address.toString +} + +class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, classLoader: Option[ClassLoader]) { + + def originalReceiver = input.getRecipient.getPath + + lazy val sender: ActorRef = + if (input.hasSender) system.provider.actorFor(system.provider.rootGuardian, input.getSender.getPath) + else system.deadLetters + + lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) + + lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, classLoader) + + override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender +} + +trait RemoteMarshallingOps { + + def log: LoggingAdapter + + def system: ActorSystemImpl + + def provider: RemoteActorRefProvider + + protected def useUntrustedMode: Boolean + + def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { + val arp = AkkaRemoteProtocol.newBuilder + arp.setMessage(rmp) + arp.build + } + + def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = { + val arp = AkkaRemoteProtocol.newBuilder + arp.setInstruction(rcp) + arp.build + } + + /** + * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. + */ + def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { + ActorRefProtocol.newBuilder.setPath(actor.path.toString).build + } + + def createRemoteMessageProtocolBuilder( + recipient: ActorRef, + message: Any, + senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = { + + val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient)) + messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef])) + + if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get)) + + messageBuilder + } + + def receiveMessage(remoteMessage: RemoteMessage) { + log.debug("received message {}", remoteMessage) + + val remoteDaemon = provider.remoteDaemon + + remoteMessage.recipient match { + case `remoteDaemon` ⇒ + remoteMessage.payload match { + case m @ (_: DaemonMsg | _: Terminated) ⇒ + try remoteDaemon ! m catch { + case e: Exception ⇒ log.error(e, "exception while processing remote command {} from {}", m, remoteMessage.sender) + } + case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) + } + case l: LocalRef ⇒ + remoteMessage.payload match { + case msg: SystemMessage ⇒ + if (useUntrustedMode) + throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message") + else l.sendSystemMessage(msg) + case _: AutoReceivedMessage if (useUntrustedMode) ⇒ + throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor") + case m ⇒ l.!(m)(remoteMessage.sender) + } + case r: RemoteActorRef ⇒ + remoteMessage.originalReceiver match { + case AddressExtractor(address) if address == provider.transport.address ⇒ + r.!(remoteMessage.payload)(remoteMessage.sender) + case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) + } + case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) + } + } +} diff --git a/akka-remote/src/main/scala/akka/remote/netty/Client.scala b/akka-remote/src/main/scala/akka/remote/netty/Client.scala new file mode 100644 index 0000000000..4a9d6607fd --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -0,0 +1,347 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.netty + +import java.net.InetSocketAddress +import org.jboss.netty.util.HashedWheelTimer +import org.jboss.netty.bootstrap.ClientBootstrap +import org.jboss.netty.channel.group.DefaultChannelGroup +import org.jboss.netty.channel.{ ChannelHandler, StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelFuture, Channel } +import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } +import org.jboss.netty.handler.execution.ExecutionHandler +import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException } +import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } +import akka.remote.{ RemoteProtocol, RemoteMessage, RemoteLifeCycleEvent, RemoteClientStarted, RemoteClientShutdown, RemoteClientException, RemoteClientError, RemoteClientDisconnected, RemoteClientConnected } +import akka.actor.{ simpleName, Address } +import akka.AkkaException +import akka.event.Logging +import akka.util.Switch +import akka.actor.ActorRef +import org.jboss.netty.channel.ChannelFutureListener +import akka.remote.RemoteClientWriteFailed +import java.net.InetAddress +import org.jboss.netty.util.TimerTask +import org.jboss.netty.util.Timeout +import java.util.concurrent.TimeUnit + +class RemoteClientMessageBufferException(message: String, cause: Throwable) extends AkkaException(message, cause) { + def this(msg: String) = this(msg, null) +} + +/** + * This is the abstract baseclass for netty remote clients, currently there's only an + * ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that + * reuses an already established connection. + */ +abstract class RemoteClient private[akka] ( + val netty: NettyRemoteTransport, + val remoteAddress: Address) { + + val log = Logging(netty.system, "RemoteClient") + + val name = simpleName(this) + "@" + remoteAddress + + private[remote] val runSwitch = new Switch() + + private[remote] def isRunning = runSwitch.isOn + + protected def currentChannel: Channel + + def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean + + def shutdown(): Boolean + + def isBoundTo(address: Address): Boolean = remoteAddress == address + + /** + * Converts the message to the wireprotocol and sends the message across the wire + */ + def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { + log.debug("Sending message {} from {} to {}", message, senderOption, recipient) + send((message, senderOption, recipient)) + } else { + val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", netty, remoteAddress) + netty.notifyListeners(RemoteClientError(exception, netty, remoteAddress)) + throw exception + } + + /** + * Sends the message across the wire + */ + private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = { + try { + val channel = currentChannel + val f = channel.write(request) + f.addListener( + new ChannelFutureListener { + def operationComplete(future: ChannelFuture) { + if (future.isCancelled || !future.isSuccess) { + netty.notifyListeners(RemoteClientWriteFailed(request, future.getCause, netty, remoteAddress)) + } + } + }) + // Check if we should back off + if (!channel.isWritable) { + val backoff = netty.settings.BackoffTimeout + if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off + } + } catch { + case e: Exception ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress)) + } + } + + override def toString = name +} + +/** + * RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node. + */ +class ActiveRemoteClient private[akka] ( + netty: NettyRemoteTransport, + remoteAddress: Address, + localAddress: Address, + val loader: Option[ClassLoader] = None) + extends RemoteClient(netty, remoteAddress) { + + import netty.settings + + //TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation) + @volatile + private var bootstrap: ClientBootstrap = _ + @volatile + private var connection: ChannelFuture = _ + @volatile + private[remote] var openChannels: DefaultChannelGroup = _ + @volatile + private var executionHandler: ExecutionHandler = _ + + @volatile + private var reconnectionTimeWindowStart = 0L + + def notifyListeners(msg: RemoteLifeCycleEvent): Unit = netty.notifyListeners(msg) + + def currentChannel = connection.getChannel + + /** + * Connect to remote server. + */ + def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = { + + def sendSecureCookie(connection: ChannelFuture) { + val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) + if (settings.SecureCookie.nonEmpty) handshake.setCookie(settings.SecureCookie.get) + handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder + .setSystem(localAddress.system) + .setHostname(localAddress.host.get) + .setPort(localAddress.port.get) + .build) + connection.getChannel.write(netty.createControlEnvelope(handshake.build)) + } + + def attemptReconnect(): Boolean = { + val remoteIP = InetAddress.getByName(remoteAddress.host.get) + log.debug("Remote client reconnecting to [{}|{}]", remoteAddress, remoteIP) + connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get)) + openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. + + if (!connection.isSuccess) { + notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress)) + false + } else { + sendSecureCookie(connection) + true + } + } + + runSwitch switchOn { + openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName) + + executionHandler = new ExecutionHandler(netty.executor) + + bootstrap = new ClientBootstrap(netty.clientChannelFactory) + bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this)) + bootstrap.setOption("tcpNoDelay", true) + bootstrap.setOption("keepAlive", true) + bootstrap.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis) + + val remoteIP = InetAddress.getByName(remoteAddress.host.get) + log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP) + + connection = bootstrap.connect(new InetSocketAddress(remoteIP, remoteAddress.port.get)) + + openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. + + if (!connection.isSuccess) { + notifyListeners(RemoteClientError(connection.getCause, netty, remoteAddress)) + false + } else { + sendSecureCookie(connection) + notifyListeners(RemoteClientStarted(netty, remoteAddress)) + true + } + } match { + case true ⇒ true + case false if reconnectIfAlreadyConnected ⇒ + connection.getChannel.close() + openChannels.remove(connection.getChannel) + + log.debug("Remote client reconnecting to [{}]", remoteAddress) + attemptReconnect() + + case false ⇒ false + } + } + + // Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients + def shutdown() = runSwitch switchOff { + log.debug("Shutting down remote client [{}]", name) + + notifyListeners(RemoteClientShutdown(netty, remoteAddress)) + try { + if ((connection ne null) && (connection.getChannel ne null)) + connection.getChannel.close() + } finally { + try { + if (openChannels ne null) openChannels.close.awaitUninterruptibly() + } finally { + connection = null + executionHandler = null + } + } + + log.debug("[{}] has been shut down", name) + } + + private[akka] def isWithinReconnectionTimeWindow: Boolean = { + if (reconnectionTimeWindowStart == 0L) { + reconnectionTimeWindowStart = System.currentTimeMillis + true + } else { + val timeLeft = (settings.ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0 + if (timeLeft) + log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft) + + timeLeft + } + } + + private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L +} + +@ChannelHandler.Sharable +class ActiveRemoteClientHandler( + val name: String, + val bootstrap: ClientBootstrap, + val remoteAddress: Address, + val timer: HashedWheelTimer, + val client: ActiveRemoteClient) + extends SimpleChannelUpstreamHandler { + + def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() { + def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() } + }, 0, TimeUnit.MILLISECONDS) + + override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) { + try { + event.getMessage match { + case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒ + val rcp = arp.getInstruction + rcp.getCommandType match { + case CommandType.SHUTDOWN ⇒ runOnceNow { client.netty.shutdownClientConnection(remoteAddress) } + case _ ⇒ //Ignore others + } + + case arp: AkkaRemoteProtocol if arp.hasMessage ⇒ + client.netty.receiveMessage(new RemoteMessage(arp.getMessage, client.netty.system, client.loader)) + + case other ⇒ + throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.netty, client.remoteAddress) + } + } catch { + case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress)) + } + } + + override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn { + if (client.isWithinReconnectionTimeWindow) { + timer.newTimeout(new TimerTask() { + def run(timeout: Timeout) = + if (client.isRunning) { + client.openChannels.remove(event.getChannel) + client.connect(reconnectIfAlreadyConnected = true) + } + }, client.netty.settings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS) + } else runOnceNow { + client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread + } + } + + override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + try { + client.notifyListeners(RemoteClientConnected(client.netty, client.remoteAddress)) + client.resetReconnectionTimeWindow + } catch { + case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress)) + } + } + + override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + client.notifyListeners(RemoteClientDisconnected(client.netty, client.remoteAddress)) + } + + override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { + val cause = event.getCause + if (cause ne null) { + client.notifyListeners(RemoteClientError(cause, client.netty, client.remoteAddress)) + cause match { + case e: ReadTimeoutException ⇒ + runOnceNow { + client.netty.shutdownClientConnection(remoteAddress) // spawn in another thread + } + case e: Exception ⇒ event.getChannel.close() + } + + } else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.netty, client.remoteAddress)) + } +} + +class ActiveRemoteClientPipelineFactory( + name: String, + bootstrap: ClientBootstrap, + executionHandler: ExecutionHandler, + remoteAddress: Address, + client: ActiveRemoteClient) extends ChannelPipelineFactory { + + import client.netty.settings + + def getPipeline: ChannelPipeline = { + val timeout = new ReadTimeoutHandler(client.netty.timer, settings.ReadTimeout.length, settings.ReadTimeout.unit) + 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, client.netty.timer, client) + + new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient) + } +} + +class PassiveRemoteClient(val currentChannel: Channel, + netty: NettyRemoteTransport, + remoteAddress: Address) + extends RemoteClient(netty, remoteAddress) { + + def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn { + netty.notifyListeners(RemoteClientStarted(netty, remoteAddress)) + log.debug("Starting remote client connection to [{}]", remoteAddress) + } + + def shutdown() = runSwitch switchOff { + log.debug("Shutting down remote client [{}]", name) + + netty.notifyListeners(RemoteClientShutdown(netty, remoteAddress)) + log.debug("[{}] has been shut down", name) + } +} + diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index db58b6b23d..4226f9dffc 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,394 +4,114 @@ package akka.remote.netty -import akka.actor.{ ActorRef, IllegalActorStateException, simpleName } -import akka.remote._ -import RemoteProtocol._ -import akka.util._ -import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture } -import org.jboss.netty.channel.socket.nio.{ NioServerSocketChannelFactory, NioClientSocketChannelFactory } -import org.jboss.netty.bootstrap.{ ServerBootstrap, ClientBootstrap } -import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender } -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 java.net.{ UnknownHostException, InetAddress } +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.locks.ReentrantReadWriteLock +import java.util.concurrent.Executors import scala.collection.mutable.HashMap -import java.net.InetSocketAddress -import java.util.concurrent.atomic._ -import akka.AkkaException -import akka.event.Logging -import org.jboss.netty.channel._ -import akka.actor.ActorSystemImpl -import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor } -import java.util.concurrent._ -import locks.ReentrantReadWriteLock +import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture } +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.channel.{ ChannelHandlerContext, ChannelFutureListener, ChannelFuture, Channel } +import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder } +import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor +import org.jboss.netty.util.HashedWheelTimer +import akka.actor.{ ActorSystemImpl, ActorRef, simpleName } import akka.dispatch.MonitorableThreadFactory - -class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null) -} - -/** - * This is the abstract baseclass for netty remote clients, currently there's only an - * ActiveRemoteClient, but others could be feasible, like a PassiveRemoteClient that - * reuses an already established connection. - */ -abstract class RemoteClient private[akka] ( - val remoteSupport: NettyRemoteSupport, - val remoteAddress: RemoteNettyAddress) { - - val log = Logging(remoteSupport.system, "RemoteClient") - - val name = simpleName(this) + "@" + remoteAddress - - private[remote] val runSwitch = new Switch() - - private[remote] def isRunning = runSwitch.isOn - - protected def currentChannel: Channel - - def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean - - def shutdown(): Boolean - - def isBoundTo(address: RemoteNettyAddress): Boolean = remoteAddress == address - - /** - * Converts the message to the wireprotocol and sends the message across the wire - */ - def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { - log.debug("Sending message {} from {} to {}", message, senderOption, recipient) - send((message, senderOption, recipient)) - } else { - val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress) - remoteSupport.notifyListeners(RemoteClientError(exception, remoteSupport, remoteAddress)) - throw exception - } - - /** - * Sends the message across the wire - */ - private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = { - try { - val channel = currentChannel - val f = channel.write(request) - f.addListener( - new ChannelFutureListener { - def operationComplete(future: ChannelFuture) { - if (future.isCancelled || !future.isSuccess) { - remoteSupport.notifyListeners(RemoteClientWriteFailed(request, future.getCause, remoteSupport, remoteAddress)) - } - } - }) - // Check if we should back off - if (!channel.isWritable) { - val backoff = remoteSupport.remote.remoteSettings.BackoffTimeout - if (backoff.length > 0 && !f.await(backoff.length, backoff.unit)) f.cancel() //Waited as long as we could, now back off - } - } catch { - case e: Exception ⇒ remoteSupport.notifyListeners(RemoteClientError(e, remoteSupport, remoteAddress)) - } - } - - override def toString = name -} - -class PassiveRemoteClient(val currentChannel: Channel, - remoteSupport: NettyRemoteSupport, - remoteAddress: RemoteNettyAddress) - extends RemoteClient(remoteSupport, remoteAddress) { - - def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = runSwitch switchOn { - remoteSupport.notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress)) - log.debug("Starting remote client connection to [{}]", remoteAddress) - } - - def shutdown() = runSwitch switchOff { - log.debug("Shutting down remote client [{}]", name) - - remoteSupport.notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress)) - log.debug("[{}] has been shut down", name) - } -} - -/** - * RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node. - */ -class ActiveRemoteClient private[akka] ( - remoteSupport: NettyRemoteSupport, - remoteAddress: RemoteNettyAddress, - localAddress: RemoteSystemAddress[ParsedTransportAddress], - val loader: Option[ClassLoader] = None) - extends RemoteClient(remoteSupport, remoteAddress) { - - if (remoteAddress.ip.isEmpty) throw new java.net.UnknownHostException(remoteAddress.host) - - import remoteSupport.clientSettings._ - - //TODO rewrite to a wrapper object (minimize volatile access and maximize encapsulation) - @volatile - private var bootstrap: ClientBootstrap = _ - @volatile - private var connection: ChannelFuture = _ - @volatile - private[remote] var openChannels: DefaultChannelGroup = _ - @volatile - private var executionHandler: ExecutionHandler = _ - - @volatile - private var reconnectionTimeWindowStart = 0L - - def notifyListeners(msg: RemoteLifeCycleEvent): Unit = remoteSupport.notifyListeners(msg) - - def currentChannel = connection.getChannel - - /** - * Connect to remote server. - */ - def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = { - - def sendSecureCookie(connection: ChannelFuture) { - val handshake = RemoteControlProtocol.newBuilder.setCommandType(CommandType.CONNECT) - if (SecureCookie.nonEmpty) handshake.setCookie(SecureCookie.get) - handshake.setOrigin(RemoteProtocol.AddressProtocol.newBuilder - .setSystem(localAddress.system) - .setHostname(localAddress.transport.host) - .setPort(localAddress.transport.port) - .build) - connection.getChannel.write(remoteSupport.createControlEnvelope(handshake.build)) - } - - def attemptReconnect(): Boolean = { - log.debug("Remote client reconnecting to [{}]", remoteAddress) - connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port)) - openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. - - if (!connection.isSuccess) { - notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress)) - false - } else { - sendSecureCookie(connection) - true - } - } - - runSwitch switchOn { - openChannels = new DefaultDisposableChannelGroup(classOf[RemoteClient].getName) - - executionHandler = new ExecutionHandler(remoteSupport.executor) - - bootstrap = new ClientBootstrap(remoteSupport.clientChannelFactory) - bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this)) - bootstrap.setOption("tcpNoDelay", true) - bootstrap.setOption("keepAlive", true) - - log.debug("Starting remote client connection to [{}]", remoteAddress) - - connection = bootstrap.connect(new InetSocketAddress(remoteAddress.ip.get, remoteAddress.port)) - - openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails. - - if (!connection.isSuccess) { - notifyListeners(RemoteClientError(connection.getCause, remoteSupport, remoteAddress)) - false - } else { - sendSecureCookie(connection) - notifyListeners(RemoteClientStarted(remoteSupport, remoteAddress)) - true - } - } match { - case true ⇒ true - case false if reconnectIfAlreadyConnected ⇒ - connection.getChannel.close() - openChannels.remove(connection.getChannel) - - log.debug("Remote client reconnecting to [{}]", remoteAddress) - attemptReconnect() - - case false ⇒ false - } - } - - // Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients - def shutdown() = runSwitch switchOff { - log.debug("Shutting down remote client [{}]", name) - - notifyListeners(RemoteClientShutdown(remoteSupport, remoteAddress)) - try { - if ((connection ne null) && (connection.getChannel ne null)) - connection.getChannel.close() - } finally { - try { - if (openChannels ne null) openChannels.close.awaitUninterruptibly() - } finally { - connection = null - executionHandler = null - } - } - - log.debug("[{}] has been shut down", name) - } - - private[akka] def isWithinReconnectionTimeWindow: Boolean = { - if (reconnectionTimeWindowStart == 0L) { - reconnectionTimeWindowStart = System.currentTimeMillis - true - } else { - val timeLeft = (ReconnectionTimeWindow.toMillis - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0 - if (timeLeft) - log.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft) - - timeLeft - } - } - - private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L -} - -class ActiveRemoteClientPipelineFactory( - name: String, - bootstrap: ClientBootstrap, - executionHandler: ExecutionHandler, - remoteAddress: RemoteNettyAddress, - client: ActiveRemoteClient) extends ChannelPipelineFactory { - - import client.remoteSupport.clientSettings._ - - def getPipeline: ChannelPipeline = { - val timeout = new ReadTimeoutHandler(client.remoteSupport.timer, ReadTimeout.length, ReadTimeout.unit) - val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4) - val lenPrep = new LengthFieldPrepender(4) - val messageDec = new RemoteMessageDecoder - val messageEnc = new RemoteMessageEncoder(client.remoteSupport) - val remoteClient = new ActiveRemoteClientHandler(name, bootstrap, remoteAddress, client.remoteSupport.timer, client) - - new StaticChannelPipeline(timeout, lenDec, messageDec, lenPrep, messageEnc, executionHandler, remoteClient) - } -} - -class RemoteMessageEncoder(remoteSupport: NettyRemoteSupport) extends ProtobufEncoder { - override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = { - msg match { - case (message: Any, sender: Option[_], recipient: ActorRef) ⇒ - super.encode(ctx, channel, - remoteSupport.createMessageSendEnvelope( - remoteSupport.createRemoteMessageProtocolBuilder( - recipient, - message, - sender.asInstanceOf[Option[ActorRef]]).build)) - case _ ⇒ super.encode(ctx, channel, msg) - } - } -} - -class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance) - -@ChannelHandler.Sharable -class ActiveRemoteClientHandler( - val name: String, - val bootstrap: ClientBootstrap, - val remoteAddress: RemoteNettyAddress, - val timer: HashedWheelTimer, - val client: ActiveRemoteClient) - extends SimpleChannelUpstreamHandler { - - def runOnceNow(thunk: ⇒ Unit): Unit = timer.newTimeout(new TimerTask() { - def run(timeout: Timeout) = try { thunk } finally { timeout.cancel() } - }, 0, TimeUnit.MILLISECONDS) - - override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) { - try { - event.getMessage match { - case arp: AkkaRemoteProtocol if arp.hasInstruction ⇒ - val rcp = arp.getInstruction - rcp.getCommandType match { - case CommandType.SHUTDOWN ⇒ runOnceNow { client.remoteSupport.shutdownClientConnection(remoteAddress) } - case _ ⇒ //Ignore others - } - - case arp: AkkaRemoteProtocol if arp.hasMessage ⇒ - client.remoteSupport.receiveMessage(new RemoteMessage(arp.getMessage, client.remoteSupport.system, client.loader)) - - case other ⇒ - throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.remoteSupport, client.remoteAddress) - } - } catch { - case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress)) - } - } - - override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn { - if (client.isWithinReconnectionTimeWindow) { - timer.newTimeout(new TimerTask() { - def run(timeout: Timeout) = - if (client.isRunning) { - client.openChannels.remove(event.getChannel) - client.connect(reconnectIfAlreadyConnected = true) - } - }, client.remoteSupport.clientSettings.ReconnectDelay.toMillis, TimeUnit.MILLISECONDS) - } else runOnceNow { - client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread - } - } - - override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - try { - client.notifyListeners(RemoteClientConnected(client.remoteSupport, client.remoteAddress)) - client.resetReconnectionTimeWindow - } catch { - case e: Exception ⇒ client.notifyListeners(RemoteClientError(e, client.remoteSupport, client.remoteAddress)) - } - } - - override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - client.notifyListeners(RemoteClientDisconnected(client.remoteSupport, client.remoteAddress)) - } - - override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - val cause = event.getCause - if (cause ne null) { - client.notifyListeners(RemoteClientError(cause, client.remoteSupport, client.remoteAddress)) - cause match { - case e: ReadTimeoutException ⇒ - runOnceNow { - client.remoteSupport.shutdownClientConnection(remoteAddress) // spawn in another thread - } - case e: Exception ⇒ event.getChannel.close() - } - - } else client.notifyListeners(RemoteClientError(new Exception("Unknown cause"), client.remoteSupport, client.remoteAddress)) - } -} +import akka.event.Logging +import akka.remote.RemoteProtocol.AkkaRemoteProtocol +import akka.remote.{ RemoteTransport, RemoteMarshallingOps, RemoteClientWriteFailed, RemoteClientException, RemoteClientError, RemoteActorRef } +import akka.util.Switch +import akka.AkkaException +import com.typesafe.config.Config +import akka.remote.RemoteSettings +import akka.actor.Address +import java.net.InetSocketAddress +import akka.remote.RemoteActorRefProvider +import akka.remote.RemoteActorRefProvider +import akka.event.LoggingAdapter /** * Provides the implementation of the Netty remote support */ -class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val address: RemoteSystemAddress[RemoteNettyAddress]) - extends RemoteSupport[RemoteNettyAddress](_system) with RemoteMarshallingOps { - val log = Logging(system, "NettyRemoteSupport") +class NettyRemoteTransport(val remoteSettings: RemoteSettings) + extends RemoteTransport with RemoteMarshallingOps { - val serverSettings = remote.remoteSettings.serverSettings - val clientSettings = remote.remoteSettings.clientSettings + val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName) - val threadFactory = new MonitorableThreadFactory("NettyRemoteSupport", remote.remoteSettings.Daemonic) + val threadFactory = new MonitorableThreadFactory("NettyRemoteTransport", settings.Daemonic) val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory) val executor = new OrderedMemoryAwareThreadPoolExecutor( - serverSettings.ExecutionPoolSize, - serverSettings.MaxChannelMemorySize, - serverSettings.MaxTotalMemorySize, - serverSettings.ExecutionPoolKeepAlive.length, - serverSettings.ExecutionPoolKeepAlive.unit, + settings.ExecutionPoolSize, + settings.MaxChannelMemorySize, + settings.MaxTotalMemorySize, + settings.ExecutionPoolKeepAlive.length, + settings.ExecutionPoolKeepAlive.unit, threadFactory) val clientChannelFactory = new NioClientSocketChannelFactory( Executors.newCachedThreadPool(threadFactory), Executors.newCachedThreadPool(threadFactory)) - private val remoteClients = new HashMap[RemoteNettyAddress, RemoteClient] + private val remoteClients = new HashMap[Address, RemoteClient] private val clientsLock = new ReentrantReadWriteLock - override protected def useUntrustedMode = serverSettings.UntrustedMode + override protected def useUntrustedMode = remoteSettings.UntrustedMode + + val server = try new NettyRemoteServer(this, Some(getClass.getClassLoader)) catch { + case ex ⇒ shutdown(); throw ex + } + + val address = { + server.channel.getLocalAddress match { + case ia: InetSocketAddress ⇒ Address("akka", remoteSettings.systemName, Some(ia.getHostName), Some(ia.getPort)) + case x ⇒ + shutdown() + throw new IllegalArgumentException("unknown address format " + x + ":" + x.getClass) + } + } + + @volatile + private var _system: ActorSystemImpl = _ + def system = _system + + @volatile + private var _provider: RemoteActorRefProvider = _ + def provider = _provider + + @volatile + private var _log: LoggingAdapter = _ + def log = _log + + def start(system: ActorSystemImpl, provider: RemoteActorRefProvider): Unit = { + _system = system + _provider = provider + _log = Logging(system, "NettyRemoteTransport") + server.start(system) + } + + def shutdown(): Unit = { + clientsLock.writeLock().lock() + try { + remoteClients foreach { case (_, client) ⇒ client.shutdown() } + remoteClients.clear() + } finally { + clientsLock.writeLock().unlock() + try { + if (server != null) server.shutdown() + } finally { + try { + timer.stop() + } finally { + try { + clientChannelFactory.releaseExternalResources() + } finally { + executor.shutdown() + } + } + } + } + } protected[akka] def send( message: Any, @@ -399,13 +119,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre recipient: RemoteActorRef, loader: Option[ClassLoader]): Unit = { - val recipientAddress = recipient.path.address match { - case RemoteSystemAddress(sys, transport) ⇒ - transport match { - case x: RemoteNettyAddress ⇒ x - case _ ⇒ throw new IllegalArgumentException("invoking NettyRemoteSupport.send with foreign target address " + transport) - } - } + val recipientAddress = recipient.path.address clientsLock.readLock.lock try { @@ -420,7 +134,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre //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, remote.remoteAddress, loader) + val client = new ActiveRemoteClient(this, recipientAddress, address, loader) client.connect() remoteClients += recipientAddress -> client client @@ -438,7 +152,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre } } - def bindClient(remoteAddress: RemoteNettyAddress, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = { + def bindClient(remoteAddress: Address, client: RemoteClient, putIfAbsent: Boolean = false): Boolean = { clientsLock.writeLock().lock() try { if (putIfAbsent && remoteClients.contains(remoteAddress)) false @@ -452,7 +166,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre } } - def unbindClient(remoteAddress: RemoteNettyAddress): Unit = { + def unbindClient(remoteAddress: Address): Unit = { clientsLock.writeLock().lock() try { remoteClients.foreach { case (k, v) ⇒ if (v.isBoundTo(remoteAddress)) { v.shutdown(); remoteClients.remove(k) } } @@ -461,7 +175,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre } } - def shutdownClientConnection(remoteAddress: RemoteNettyAddress): Boolean = { + def shutdownClientConnection(remoteAddress: Address): Boolean = { clientsLock.writeLock().lock() try { remoteClients.remove(remoteAddress) match { @@ -473,7 +187,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre } } - def restartClientConnection(remoteAddress: RemoteNettyAddress): Boolean = { + def restartClientConnection(remoteAddress: Address): Boolean = { clientsLock.readLock().lock() try { remoteClients.get(remoteAddress) match { @@ -485,229 +199,24 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre } } - /** - * Server section - */ - @volatile - private var currentServer: NettyRemoteServer = _ +} - def name = currentServer match { - case null ⇒ remote.remoteAddress.toString - case server ⇒ server.name - } - - private val _isRunning = new Switch(false) - - def isRunning = _isRunning.isOn - - def start(loader: Option[ClassLoader] = None): Unit = - _isRunning switchOn { currentServer = new NettyRemoteServer(this, loader, address) } - - /** - * Common section - */ - - def shutdown(): Unit = _isRunning switchOff { - clientsLock.writeLock().lock() - try { - remoteClients foreach { case (_, client) ⇒ client.shutdown() } - remoteClients.clear() - } finally { - clientsLock.writeLock().unlock() - try { - val s = currentServer - currentServer = null - s.shutdown() - } finally { - try { - timer.stop() - } finally { - try { - clientChannelFactory.releaseExternalResources() - } finally { - executor.shutdown() - } - } - } +class RemoteMessageEncoder(remoteSupport: NettyRemoteTransport) extends ProtobufEncoder { + override def encode(ctx: ChannelHandlerContext, channel: Channel, msg: AnyRef): AnyRef = { + msg match { + case (message: Any, sender: Option[_], recipient: ActorRef) ⇒ + super.encode(ctx, channel, + remoteSupport.createMessageSendEnvelope( + remoteSupport.createRemoteMessageProtocolBuilder( + recipient, + message, + sender.asInstanceOf[Option[ActorRef]]).build)) + case _ ⇒ super.encode(ctx, channel, msg) } } } -class NettyRemoteServer( - val remoteSupport: NettyRemoteSupport, - val loader: Option[ClassLoader], - val address: RemoteSystemAddress[RemoteNettyAddress]) { - val log = Logging(remoteSupport.system, "NettyRemoteServer") - import remoteSupport.serverSettings._ - - if (address.transport.ip.isEmpty) throw new java.net.UnknownHostException(address.transport.host) - - val name = "NettyRemoteServer@" + address - - private val factory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(remoteSupport.threadFactory), - Executors.newCachedThreadPool(remoteSupport.threadFactory)) - - private val bootstrap = new ServerBootstrap(factory) - - 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, executionHandler, loader, remoteSupport) - bootstrap.setPipelineFactory(pipelineFactory) - bootstrap.setOption("backlog", Backlog) - bootstrap.setOption("child.tcpNoDelay", true) - bootstrap.setOption("child.keepAlive", true) - bootstrap.setOption("child.reuseAddress", true) - bootstrap.setOption("child.connectTimeoutMillis", ConnectionTimeout.toMillis) - - openChannels.add(bootstrap.bind(new InetSocketAddress(address.transport.ip.get, address.transport.port))) - remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport)) - - def shutdown() { - try { - val shutdownSignal = { - val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN) - b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder - .setSystem(address.system) - .setHostname(address.transport.host) - .setPort(address.transport.port) - .build) - if (SecureCookie.nonEmpty) - b.setCookie(SecureCookie.get) - b.build - } - openChannels.write(remoteSupport.createControlEnvelope(shutdownSignal)).awaitUninterruptibly - openChannels.disconnect - openChannels.close.awaitUninterruptibly - bootstrap.releaseExternalResources() - remoteSupport.notifyListeners(RemoteServerShutdown(remoteSupport)) - } catch { - case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport)) - } - } -} - -class RemoteServerPipelineFactory( - val name: String, - val openChannels: ChannelGroup, - val executionHandler: ExecutionHandler, - val loader: Option[ClassLoader], - val remoteSupport: NettyRemoteSupport) extends ChannelPipelineFactory { - - import remoteSupport.serverSettings._ - - def getPipeline: ChannelPipeline = { - val lenDec = new LengthFieldBasedFrameDecoder(MessageFrameSize, 0, 4, 0, 4) - val lenPrep = new LengthFieldPrepender(4) - val messageDec = new RemoteMessageDecoder - val messageEnc = new RemoteMessageEncoder(remoteSupport) - - val authenticator = if (RequireCookie) new RemoteServerAuthenticationHandler(SecureCookie) :: Nil else Nil - val remoteServer = new RemoteServerHandler(name, openChannels, loader, remoteSupport) - val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil - new StaticChannelPipeline(stages: _*) - } -} - -@ChannelHandler.Sharable -class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler { - val authenticated = new AnyRef - - override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match { - case None ⇒ ctx.sendUpstream(event) - case Some(cookie) ⇒ - ctx.getAttachment match { - case `authenticated` ⇒ ctx.sendUpstream(event) - case null ⇒ event.getMessage match { - case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒ - val instruction = remoteProtocol.getInstruction - instruction.getCookie match { - case `cookie` ⇒ - ctx.setAttachment(authenticated) - ctx.sendUpstream(event) - case _ ⇒ - throw new SecurityException( - "The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie") - } - case _ ⇒ - throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!") - } - } - } -} - -@ChannelHandler.Sharable -class RemoteServerHandler( - val name: String, - val openChannels: ChannelGroup, - val applicationLoader: Option[ClassLoader], - val remoteSupport: NettyRemoteSupport) extends SimpleChannelUpstreamHandler { - - val log = Logging(remoteSupport.system, "RemoteServerHandler") - - import remoteSupport.serverSettings._ - - /** - * ChannelOpen overridden to store open channels for a clean postStop of a node. - * If a channel is closed before, it is automatically removed from the open channels group. - */ - override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel) - - override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - val clientAddress = getClientAddress(ctx.getChannel) - remoteSupport.notifyListeners(RemoteServerClientConnected(remoteSupport, clientAddress)) - } - - override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { - val clientAddress = getClientAddress(ctx.getChannel) - remoteSupport.notifyListeners(RemoteServerClientDisconnected(remoteSupport, clientAddress)) - } - - override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match { - case s @ Some(address) ⇒ - if (UsePassiveConnections) - remoteSupport.unbindClient(address) - remoteSupport.notifyListeners(RemoteServerClientClosed(remoteSupport, s)) - case None ⇒ - remoteSupport.notifyListeners(RemoteServerClientClosed[RemoteNettyAddress](remoteSupport, None)) - } - - override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try { - event.getMessage match { - case remote: AkkaRemoteProtocol if remote.hasMessage ⇒ - remoteSupport.receiveMessage(new RemoteMessage(remote.getMessage, remoteSupport.system, applicationLoader)) - - case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒ - val instruction = remote.getInstruction - instruction.getCommandType match { - case CommandType.CONNECT if UsePassiveConnections ⇒ - val origin = instruction.getOrigin - val inbound = RemoteNettyAddress(origin.getHostname, origin.getPort) - val client = new PassiveRemoteClient(event.getChannel, remoteSupport, inbound) - remoteSupport.bindClient(inbound, client) - case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed - case _ ⇒ //Unknown command - } - case _ ⇒ //ignore - } - } catch { - case e: Exception ⇒ remoteSupport.notifyListeners(RemoteServerError(e, remoteSupport)) - } - - override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - remoteSupport.notifyListeners(RemoteServerError(event.getCause, remoteSupport)) - event.getChannel.close() - } - - private def getClientAddress(c: Channel): Option[RemoteNettyAddress] = - c.getRemoteAddress match { - case inet: InetSocketAddress ⇒ Some(RemoteNettyAddress(inet.getHostName, Some(inet.getAddress), inet.getPort)) - case _ ⇒ None - } -} +class RemoteMessageDecoder extends ProtobufDecoder(AkkaRemoteProtocol.getDefaultInstance) class DefaultDisposableChannelGroup(name: String) extends DefaultChannelGroup(name) { protected val guard = new ReentrantReadWriteLock diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala new file mode 100644 index 0000000000..a50d3653e8 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -0,0 +1,202 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.netty + +import java.net.InetSocketAddress +import java.util.concurrent.Executors +import scala.Option.option2Iterable +import org.jboss.netty.bootstrap.ServerBootstrap +import org.jboss.netty.channel.ChannelHandler.Sharable +import org.jboss.netty.channel.group.ChannelGroup +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory +import org.jboss.netty.channel.{ StaticChannelPipeline, SimpleChannelUpstreamHandler, MessageEvent, ExceptionEvent, ChannelStateEvent, ChannelPipelineFactory, ChannelPipeline, ChannelHandlerContext, ChannelHandler, Channel } +import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } +import org.jboss.netty.handler.execution.ExecutionHandler +import akka.event.Logging +import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } +import akka.remote.{ RemoteServerStarted, RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage } +import akka.actor.Address +import java.net.InetAddress +import akka.actor.ActorSystemImpl +import org.jboss.netty.channel.ChannelLocal + +class NettyRemoteServer( + val netty: NettyRemoteTransport, + val loader: Option[ClassLoader]) { + + import netty.settings + + val ip = InetAddress.getByName(settings.Hostname) + + private val factory = new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(netty.threadFactory), + Executors.newCachedThreadPool(netty.threadFactory)) + + private val bootstrap = new ServerBootstrap(factory) + + private val executionHandler = new ExecutionHandler(netty.executor) + + // group of open channels, used for clean-up + private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") + + val pipelineFactory = new RemoteServerPipelineFactory(openChannels, executionHandler, loader, netty) + bootstrap.setPipelineFactory(pipelineFactory) + bootstrap.setOption("backlog", settings.Backlog) + bootstrap.setOption("tcpNoDelay", true) + bootstrap.setOption("keepAlive", true) + bootstrap.setOption("reuseAddress", true) + + val channel = bootstrap.bind(new InetSocketAddress(ip, settings.Port)) + + openChannels.add(channel) + + def start(system: ActorSystemImpl) { + netty.notifyListeners(RemoteServerStarted(netty)) + // TODO uncork the pipeline, which was ... + // TODO ... corked before in order not to allow anything through before init is complete + } + + def shutdown() { + try { + val shutdownSignal = { + val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN) + b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder + .setSystem(settings.systemName) + .setHostname(settings.Hostname) + .setPort(settings.Port) + .build) + if (settings.SecureCookie.nonEmpty) + b.setCookie(settings.SecureCookie.get) + b.build + } + openChannels.write(netty.createControlEnvelope(shutdownSignal)).awaitUninterruptibly + openChannels.disconnect + openChannels.close.awaitUninterruptibly + bootstrap.releaseExternalResources() + netty.notifyListeners(RemoteServerShutdown(netty)) + } catch { + case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty)) + } + } +} + +class RemoteServerPipelineFactory( + val openChannels: ChannelGroup, + val executionHandler: ExecutionHandler, + val loader: Option[ClassLoader], + 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, loader, netty) + val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil + new StaticChannelPipeline(stages: _*) + } +} + +@ChannelHandler.Sharable +class RemoteServerAuthenticationHandler(secureCookie: Option[String]) extends SimpleChannelUpstreamHandler { + val authenticated = new AnyRef + + override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = secureCookie match { + case None ⇒ ctx.sendUpstream(event) + case Some(cookie) ⇒ + ctx.getAttachment match { + case `authenticated` ⇒ ctx.sendUpstream(event) + case null ⇒ event.getMessage match { + case remoteProtocol: AkkaRemoteProtocol if remoteProtocol.hasInstruction ⇒ + val instruction = remoteProtocol.getInstruction + instruction.getCookie match { + case `cookie` ⇒ + ctx.setAttachment(authenticated) + ctx.sendUpstream(event) + case _ ⇒ + throw new SecurityException( + "The remote client [" + ctx.getChannel.getRemoteAddress + "] secure cookie is not the same as remote server secure cookie") + } + case _ ⇒ + throw new SecurityException("The remote client [" + ctx.getChannel.getRemoteAddress + "] is not authorized!") + } + } + } +} + +object ChannelLocalSystem extends ChannelLocal[ActorSystemImpl] { + override def initialValue(ch: Channel): ActorSystemImpl = null +} + +@ChannelHandler.Sharable +class RemoteServerHandler( + val openChannels: ChannelGroup, + val applicationLoader: Option[ClassLoader], + val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler { + + import netty.settings + + /** + * ChannelOpen overridden to store open channels for a clean postStop of a node. + * If a channel is closed before, it is automatically removed from the open channels group. + */ + override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel) + + override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val clientAddress = getClientAddress(ctx.getChannel) + netty.notifyListeners(RemoteServerClientConnected(netty, clientAddress)) + } + + override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { + val clientAddress = getClientAddress(ctx.getChannel) + netty.notifyListeners(RemoteServerClientDisconnected(netty, clientAddress)) + } + + override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = getClientAddress(ctx.getChannel) match { + case s @ Some(address) ⇒ + if (settings.UsePassiveConnections) + netty.unbindClient(address) + netty.notifyListeners(RemoteServerClientClosed(netty, s)) + case None ⇒ + netty.notifyListeners(RemoteServerClientClosed(netty, None)) + } + + override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try { + event.getMessage match { + case remote: AkkaRemoteProtocol if remote.hasMessage ⇒ + netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system, applicationLoader)) + + case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒ + val instruction = remote.getInstruction + instruction.getCommandType match { + case CommandType.CONNECT if settings.UsePassiveConnections ⇒ + val origin = instruction.getOrigin + val inbound = Address("akka", origin.getSystem, Some(origin.getHostname), Some(origin.getPort)) + val client = new PassiveRemoteClient(event.getChannel, netty, inbound) + netty.bindClient(inbound, client) + case CommandType.SHUTDOWN ⇒ //Will be unbound in channelClosed + case _ ⇒ //Unknown command + } + case _ ⇒ //ignore + } + } catch { + case e: Exception ⇒ netty.notifyListeners(RemoteServerError(e, netty)) + } + + override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { + netty.notifyListeners(RemoteServerError(event.getCause, netty)) + event.getChannel.close() + } + + private def getClientAddress(c: Channel): Option[Address] = + c.getRemoteAddress match { + case inet: InetSocketAddress ⇒ Some(Address("akka", "unknown(yet)", Some(inet.getAddress.toString), Some(inet.getPort))) + case _ ⇒ None + } +} + diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala new file mode 100644 index 0000000000..9596f1b083 --- /dev/null +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -0,0 +1,63 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.remote.netty + +import com.typesafe.config.Config +import akka.util.Duration +import java.util.concurrent.TimeUnit._ +import java.net.InetAddress +import akka.config.ConfigurationException + +class NettySettings(config: Config, val systemName: String) { + + import config._ + + val Daemonic = getBoolean("daemonic") + val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS) + + val SecureCookie: Option[String] = getString("secure-cookie") match { + case "" ⇒ None + case cookie ⇒ Some(cookie) + } + val RequireCookie = { + val requireCookie = getBoolean("require-cookie") + if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException( + "Configuration option 'akka.remote.netty.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.netty.secure-cookie'.") + requireCookie + } + + val UsePassiveConnections = getBoolean("use-passive-connections") + + val ReconnectionTimeWindow = Duration(getMilliseconds("reconnection-time-window"), MILLISECONDS) + val ReadTimeout = Duration(getMilliseconds("read-timeout"), MILLISECONDS) + val ReconnectDelay = Duration(getMilliseconds("reconnect-delay"), MILLISECONDS) + val MessageFrameSize = getBytes("message-frame-size").toInt + + val Hostname = getString("hostname") match { + case "" ⇒ InetAddress.getLocalHost.getHostAddress + case value ⇒ value + } + val Port = getInt("port") + val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) + + val Backlog = getInt("backlog") + + val ExecutionPoolKeepAlive = Duration(getMilliseconds("execution-pool-keepalive"), MILLISECONDS) + + 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 ⇒ sz + } + + val MaxChannelMemorySize = getBytes("max-channel-memory-size") match { + case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-channel-memory-size is less than 0 bytes") + case sz ⇒ sz + } + + val MaxTotalMemorySize = getBytes("max-total-memory-size") match { + case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.netty.max-total-memory-size is less than 0 bytes") + case sz ⇒ sz + } + +} \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/package.scala b/akka-remote/src/main/scala/akka/remote/package.scala deleted file mode 100644 index e2514d6592..0000000000 --- a/akka-remote/src/main/scala/akka/remote/package.scala +++ /dev/null @@ -1,8 +0,0 @@ -/** - * Copyright (C) 2009-2010 Typesafe Inc. - */ -package akka - -package object remote { - type TransportsMap = Map[String, (String, Int) ⇒ Either[String, RemoteTransportAddress]] -} \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index dffb874be6..43bd2d06df 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -12,7 +12,7 @@ import akka.actor.InternalActorRef import akka.actor.Props import akka.config.ConfigurationException import akka.remote.RemoteScope -import akka.remote.RemoteAddressExtractor +import akka.actor.AddressExtractor /** * [[akka.routing.RouterConfig]] implementation for remote deployment on defined @@ -46,8 +46,8 @@ class RemoteRouteeProvider(nodes: Iterable[String], _context: ActorContext, _res // need this iterator as instance variable since Resizer may call createRoutees several times private val nodeAddressIter = { val nodeAddresses = nodes map { - case RemoteAddressExtractor(a) ⇒ a - case x ⇒ throw new ConfigurationException("unparseable remote node " + x) + case AddressExtractor(a) ⇒ a + case x ⇒ throw new ConfigurationException("unparseable remote node " + x) } Stream.continually(nodeAddresses).flatten.iterator } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala index 597b552fe9..2d409811cc 100755 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala @@ -21,8 +21,8 @@ trait AbstractRemoteActorMultiJvmSpec { case (idx, host) => ConfigFactory.parseString(""" akka { - remote.server.hostname="%s" - remote.server.port = "%d" + remote.netty.hostname="%s" + remote.netty.port = "%d" }""".format(host, 9990+idx, idx)) withFallback commonConfig } } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala index 2a38a33402..8b9a79a64e 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala @@ -32,14 +32,4 @@ abstract class AkkaRemoteSpec(config: Config) extends AkkaSpec(config.withFallback(AkkaRemoteSpec.testConf)) with MultiJvmSync { - /** - * Helper function for accessing the underlying remoting. - */ - def remote: Remote = { - system.asInstanceOf[ActorSystemImpl].provider match { - case r: RemoteActorRefProvider ⇒ r.remote - case _ ⇒ throw new Exception("Remoting is not enabled") - } - } - } diff --git a/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala b/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala index 5ac2f281ab..17a848b8d3 100644 --- a/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/AccrualFailureDetectorSpec.scala @@ -2,11 +2,12 @@ package akka.remote import java.net.InetSocketAddress import akka.testkit.AkkaSpec +import akka.actor.Address class AccrualFailureDetectorSpec extends AkkaSpec { "An AccrualFailureDetector" must { - val conn = RemoteNettyAddress("localhost", 2552) + val conn = Address("akka", "", Some("localhost"), Some(2552)) "mark node as available after a series of successful heartbeats" in { val fd = new AccrualFailureDetector() diff --git a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala index e02b072e2e..4b46fdddfc 100644 --- a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala @@ -6,7 +6,7 @@ package akka.remote import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } -import akka.remote.netty.NettyRemoteSupport +import akka.remote.netty.NettyRemoteTransport import akka.actor.Actor import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 58199b4683..ddf1f338a6 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -33,7 +33,7 @@ object RemoteCommunicationSpec { class RemoteCommunicationSpec extends AkkaSpec(""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" - remote.server { + remote.netty { hostname = localhost port = 12345 } @@ -47,7 +47,7 @@ akka { import RemoteCommunicationSpec._ - val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config) + val conf = ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config) val other = ActorSystem("remote_sys", conf) val remote = other.actorOf(Props(new Actor { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 49502b99ea..72a980db93 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -1,6 +1,10 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ package akka.remote import akka.testkit.AkkaSpec +import akka.util.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RemoteConfigSpec extends AkkaSpec("") { @@ -8,54 +12,20 @@ class RemoteConfigSpec extends AkkaSpec("") { "RemoteExtension" must { "be able to parse remote and cluster config elements" in { - val config = system.settings.config - import config._ + val settings = new RemoteSettings(system.settings.config, "") + import settings._ - //akka.remote - getString("akka.remote.transport") must equal("akka.remote.netty.NettyRemoteSupport") - getString("akka.remote.secure-cookie") must equal("") - getBoolean("akka.remote.use-passive-connections") must equal(true) - getMilliseconds("akka.remote.backoff-timeout") must equal(0) - getBoolean("akka.remote.daemonic") must equal(true) - // getMilliseconds("akka.remote.remote-daemon-ack-timeout") must equal(30 * 1000) + RemoteTransport must be("akka.remote.netty.NettyRemoteTransport") + UntrustedMode must be(false) + RemoteSystemDaemonAckTimeout must be(30 seconds) - //akka.remote.server - getInt("akka.remote.server.port") must equal(2552) - getBytes("akka.remote.server.message-frame-size") must equal(1048576L) - getMilliseconds("akka.remote.server.connection-timeout") must equal(120 * 1000) - getBoolean("akka.remote.server.require-cookie") must equal(false) - getBoolean("akka.remote.server.untrusted-mode") must equal(false) - getInt("akka.remote.server.backlog") must equal(4096) + FailureDetectorThreshold must be(8) + FailureDetectorMaxSampleSize must be(1000) - getMilliseconds("akka.remote.server.execution-pool-keepalive") must equal(60 * 1000) + InitialDelayForGossip must be(5 seconds) + GossipFrequency must be(1 second) + SeedNodes must be(Set()) - getInt("akka.remote.server.execution-pool-size") must equal(4) - - getBytes("akka.remote.server.max-channel-memory-size") must equal(0) - getBytes("akka.remote.server.max-total-memory-size") must equal(0) - - //akka.remote.client - getMilliseconds("akka.remote.client.reconnect-delay") must equal(5 * 1000) - getMilliseconds("akka.remote.client.read-timeout") must equal(3600 * 1000) - getMilliseconds("akka.remote.client.reconnection-time-window") must equal(600 * 1000) - - // TODO cluster config will go into akka-cluster/reference.conf when we enable that module - //akka.cluster - getStringList("akka.cluster.seed-nodes") must equal(new java.util.ArrayList[String]) - - // getMilliseconds("akka.cluster.max-time-to-wait-until-connected") must equal(30 * 1000) - // getMilliseconds("akka.cluster.session-timeout") must equal(60 * 1000) - // getMilliseconds("akka.cluster.connection-timeout") must equal(60 * 1000) - // getBoolean("akka.cluster.include-ref-node-in-replica-set") must equal(true) - // getString("akka.cluster.log-directory") must equal("_akka_cluster") - - // //akka.cluster.replication - // getString("akka.cluster.replication.digest-type") must equal("MAC") - // getString("akka.cluster.replication.password") must equal("secret") - // getInt("akka.cluster.replication.ensemble-size") must equal(3) - // getInt("akka.cluster.replication.quorum-size") must equal(2) - // getInt("akka.cluster.replication.snapshot-frequency") must equal(1000) - // getMilliseconds("akka.cluster.replication.timeout") must equal(30 * 1000) } } } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala index 80a4f3cffe..942ca0cd8d 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala @@ -16,14 +16,14 @@ akka { /watchers.remote = "akka://other@127.0.0.1:2666" } } - remote.server { + remote.netty { hostname = "127.0.0.1" port = 2665 } } """)) with ImplicitSender with DefaultTimeout with DeathWatchSpec { - val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.server.port=2666").withFallback(system.settings.config)) + val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.port=2666").withFallback(system.settings.config)) override def atTermination() { other.shutdown() diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala index 15016748d0..37917e9410 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala @@ -41,7 +41,7 @@ class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) { service, deployment.get.config, RoundRobinRouter(3), - RemoteScope(UnparsedSystemAddress(Some("sys"), UnparsedTransportAddress("akka", "wallace", 2552)))))) + RemoteScope(Address("akka", "sys", Some("wallace"), Some(2552)))))) } } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index 6418f93966..fa96afb82b 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -20,7 +20,7 @@ object RemoteRouterSpec { class RemoteRouterSpec extends AkkaSpec(""" akka { actor.provider = "akka.remote.RemoteActorRefProvider" - remote.server { + remote.netty { hostname = localhost port = 12345 } @@ -44,7 +44,7 @@ akka { import RemoteRouterSpec._ - val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config) + val conf = ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config) val other = ActorSystem("remote_sys", conf) override def atTermination() { From c5fc153a1012334272598c73114f3633823c3f80 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 27 Jan 2012 12:14:28 +0100 Subject: [PATCH 02/13] rework system initialization, remove remote address from LocalActorRef - move all creation of ActorRefs into the ActorRefProvider (deadLetters, locker) - rootPath does not contain remote transport address any longer in order to start the LocalActorRefProvider before the RemoteTransport; the transport address is inserted during serialization only, which enables us later to have more than one transport available for one actor system (maybe even needed for clustering) - fix inheritance between DeadLetterActorRef and EmptyLocalActorRef - document some start-up dependencies by reordering constructor code - fix remote tests which used self.path for identifying remote actors (since that no longer includes the remote transport address) --- .../scala/akka/actor/ActorLookupSpec.scala | 7 +- .../test/scala/akka/actor/ActorRefSpec.scala | 4 +- .../src/main/scala/akka/actor/ActorPath.scala | 19 ++++ .../src/main/scala/akka/actor/ActorRef.scala | 75 +++++++-------- .../scala/akka/actor/ActorRefProvider.scala | 31 +++++-- .../main/scala/akka/actor/ActorSystem.scala | 65 ++++++------- .../akka/serialization/Serialization.scala | 8 +- .../akka/remote/RemoteActorRefProvider.scala | 92 +++++++++++-------- .../main/scala/akka/remote/RemoteDaemon.scala | 5 +- .../scala/akka/remote/RemoteSettings.scala | 2 + .../scala/akka/remote/RemoteTransport.scala | 24 +++-- .../main/scala/akka/remote/netty/Client.scala | 2 +- .../remote/netty/NettyRemoteSupport.scala | 30 +----- .../main/scala/akka/remote/netty/Server.scala | 11 +-- .../scala/akka/remote/netty/Settings.scala | 10 +- .../DirectRoutedRemoteActorMultiJvmSpec.scala | 6 +- .../remote/NewRemoteActorMultiJvmSpec.scala | 6 +- .../RandomRoutedRemoteActorMultiJvmSpec.scala | 6 +- ...ndRobinRoutedRemoteActorMultiJvmSpec.scala | 6 +- ...rGatherRoutedRemoteActorMultiJvmSpec.scala | 6 +- .../scala/akka/remote/RemoteRouterSpec.scala | 14 +-- 21 files changed, 225 insertions(+), 204 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index 3d4f61caa1..3b18c742ce 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -46,9 +46,10 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout { val syst = sysImpl.systemGuardian val root = sysImpl.lookupRoot - def empty(path: String) = new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, path match { - case RelativeActorPath(elems) ⇒ system.actorFor("/").path / elems - }) + def empty(path: String) = + new EmptyLocalActorRef(sysImpl.provider, path match { + case RelativeActorPath(elems) ⇒ system.actorFor("/").path / elems + }, system.eventStream) "An ActorSystem" must { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index a4dbb4d1cb..a2c3c7da5a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -290,7 +290,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val sysImpl = system.asInstanceOf[ActorSystemImpl] val addr = sysImpl.provider.rootPath.address - val serialized = SerializedActorRef(addr + "/non-existing") + val serialized = SerializedActorRef(RootActorPath(addr, "/non-existing")) out.writeObject(serialized) @@ -299,7 +299,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { Serialization.currentSystem.withValue(sysImpl) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) - in.readObject must be === new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, system.actorFor("/").path / "non-existing") + in.readObject must be === new EmptyLocalActorRef(sysImpl.provider, system.actorFor("/").path / "non-existing", system.eventStream) } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index d0f0d8154b..99be8bae0e 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -87,6 +87,12 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable { */ def root: RootActorPath + /** + * Generate String representation, replacing the Address in the RootActor + * Path with the given one unless this path’s address includes host and port + * information. + */ + def toStringWithAddress(address: Address): String } /** @@ -105,6 +111,10 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act override val toString = address + name + def toStringWithAddress(addr: Address): String = + if (address.host.isDefined) address + name + else addr + name + def compareTo(other: ActorPath) = other match { case r: RootActorPath ⇒ toString compareTo r.toString case c: ChildActorPath ⇒ 1 @@ -151,6 +161,15 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto rec(parent, new StringBuilder(32).append(name)).toString } + override def toStringWithAddress(addr: Address) = { + @tailrec + def rec(p: ActorPath, s: StringBuilder): StringBuilder = p match { + case r: RootActorPath ⇒ s.insert(0, r.toStringWithAddress(addr)) + case _ ⇒ rec(p.parent, s.insert(0, '/').insert(0, p.name)) + } + rec(parent, new StringBuilder(32).append(name)).toString + } + override def equals(other: Any): Boolean = { @tailrec def rec(left: ActorPath, right: ActorPath): Boolean = diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 7a448c628a..753adaa9fa 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -329,13 +329,13 @@ private[akka] class LocalActorRef private[akka] ( def restart(cause: Throwable): Unit = actorCell.restart(cause) @throws(classOf[java.io.ObjectStreamException]) - protected def writeReplace(): AnyRef = SerializedActorRef(path.toString) + protected def writeReplace(): AnyRef = SerializedActorRef(path) } /** * Memento pattern for serializing ActorRefs transparently */ -case class SerializedActorRef(path: String) { +case class SerializedActorRef private (path: String) { import akka.serialization.Serialization.currentSystem @throws(classOf[java.io.ObjectStreamException]) @@ -349,6 +349,15 @@ case class SerializedActorRef(path: String) { } } +object SerializedActorRef { + def apply(path: ActorPath): SerializedActorRef = { + Serialization.currentTransportAddress.value match { + case null ⇒ new SerializedActorRef(path.toString) + case addr ⇒ new SerializedActorRef(path.toStringWithAddress(addr)) + } + } +} + /** * Trait for ActorRef implementations where all methods contain default stubs. */ @@ -375,7 +384,7 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { def restart(cause: Throwable): Unit = () @throws(classOf[java.io.ObjectStreamException]) - protected def writeReplace(): AnyRef = SerializedActorRef(path.toString) + protected def writeReplace(): AnyRef = SerializedActorRef(path) } private[akka] object MinimalActorRef { @@ -398,57 +407,39 @@ private[akka] object DeadLetterActorRef { val serialized = new SerializedDeadLetterActorRef } -private[akka] trait DeadLetterActorRefLike extends MinimalActorRef { - - def eventStream: EventStream - - @volatile - private var _path: ActorPath = _ - def path: ActorPath = { - assert(_path != null) - _path - } - - @volatile - private var _provider: ActorRefProvider = _ - def provider = _provider - - private[akka] def init(provider: ActorRefProvider, path: ActorPath) { - _path = path - _provider = provider - } - - override def isTerminated(): Boolean = true - - override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match { - case d: DeadLetter ⇒ eventStream.publish(d) - case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) - } -} - -private[akka] class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRefLike { - @throws(classOf[java.io.ObjectStreamException]) - override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized -} - /** * This special dead letter reference has a name: it is that which is returned * by a local look-up which is unsuccessful. */ private[akka] class EmptyLocalActorRef( - val eventStream: EventStream, - _provider: ActorRefProvider, - _dispatcher: MessageDispatcher, - _path: ActorPath) extends DeadLetterActorRefLike { + val provider: ActorRefProvider, + val path: ActorPath, + val eventStream: EventStream) extends MinimalActorRef { - init(_provider, _path) + override def isTerminated(): Boolean = true override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match { - case d: DeadLetter ⇒ // do NOT form endless loops + case d: DeadLetter ⇒ // do NOT form endless loops, since deadLetters will resend! case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) } } +/** + * Internal implementation of the dead letter destination: will publish any + * received message to the eventStream, wrapped as [[akka.actor.DeadLetter]]. + */ +private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream) + extends EmptyLocalActorRef(_provider, _path, _eventStream) { + + override def !(message: Any)(implicit sender: ActorRef = this): Unit = message match { + case d: DeadLetter ⇒ eventStream.publish(d) + case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) + } + + @throws(classOf[java.io.ObjectStreamException]) + override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized +} + /** * Internal implementation detail used for paths like “/temp” */ diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index c47b261b57..369e1429db 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -33,11 +33,22 @@ trait ActorRefProvider { */ def systemGuardian: InternalActorRef + /** + * Dead letter destination for this provider. + */ + def deadLetters: ActorRef + /** * Reference to the death watch service. */ def deathWatch: DeathWatch + /** + * Care-taker of actor refs which await final termination but cannot be kept + * in their parent’s children list because the name shall be freed. + */ + def locker: Locker + /** * The root path for all actors within this actor system, including remote * address if enabled. @@ -281,25 +292,29 @@ class LocalActorRefProvider( val settings: ActorSystem.Settings, val eventStream: EventStream, val scheduler: Scheduler, - val deadLetters: InternalActorRef, - val rootPath: ActorPath, val deployer: Deployer) extends ActorRefProvider { + // this is the constructor needed for reflectively instantiating the provider def this(_systemName: String, settings: ActorSystem.Settings, eventStream: EventStream, - scheduler: Scheduler, - deadLetters: InternalActorRef) = + scheduler: Scheduler) = this(_systemName, settings, eventStream, scheduler, - deadLetters, - new RootActorPath(Address("akka", _systemName)), new Deployer(settings)) + val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) + val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")") + val deadLetters = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream) + + val deathWatch = new LocalDeathWatch(1024) //TODO make configrable + + val locker: Locker = new Locker(scheduler, settings.ReaperInterval, this, rootPath / "locker", deathWatch) + /* * generate name for temporary actor refs */ @@ -455,8 +470,6 @@ class LocalActorRefProvider( tempContainer.removeChild(path.name) } - val deathWatch = new LocalDeathWatch(1024) //TODO make configrable - def init(_system: ActorSystemImpl) { system = _system // chain death watchers so that killing guardian stops the application @@ -492,7 +505,7 @@ class LocalActorRefProvider( } else ref.getChild(path.iterator) match { case Nobody ⇒ log.debug("look-up of path sequence '{}' failed", path) - new EmptyLocalActorRef(eventStream, system.provider, dispatcher, ref.path / path) + new EmptyLocalActorRef(system.provider, ref.path / path, eventStream) case x ⇒ x } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 1347d4e0be..50b69b9ae8 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -267,7 +267,7 @@ abstract class ActorSystem extends ActorRefFactory { * (below which the logging actors reside) and the execute all registered * termination handlers (see [[ActorSystem.registerOnTermination]]). */ - def shutdown() + def shutdown(): Unit /** * Registers the provided extension and creates its payload, if this extension isn't already registered @@ -331,8 +331,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten import ActorSystem._ - final val settings = new Settings(applicationConfig, name) - final val threadFactory = new MonitorableThreadFactory(name, settings.Daemonicity) + final val settings: Settings = new Settings(applicationConfig, name) + final val threadFactory: MonitorableThreadFactory = new MonitorableThreadFactory(name, settings.Daemonicity) def logConfiguration(): Unit = log.info(settings.toString) @@ -377,35 +377,19 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten import settings._ // this provides basic logging (to stdout) until .start() is called below - val eventStream = new EventStream(DebugEventStream) + val eventStream: EventStream = new EventStream(DebugEventStream) eventStream.startStdoutLogger(settings) - // unfortunately we need logging before we know the rootpath address, which wants to be inserted here - @volatile - private var _log = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass) - def log = _log + val log: LoggingAdapter = new BusLogging(eventStream, "ActorSystem(" + name + ")", this.getClass) - val scheduler = createScheduler() - - val deadLetters = new DeadLetterActorRef(eventStream) - val deadLetterMailbox = new Mailbox(null) { - becomeClosed() - override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } - override def dequeue() = null - override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } - override def systemDrain(): SystemMessage = null - override def hasMessages = false - override def hasSystemMessages = false - override def numberOfMessages = 0 - } + val scheduler: Scheduler = createScheduler() val provider: ActorRefProvider = { val arguments = Seq( classOf[String] -> name, classOf[Settings] -> settings, classOf[EventStream] -> eventStream, - classOf[Scheduler] -> scheduler, - classOf[InternalActorRef] -> deadLetters) + classOf[Scheduler] -> scheduler) val loader = Thread.currentThread.getContextClassLoader match { case null ⇒ getClass.getClassLoader @@ -418,8 +402,23 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten } } - val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler)) - val dispatcher = dispatchers.defaultGlobalDispatcher + def deadLetters: ActorRef = provider.deadLetters + + val deadLetterMailbox: Mailbox = new Mailbox(null) { + becomeClosed() + override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) } + override def dequeue() = null + override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) } + override def systemDrain(): SystemMessage = null + override def hasMessages = false + override def hasSystemMessages = false + override def numberOfMessages = 0 + } + + def locker: Locker = provider.locker + + val dispatchers: Dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler)) + val dispatcher: MessageDispatcher = dispatchers.defaultGlobalDispatcher def terminationFuture: Future[Unit] = provider.terminationFuture def lookupRoot: InternalActorRef = provider.rootGuardian @@ -433,21 +432,13 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten private lazy val _start: this.type = { // the provider is expected to start default loggers, LocalActorRefProvider does this provider.init(this) - _log = new BusLogging(eventStream, "ActorSystem(" + lookupRoot.path.address + ")", this.getClass) - deadLetters.init(provider, lookupRoot.path / "deadLetters") registerOnTermination(stopScheduler()) - // this starts the reaper actor and the user-configured logging subscribers, which are also actors - _locker = new Locker(scheduler, ReaperInterval, provider, lookupRoot.path / "locker", deathWatch) loadExtensions() if (LogConfigOnStart) logConfiguration() this } - @volatile - private var _locker: Locker = _ // initialized in start() - def locker = _locker - - def start() = _start + def start(): this.type = _start private lazy val terminationCallbacks = { val callbacks = new TerminationCallbacks @@ -459,9 +450,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) } def awaitTermination() = awaitTermination(Duration.Inf) - def shutdown() { - stop(guardian) - } + def shutdown(): Unit = stop(guardian) /** * Create the scheduler service. This one needs one special behavior: if @@ -557,7 +546,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten } } - override def toString = lookupRoot.path.root.address.toString + override def toString: String = lookupRoot.path.root.address.toString final class TerminationCallbacks extends Runnable with Awaitable[Unit] { private val lock = new ReentrantGuard diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 78cb370b68..f56862b2fb 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -9,7 +9,7 @@ import akka.util.ReflectiveAccess import scala.util.DynamicVariable import com.typesafe.config.Config import akka.config.ConfigurationException -import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem } +import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem, Address } case class NoSerializerFoundException(m: String) extends AkkaException(m) @@ -27,6 +27,12 @@ object Serialization { */ val currentSystem = new DynamicVariable[ActorSystem](null) + /** + * This holds a reference to the current transport address to be inserted + * into local actor refs during serialization. + */ + val currentTransportAddress = new DynamicVariable[Address](null) + class Settings(val config: Config) { import scala.collection.JavaConverters._ diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index a41ca1984d..aefa770eaf 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -6,7 +6,7 @@ package akka.remote import akka.actor._ import akka.dispatch._ -import akka.event.{ DeathWatch, Logging } +import akka.event.{ DeathWatch, Logging, LoggingAdapter } import akka.event.EventStream import akka.config.ConfigurationException import java.util.concurrent.{ TimeoutException } @@ -22,43 +22,40 @@ class RemoteActorRefProvider( val systemName: String, val settings: ActorSystem.Settings, val eventStream: EventStream, - val scheduler: Scheduler, - _deadLetters: InternalActorRef) extends ActorRefProvider { + val scheduler: Scheduler) extends ActorRefProvider { val remoteSettings = new RemoteSettings(settings.config, systemName) + val deployer = new RemoteDeployer(settings) + + private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, deployer) + + @volatile + private var _log = local.log + def log: LoggingAdapter = _log + + def rootPath = local.rootPath + def locker = local.locker + def deadLetters = local.deadLetters + + val deathWatch = new RemoteDeathWatch(local.deathWatch, this) + + val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize) + + // these are only available after init() def rootGuardian = local.rootGuardian def guardian = local.guardian def systemGuardian = local.systemGuardian def terminationFuture = local.terminationFuture def dispatcher = local.dispatcher - def registerTempActor(actorRef: InternalActorRef, path: ActorPath) = local.registerTempActor(actorRef, path) def unregisterTempActor(path: ActorPath) = local.unregisterTempActor(path) def tempPath() = local.tempPath() def tempContainer = local.tempContainer - val deployer = new RemoteDeployer(settings) - - val transport: RemoteTransport = { - val fqn = remoteSettings.RemoteTransport - // TODO check if this classloader is the right one - ReflectiveAccess.createInstance[RemoteTransport]( - fqn, - Seq(classOf[RemoteSettings] -> remoteSettings), - getClass.getClassLoader) match { - case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) - case Right(remote) ⇒ remote - } - } - - val log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")") - - val rootPath: ActorPath = RootActorPath(transport.address) - - private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath, deployer) - - val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize) + @volatile + private var _transport: RemoteTransport = _ + def transport: RemoteTransport = _transport @volatile private var _serialization: Serialization = _ @@ -72,15 +69,35 @@ class RemoteActorRefProvider( private var _networkEventStream: NetworkEventStream = _ def networkEventStream = _networkEventStream - val deathWatch = new RemoteDeathWatch(local.deathWatch, this) - def init(system: ActorSystemImpl) { local.init(system) - _remoteDaemon = new RemoteSystemDaemon(system, transport.address, rootPath / "remote", rootGuardian, log) + _remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log) + local.registerExtraNames(Map(("remote", remoteDaemon))) + _serialization = SerializationExtension(system) - transport.start(system, this) + _networkEventStream = new NetworkEventStream(system) + system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent]) + + _transport = { + val fqn = remoteSettings.RemoteTransport + // TODO check if this classloader is the right one; hint: this class was loaded by contextClassLoader if that was not null + ReflectiveAccess.createInstance[RemoteTransport]( + fqn, + Seq(classOf[RemoteSettings] -> remoteSettings, + classOf[ActorSystemImpl] -> system, + classOf[RemoteActorRefProvider] -> this), + getClass.getClassLoader) match { + case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) + case Right(remote) ⇒ remote + } + } + + _log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")") + + // this enables reception of remote requests + _transport.start() val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor { def receive = { @@ -90,12 +107,8 @@ class RemoteActorRefProvider( } }), "RemoteClientLifeCycleListener") - _networkEventStream = new NetworkEventStream(system) - - system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent]) system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent]) - local.registerExtraNames(Map(("remote", remoteDaemon))) terminationFuture.onComplete(_ ⇒ transport.shutdown()) } @@ -149,7 +162,7 @@ class RemoteActorRefProvider( case Some(Deploy(_, _, _, RemoteScope(addr))) ⇒ if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment) else { - val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements + val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements useActorOnNode(rpath, props.creator, supervisor) new RemoteActorRef(this, transport, rpath, supervisor, None) } @@ -159,14 +172,13 @@ class RemoteActorRefProvider( } } - def actorFor(path: ActorPath): InternalActorRef = path.root match { - case `rootPath` ⇒ actorFor(rootGuardian, path.elements) - case _ ⇒ new RemoteActorRef(this, transport, path, Nobody, None) - } + def actorFor(path: ActorPath): InternalActorRef = + if (path.address == rootPath.address || path.address == transport.address) actorFor(rootGuardian, path.elements) + else new RemoteActorRef(this, transport, path, Nobody, None) def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match { case ActorPathExtractor(address, elems) ⇒ - if (address == rootPath.address) actorFor(rootGuardian, elems) + if (address == rootPath.address || address == transport.address) actorFor(rootGuardian, elems) else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody, None) case _ ⇒ local.actorFor(ref, path) } @@ -227,7 +239,7 @@ private[akka] class RemoteActorRef private[akka] ( def restart(cause: Throwable): Unit = sendSystemMessage(Recreate(cause)) @throws(classOf[java.io.ObjectStreamException]) - private def writeReplace(): AnyRef = SerializedActorRef(path.toString) + private def writeReplace(): AnyRef = SerializedActorRef(path) } class RemoteDeathWatch(val local: LocalDeathWatch, val provider: RemoteActorRefProvider) extends DeathWatch { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index a002f6fe46..1274e99416 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -18,7 +18,7 @@ case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMs * * It acts as the brain of the remote that responds to system remote events (messages) and undertakes action. */ -class RemoteSystemDaemon(system: ActorSystemImpl, address: Address, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) +class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) extends VirtualPathContainer(system.provider, _path, _parent, _log) { /** @@ -52,7 +52,8 @@ class RemoteSystemDaemon(system: ActorSystemImpl, address: Address, _path: Actor message match { case DaemonMsgCreate(factory, path, supervisor) ⇒ path match { - case ActorPathExtractor(`address`, elems) if elems.nonEmpty && elems.head == "remote" ⇒ + case ActorPathExtractor(address, elems) if elems.nonEmpty && elems.head == "remote" ⇒ + // TODO RK currently the extracted “address” is just ignored, is that okay? // TODO RK canonicalize path so as not to duplicate it always #1446 val subpath = elems.drop(1) val path = this.path / subpath diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 4c671b7e05..6509d19383 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -17,6 +17,8 @@ class RemoteSettings(val config: Config, val systemName: String) { import config._ val RemoteTransport = getString("akka.remote.transport") + val LogReceive = getBoolean("akka.remote.log-received-messages") + val LogSend = getBoolean("akka.remote.log-sent-messages") // AccrualFailureDetector val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold") diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 19fd6c1c43..8d29111672 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -5,12 +5,12 @@ package akka.remote import scala.reflect.BeanProperty - import akka.actor.{ Terminated, LocalRef, InternalActorRef, AutoReceivedMessage, AddressExtractor, Address, ActorSystemImpl, ActorSystem, ActorRef } import akka.dispatch.SystemMessage import akka.event.{ LoggingAdapter, Logging } import akka.remote.RemoteProtocol.{ RemoteMessageProtocol, RemoteControlProtocol, AkkaRemoteProtocol, ActorRefProtocol } import akka.AkkaException +import akka.serialization.Serialization /** * Remote life-cycle events. @@ -199,9 +199,9 @@ abstract class RemoteTransport { def system: ActorSystem /** - * Starts up the remoting + * Start up the transport, i.e. enable incoming connections. */ - def start(system: ActorSystemImpl, provider: RemoteActorRefProvider): Unit + def start(): Unit /** * Shuts down a specific client connected to the supplied remote address returns true if successful @@ -251,6 +251,8 @@ trait RemoteMarshallingOps { def provider: RemoteActorRefProvider + def address: Address + protected def useUntrustedMode: Boolean def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = { @@ -269,7 +271,7 @@ trait RemoteMarshallingOps { * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message. */ def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = { - ActorRefProtocol.newBuilder.setPath(actor.path.toString).build + ActorRefProtocol.newBuilder.setPath(actor.path.toStringWithAddress(address)).build } def createRemoteMessageProtocolBuilder( @@ -278,20 +280,21 @@ trait RemoteMarshallingOps { senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = { val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient)) - messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef])) - if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get)) + Serialization.currentTransportAddress.withValue(address) { + messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef])) + } + messageBuilder } def receiveMessage(remoteMessage: RemoteMessage) { - log.debug("received message {}", remoteMessage) - val remoteDaemon = provider.remoteDaemon remoteMessage.recipient match { case `remoteDaemon` ⇒ + if (provider.remoteSettings.LogReceive) log.debug("received daemon message {}", remoteMessage) remoteMessage.payload match { case m @ (_: DaemonMsg | _: Terminated) ⇒ try remoteDaemon ! m catch { @@ -300,6 +303,7 @@ trait RemoteMarshallingOps { case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender) } case l: LocalRef ⇒ + if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage) remoteMessage.payload match { case msg: SystemMessage ⇒ if (useUntrustedMode) @@ -309,9 +313,11 @@ trait RemoteMarshallingOps { throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor") case m ⇒ l.!(m)(remoteMessage.sender) } - case r: RemoteActorRef ⇒ + case r: RemoteRef ⇒ + if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage) remoteMessage.originalReceiver match { case AddressExtractor(address) if address == provider.transport.address ⇒ + // if it was originally addressed to us but is in fact remote from our point of view (i.e. remote-deployed) r.!(remoteMessage.payload)(remoteMessage.sender) case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r) } diff --git a/akka-remote/src/main/scala/akka/remote/netty/Client.scala b/akka-remote/src/main/scala/akka/remote/netty/Client.scala index 4a9d6607fd..b72fd8b893 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Client.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -58,7 +58,7 @@ abstract class RemoteClient private[akka] ( * Converts the message to the wireprotocol and sends the message across the wire */ def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { - log.debug("Sending message {} from {} to {}", message, senderOption, recipient) + if (netty.remoteSettings.LogSend) log.debug("Sending message {} from {} to {}", message, senderOption, recipient) send((message, senderOption, recipient)) } else { val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", netty, remoteAddress) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 4226f9dffc..7ad10c92ff 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -33,7 +33,7 @@ import akka.event.LoggingAdapter /** * Provides the implementation of the Netty remote support */ -class NettyRemoteTransport(val remoteSettings: RemoteSettings) +class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: ActorSystemImpl, val provider: RemoteActorRefProvider) extends RemoteTransport with RemoteMarshallingOps { val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName) @@ -62,33 +62,11 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings) case ex ⇒ shutdown(); throw ex } - val address = { - server.channel.getLocalAddress match { - case ia: InetSocketAddress ⇒ Address("akka", remoteSettings.systemName, Some(ia.getHostName), Some(ia.getPort)) - case x ⇒ - shutdown() - throw new IllegalArgumentException("unknown address format " + x + ":" + x.getClass) - } - } + val address = Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(settings.Port)) - @volatile - private var _system: ActorSystemImpl = _ - def system = _system + val log = Logging(system.eventStream, "NettyRemoteTransport(" + address + ")") - @volatile - private var _provider: RemoteActorRefProvider = _ - def provider = _provider - - @volatile - private var _log: LoggingAdapter = _ - def log = _log - - def start(system: ActorSystemImpl, provider: RemoteActorRefProvider): Unit = { - _system = system - _provider = provider - _log = Logging(system, "NettyRemoteTransport") - server.start(system) - } + def start(): Unit = server.start() def shutdown(): Unit = { clientsLock.writeLock().lock() diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index a50d3653e8..f695042331 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -44,17 +44,12 @@ class NettyRemoteServer( bootstrap.setPipelineFactory(pipelineFactory) bootstrap.setOption("backlog", settings.Backlog) bootstrap.setOption("tcpNoDelay", true) - bootstrap.setOption("keepAlive", true) + bootstrap.setOption("child.keepAlive", true) bootstrap.setOption("reuseAddress", true) - val channel = bootstrap.bind(new InetSocketAddress(ip, settings.Port)) - - openChannels.add(channel) - - def start(system: ActorSystemImpl) { + def start(): Unit = { + openChannels.add(bootstrap.bind(new InetSocketAddress(ip, settings.Port))) netty.notifyListeners(RemoteServerStarted(netty)) - // TODO uncork the pipeline, which was ... - // TODO ... corked before in order not to allow anything through before init is complete } def shutdown() { diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala index 9596f1b083..4e31720b2b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -38,7 +38,15 @@ class NettySettings(config: Config, val systemName: String) { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } - val Port = getInt("port") + val Port = getInt("port") match { + case 0 ⇒ + try { + val s = new java.net.ServerSocket(0) + try s.getLocalPort finally s.close() + } catch { case e ⇒ throw new ConfigurationException("Unable to obtain random port", e) } + case other ⇒ other + } + val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) val Backlog = getInt("backlog") diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index ccfdea0189..1c7d4b2602 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -1,6 +1,6 @@ package akka.remote -import akka.actor.{ Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.testkit._ import akka.dispatch.Await import akka.pattern.ask @@ -10,7 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ sender ! self.path.address.hostPort + case "identify" ⇒ sender ! self } } @@ -53,7 +53,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(nodeConfigs(1) val actor = system.actorOf(Props[SomeActor], "service-hello") actor.isInstanceOf[RemoteActorRef] must be(true) - Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0)) + Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0)) barrier("done") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index 59e6d4e850..a91b203707 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -1,6 +1,6 @@ package akka.remote -import akka.actor.{ Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.testkit._ import akka.dispatch.Await import akka.pattern.ask @@ -10,7 +10,7 @@ object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ sender ! self.path.address.hostPort + case "identify" ⇒ sender ! self } } @@ -53,7 +53,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(NewRemoteActorMultiJvmS barrier("start") val actor = system.actorOf(Props[SomeActor], "service-hello") - Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0)) + Await.result(actor ? "identify", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort must equal(akkaSpec(0)) barrier("done") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index 89d5da3bd2..f895708294 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -1,6 +1,6 @@ package akka.remote -import akka.actor.{ Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await @@ -10,7 +10,7 @@ object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp override def NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! self.path.address.hostPort + case "hit" ⇒ sender ! self case "end" ⇒ context.stop(self) } } @@ -89,7 +89,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RandomRoutedRe for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { - val nodeName = Await.result(actor ? "hit", timeout.duration).toString + val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort replies = replies + (nodeName -> (replies(nodeName) + 1)) } } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 0b3db1c8bd..21c7e4cf64 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -1,6 +1,6 @@ package akka.remote -import akka.actor.{ Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await @@ -11,7 +11,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJ class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! self.path.address.hostPort + case "hit" ⇒ sender ! self case "end" ⇒ context.stop(self) } } @@ -90,7 +90,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RoundRobin for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { - val nodeName = Await.result(actor ? "hit", timeout.duration).toString + val nodeName = Await.result(actor ? "hit", timeout.duration).asInstanceOf[ActorRef].path.address.hostPort replies = replies + (nodeName -> (replies(nodeName) + 1)) } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala index d2cb625469..0eb3caa8e4 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala @@ -1,6 +1,6 @@ package akka.remote -import akka.actor.{ Actor, Props } +import akka.actor.{ Actor, ActorRef, Props } import akka.routing._ import akka.testkit._ import akka.util.duration._ @@ -9,7 +9,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMul override def NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ sender ! self.path.address.hostPort + case "hit" ⇒ sender ! self case "end" ⇒ context.stop(self) } } @@ -89,7 +89,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter } val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { - case name: String ⇒ (name, 1) + case ref: ActorRef ⇒ (ref.asInstanceOf[ActorRef].path.address.hostPort, 1) }).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) { case (m, (n, c)) ⇒ m + (n -> (m(n) + c)) } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index 367c6faf57..ac3833130c 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -11,7 +11,7 @@ import com.typesafe.config._ object RemoteRouterSpec { class Echo extends Actor { def receive = { - case _ ⇒ sender ! self.path + case _ ⇒ sender ! self } } } @@ -56,26 +56,26 @@ akka { "deploy its children on remote host driven by configuration" in { val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub") router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1" router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2" } "deploy its children on remote host driven by programatic definition" in { val router = system.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2), Seq("akka://remote_sys@localhost:12346"))), "blub2") router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c1" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c1" router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c2" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c2" } "deploy dynamic resizable number of children on remote host driven by configuration" in { val router = system.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub") router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c1" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c1" router ! "" - expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c2" + expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c2" } } From 6072099f7159cbfc05b7c10f9150fb3276c5a043 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bone=CC=81r?= Date: Fri, 27 Jan 2012 12:41:47 +0100 Subject: [PATCH 03/13] Added section about what happened to Cloudy Akka --- akka-docs/intro/what-is-akka.rst | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/akka-docs/intro/what-is-akka.rst b/akka-docs/intro/what-is-akka.rst index 6cc7c591c0..80a8b0e62c 100644 --- a/akka-docs/intro/what-is-akka.rst +++ b/akka-docs/intro/what-is-akka.rst @@ -82,6 +82,20 @@ Akka can be used in two different ways See the :ref:`deployment-scenarios` for details. +What happened to Cloudy Akka? +============================= + +The commercial offering was earlier referred to as Cloudy Akka. This offering +consisted of two things: + +- Cluster support for Akka +- Monitoring & Management (formerly called Atmos) + +Cloudy Akka have been discontinued and the Cluster support is now being moved into the +Open Source version of Akka (the upcoming Akka 2.1), while the Monitoring & Management +(Atmos) is now rebranded into Typesafe Console and is part of the commercial subscription +for the Typesafe Stack (see below for details). + Typesafe Stack ============== @@ -100,6 +114,7 @@ Typesafe Console On top of the Typesafe Stack we have also have commercial product called Typesafe Console which provides the following features: +#. Slick Web UI with real-time view into the system #. Management through Dashboard, JMX and REST #. Dapper-style tracing of messages across components and remote nodes #. Real-time statistics From ac1ee9ae919d34252acf87f264b05e5b422bc8fe Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 27 Jan 2012 13:30:43 +0100 Subject: [PATCH 04/13] rework use of ClassLoaders, see #1736 --- .../main/scala/akka/actor/ActorSystem.scala | 14 ++++---- .../akka/dispatch/AbstractDispatcher.scala | 5 +-- .../akka/dispatch/ThreadPoolBuilder.scala | 2 ++ .../akka/serialization/Serialization.scala | 4 +-- .../akka/actor/mailbox/DurableMailbox.scala | 2 +- .../actor/mailbox/BSONSerialization.scala | 2 +- .../scala/akka/remote/MessageSerializer.scala | 4 +-- .../akka/remote/RemoteActorRefProvider.scala | 34 +++++++++---------- .../akka/remote/RemoteConnectionManager.scala | 2 +- .../scala/akka/remote/RemoteTransport.scala | 7 ++-- .../main/scala/akka/remote/netty/Client.scala | 5 ++- .../remote/netty/NettyRemoteSupport.scala | 28 ++++++--------- .../main/scala/akka/remote/netty/Server.scala | 12 +++---- 13 files changed, 54 insertions(+), 67 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 50b69b9ae8..81e0761bb0 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -332,7 +332,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten import ActorSystem._ final val settings: Settings = new Settings(applicationConfig, name) - final val threadFactory: MonitorableThreadFactory = new MonitorableThreadFactory(name, settings.Daemonicity) + + final val threadFactory: MonitorableThreadFactory = + MonitorableThreadFactory(name, settings.Daemonicity, Option(Thread.currentThread.getContextClassLoader)) def logConfiguration(): Unit = log.info(settings.toString) @@ -391,10 +393,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten classOf[EventStream] -> eventStream, classOf[Scheduler] -> scheduler) - val loader = Thread.currentThread.getContextClassLoader match { - case null ⇒ getClass.getClassLoader - case l ⇒ l - } + val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, loader) match { case Left(e) ⇒ throw e @@ -534,9 +533,10 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten private def loadExtensions() { import scala.collection.JavaConversions._ + val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse this.getClass.getClassLoader settings.config.getStringList("akka.extensions") foreach { fqcn ⇒ - import ReflectiveAccess._ - getObjectFor[AnyRef](fqcn).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match { + import ReflectiveAccess.{ getObjectFor, createInstance, noParams, noArgs } + getObjectFor[AnyRef](fqcn, loader).fold(_ ⇒ createInstance[AnyRef](fqcn, noParams, noArgs), Right(_)) match { case Right(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()); case Right(p: ExtensionId[_]) ⇒ registerExtension(p); case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 29be04fe40..2f6b330cc8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -321,8 +321,9 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit BoundedMailbox(capacity, duration) } case fqcn ⇒ - val constructorSignature = Array[Class[_]](classOf[Config]) - ReflectiveAccess.createInstance[MailboxType](fqcn, constructorSignature, Array[AnyRef](config)) match { + val args = Seq(classOf[Config] -> config) + val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader + ReflectiveAccess.createInstance[MailboxType](fqcn, args, loader) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index f8927f667a..8998ccca03 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -159,6 +159,7 @@ object MonitorableThreadFactory { case class MonitorableThreadFactory(name: String, daemonic: Boolean, + contextClassLoader: Option[ClassLoader], exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing) extends ThreadFactory { protected val counter = new AtomicLong @@ -167,6 +168,7 @@ case class MonitorableThreadFactory(name: String, val t = new Thread(runnable, name + counter.incrementAndGet()) t.setUncaughtExceptionHandler(exceptionHandler) t.setDaemon(daemonic) + contextClassLoader foreach (t.setContextClassLoader(_)) t } } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index f56862b2fb..750b2e5c35 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -81,10 +81,10 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]], - classLoader: Option[ClassLoader]): Either[Exception, AnyRef] = + classLoader: ClassLoader): Either[Exception, AnyRef] = try { currentSystem.withValue(system) { - Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, classLoader)) + Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz, Some(classLoader))) } } catch { case e: Exception ⇒ Left(e) } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index e00520d92c..6638c380bf 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -44,7 +44,7 @@ trait DurableMessageSerialization { def deserializeActorRef(refProtocol: ActorRefProtocol): ActorRef = owner.system.actorFor(refProtocol.getPath) val durableMessage = RemoteMessageProtocol.parseFrom(bytes) - val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage) + val message = MessageSerializer.deserialize(owner.system, durableMessage.getMessage, getClass.getClassLoader) val sender = deserializeActorRef(durableMessage.getSender) new Envelope(message, sender)(owner.system) diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 470993fdf3..217f46d6ec 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -67,7 +67,7 @@ class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObjec val doc = deserializer.decodeAndFetch(in).asInstanceOf[BSONDocument] system.log.debug("Deserializing a durable message from MongoDB: {}", doc) val msgData = MessageProtocol.parseFrom(doc.as[org.bson.types.Binary]("message").getData) - val msg = MessageSerializer.deserialize(system, msgData) + val msg = MessageSerializer.deserialize(system, msgData, getClass.getClassLoader) val ownerPath = doc.as[String]("ownerPath") val senderPath = doc.as[String]("senderPath") val sender = systemImpl.actorFor(senderPath) diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 878023c064..5301f2bdd0 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -12,11 +12,11 @@ import akka.util.ReflectiveAccess object MessageSerializer { - def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: Option[ClassLoader] = None): AnyRef = { + def deserialize(system: ActorSystem, messageProtocol: MessageProtocol, classLoader: ClassLoader): AnyRef = { val clazz = if (messageProtocol.hasMessageManifest) { Option(ReflectiveAccess.getClassFor[AnyRef]( messageProtocol.getMessageManifest.toStringUtf8, - classLoader.getOrElse(ReflectiveAccess.loader)) match { + classLoader) match { case Left(e) ⇒ throw e case Right(r) ⇒ r }) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index aefa770eaf..61b675fe55 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -82,16 +82,15 @@ class RemoteActorRefProvider( _transport = { val fqn = remoteSettings.RemoteTransport - // TODO check if this classloader is the right one; hint: this class was loaded by contextClassLoader if that was not null - ReflectiveAccess.createInstance[RemoteTransport]( - fqn, - Seq(classOf[RemoteSettings] -> remoteSettings, - classOf[ActorSystemImpl] -> system, - classOf[RemoteActorRefProvider] -> this), - getClass.getClassLoader) match { - case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) - case Right(remote) ⇒ remote - } + val args = Seq( + classOf[RemoteSettings] -> remoteSettings, + classOf[ActorSystemImpl] -> system, + classOf[RemoteActorRefProvider] -> this) + + ReflectiveAccess.createInstance[RemoteTransport](fqn, args, getClass.getClassLoader) match { + case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem) + case Right(remote) ⇒ remote + } } _log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")") @@ -164,7 +163,7 @@ class RemoteActorRefProvider( else { val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements useActorOnNode(rpath, props.creator, supervisor) - new RemoteActorRef(this, transport, rpath, supervisor, None) + new RemoteActorRef(this, transport, rpath, supervisor) } case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment) @@ -174,12 +173,12 @@ class RemoteActorRefProvider( def actorFor(path: ActorPath): InternalActorRef = if (path.address == rootPath.address || path.address == transport.address) actorFor(rootGuardian, path.elements) - else new RemoteActorRef(this, transport, path, Nobody, None) + else new RemoteActorRef(this, transport, path, Nobody) def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match { case ActorPathExtractor(address, elems) ⇒ if (address == rootPath.address || address == transport.address) actorFor(rootGuardian, elems) - else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody, None) + else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody) case _ ⇒ local.actorFor(ref, path) } @@ -208,8 +207,7 @@ private[akka] class RemoteActorRef private[akka] ( val provider: RemoteActorRefProvider, remote: RemoteTransport, val path: ActorPath, - val getParent: InternalActorRef, - loader: Option[ClassLoader]) + val getParent: InternalActorRef) extends InternalActorRef with RemoteRef { def getChild(name: Iterator[String]): InternalActorRef = { @@ -217,7 +215,7 @@ private[akka] class RemoteActorRef private[akka] ( s.headOption match { case None ⇒ this case Some("..") ⇒ getParent getChild name - case _ ⇒ new RemoteActorRef(provider, remote, path / s, Nobody, loader) + case _ ⇒ new RemoteActorRef(provider, remote, path / s, Nobody) } } @@ -226,9 +224,9 @@ private[akka] class RemoteActorRef private[akka] ( def isTerminated: Boolean = !running - def sendSystemMessage(message: SystemMessage): Unit = remote.send(message, None, this, loader) + def sendSystemMessage(message: SystemMessage): Unit = remote.send(message, None, this) - override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), this, loader) + override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), this) def suspend(): Unit = sendSystemMessage(Suspend()) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index 9cd974a274..fd2a9135d7 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -146,5 +146,5 @@ class RemoteConnectionManager( } private[remote] def newConnection(remoteAddress: Address, actorPath: ActorPath) = - new RemoteActorRef(remote, remote.transport, actorPath, Nobody, None) + new RemoteActorRef(remote, remote.transport, actorPath, Nobody) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index 8d29111672..c2bc63457a 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -217,8 +217,7 @@ abstract class RemoteTransport { protected[akka] def send(message: Any, senderOption: Option[ActorRef], - recipient: RemoteActorRef, - loader: Option[ClassLoader]): Unit + recipient: RemoteActorRef): Unit protected[akka] def notifyListeners(message: RemoteLifeCycleEvent): Unit = { system.eventStream.publish(message) @@ -228,7 +227,7 @@ abstract class RemoteTransport { override def toString = address.toString } -class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, classLoader: Option[ClassLoader]) { +class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl) { def originalReceiver = input.getRecipient.getPath @@ -238,7 +237,7 @@ class RemoteMessage(input: RemoteMessageProtocol, system: ActorSystemImpl, class lazy val recipient: InternalActorRef = system.provider.actorFor(system.provider.rootGuardian, originalReceiver) - lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, classLoader) + lazy val payload: AnyRef = MessageSerializer.deserialize(system, input.getMessage, getClass.getClassLoader) override def toString = "RemoteMessage: " + payload + " to " + recipient + "<+{" + originalReceiver + "} from " + sender } diff --git a/akka-remote/src/main/scala/akka/remote/netty/Client.scala b/akka-remote/src/main/scala/akka/remote/netty/Client.scala index b72fd8b893..2d7d6218f7 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Client.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -100,8 +100,7 @@ abstract class RemoteClient private[akka] ( class ActiveRemoteClient private[akka] ( netty: NettyRemoteTransport, remoteAddress: Address, - localAddress: Address, - val loader: Option[ClassLoader] = None) + localAddress: Address) extends RemoteClient(netty, remoteAddress) { import netty.settings @@ -253,7 +252,7 @@ class ActiveRemoteClientHandler( } case arp: AkkaRemoteProtocol if arp.hasMessage ⇒ - client.netty.receiveMessage(new RemoteMessage(arp.getMessage, client.netty.system, client.loader)) + client.netty.receiveMessage(new RemoteMessage(arp.getMessage, client.netty.system)) case other ⇒ throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.netty, client.remoteAddress) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 7ad10c92ff..28dbdb3df6 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,31 +4,24 @@ package akka.remote.netty -import java.net.{ UnknownHostException, InetAddress } import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock 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, ChannelFutureListener, ChannelFuture, Channel } +import org.jboss.netty.channel.{ ChannelHandlerContext, Channel } import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder } import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor import org.jboss.netty.util.HashedWheelTimer -import akka.actor.{ ActorSystemImpl, ActorRef, simpleName } + +import akka.actor.{ Address, ActorSystemImpl, ActorRef } import akka.dispatch.MonitorableThreadFactory import akka.event.Logging import akka.remote.RemoteProtocol.AkkaRemoteProtocol -import akka.remote.{ RemoteTransport, RemoteMarshallingOps, RemoteClientWriteFailed, RemoteClientException, RemoteClientError, RemoteActorRef } -import akka.util.Switch -import akka.AkkaException -import com.typesafe.config.Config -import akka.remote.RemoteSettings -import akka.actor.Address -import java.net.InetSocketAddress -import akka.remote.RemoteActorRefProvider -import akka.remote.RemoteActorRefProvider -import akka.event.LoggingAdapter +import akka.remote.{ RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef } /** * Provides the implementation of the Netty remote support @@ -38,7 +31,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor val settings = new NettySettings(remoteSettings.config.getConfig("akka.remote.netty"), remoteSettings.systemName) - val threadFactory = new MonitorableThreadFactory("NettyRemoteTransport", settings.Daemonic) + val threadFactory = new MonitorableThreadFactory("NettyRemoteTransport", settings.Daemonic, Some(getClass.getClassLoader)) val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory) val executor = new OrderedMemoryAwareThreadPoolExecutor( @@ -58,7 +51,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor override protected def useUntrustedMode = remoteSettings.UntrustedMode - val server = try new NettyRemoteServer(this, Some(getClass.getClassLoader)) catch { + val server = try new NettyRemoteServer(this) catch { case ex ⇒ shutdown(); throw ex } @@ -94,8 +87,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor protected[akka] def send( message: Any, senderOption: Option[ActorRef], - recipient: RemoteActorRef, - loader: Option[ClassLoader]): Unit = { + recipient: RemoteActorRef): Unit = { val recipientAddress = recipient.path.address @@ -112,7 +104,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor //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, loader) + val client = new ActiveRemoteClient(this, recipientAddress, address) client.connect() remoteClients += recipientAddress -> client client diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index f695042331..749e01d63f 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -21,9 +21,7 @@ import java.net.InetAddress import akka.actor.ActorSystemImpl import org.jboss.netty.channel.ChannelLocal -class NettyRemoteServer( - val netty: NettyRemoteTransport, - val loader: Option[ClassLoader]) { +class NettyRemoteServer(val netty: NettyRemoteTransport) { import netty.settings @@ -40,7 +38,7 @@ class NettyRemoteServer( // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") - val pipelineFactory = new RemoteServerPipelineFactory(openChannels, executionHandler, loader, netty) + val pipelineFactory = new RemoteServerPipelineFactory(openChannels, executionHandler, netty) bootstrap.setPipelineFactory(pipelineFactory) bootstrap.setOption("backlog", settings.Backlog) bootstrap.setOption("tcpNoDelay", true) @@ -79,7 +77,6 @@ class NettyRemoteServer( class RemoteServerPipelineFactory( val openChannels: ChannelGroup, val executionHandler: ExecutionHandler, - val loader: Option[ClassLoader], val netty: NettyRemoteTransport) extends ChannelPipelineFactory { import netty.settings @@ -91,7 +88,7 @@ class RemoteServerPipelineFactory( val messageEnc = new RemoteMessageEncoder(netty) val authenticator = if (settings.RequireCookie) new RemoteServerAuthenticationHandler(settings.SecureCookie) :: Nil else Nil - val remoteServer = new RemoteServerHandler(openChannels, loader, netty) + val remoteServer = new RemoteServerHandler(openChannels, netty) val stages: List[ChannelHandler] = lenDec :: messageDec :: lenPrep :: messageEnc :: executionHandler :: authenticator ::: remoteServer :: Nil new StaticChannelPipeline(stages: _*) } @@ -131,7 +128,6 @@ object ChannelLocalSystem extends ChannelLocal[ActorSystemImpl] { @ChannelHandler.Sharable class RemoteServerHandler( val openChannels: ChannelGroup, - val applicationLoader: Option[ClassLoader], val netty: NettyRemoteTransport) extends SimpleChannelUpstreamHandler { import netty.settings @@ -164,7 +160,7 @@ class RemoteServerHandler( override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = try { event.getMessage match { case remote: AkkaRemoteProtocol if remote.hasMessage ⇒ - netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system, applicationLoader)) + netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system)) case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒ val instruction = remote.getInstruction From 4482f14650d4cf567591684c267938b096aa3fb4 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 27 Jan 2012 15:21:05 +0100 Subject: [PATCH 05/13] properly fix the port==0 issue and use it in more tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - add documentation to RemoteTransport interface what is expected wrt. address’ availability before, during and after start() --- .../scala/akka/remote/RemoteTransport.scala | 8 +++++++ .../remote/netty/NettyRemoteSupport.scala | 21 +++++++++++++++--- .../main/scala/akka/remote/netty/Server.scala | 17 +++++++++++++- .../scala/akka/remote/netty/Settings.scala | 9 +------- .../akka/remote/RemoteDeathWatchSpec.scala | 2 +- .../scala/akka/remote/RemoteRouterSpec.scala | 22 +++++++++---------- 6 files changed, 55 insertions(+), 24 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala index c2bc63457a..3fbe5913b2 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteTransport.scala @@ -182,6 +182,14 @@ class RemoteClientException private[akka] ( class RemoteTransportException(message: String, cause: Throwable) extends AkkaException(message, cause) +/** + * The remote transport is responsible for sending and receiving messages. + * Each transport has an address, which it should provide in + * Serialization.currentTransportAddress (thread-local) while serializing + * actor references (which might also be part of messages). This address must + * be available (i.e. fully initialized) by the time the first message is + * received or when the start() method returns, whatever happens first. + */ abstract class RemoteTransport { /** * Shuts down the remoting diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 28dbdb3df6..4179beeb3d 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,6 +4,8 @@ package akka.remote.netty +import java.net.InetSocketAddress +import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.concurrent.Executors @@ -21,7 +23,7 @@ import akka.actor.{ Address, ActorSystemImpl, ActorRef } import akka.dispatch.MonitorableThreadFactory import akka.event.Logging import akka.remote.RemoteProtocol.AkkaRemoteProtocol -import akka.remote.{ RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef } +import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef } /** * Provides the implementation of the Netty remote support @@ -55,11 +57,24 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor case ex ⇒ shutdown(); throw ex } - val address = Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(settings.Port)) + // the address is set in start() or from the RemoteServerHandler, whichever comes first + private val _address = new AtomicReference[Address] + private[akka] def setAddressFromChannel(ch: Channel) = { + val addr = ch.getLocalAddress match { + case sa: InetSocketAddress ⇒ sa + case x ⇒ throw new RemoteTransportException("unknown local address type " + x.getClass, null) + } + _address.compareAndSet(null, Address("akka", remoteSettings.systemName, Some(settings.Hostname), Some(addr.getPort))) + } + + def address = _address.get val log = Logging(system.eventStream, "NettyRemoteTransport(" + address + ")") - def start(): Unit = server.start() + def start(): Unit = { + server.start() + setAddressFromChannel(server.channel) + } def shutdown(): Unit = { clientsLock.writeLock().lock() diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index 749e01d63f..9ebeb8f3e8 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -20,6 +20,7 @@ import akka.actor.Address import java.net.InetAddress import akka.actor.ActorSystemImpl import org.jboss.netty.channel.ChannelLocal +import org.jboss.netty.channel.ChannelEvent class NettyRemoteServer(val netty: NettyRemoteTransport) { @@ -45,8 +46,12 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) { bootstrap.setOption("child.keepAlive", true) bootstrap.setOption("reuseAddress", true) + @volatile + private[akka] var channel: Channel = _ + def start(): Unit = { - openChannels.add(bootstrap.bind(new InetSocketAddress(ip, settings.Port))) + channel = bootstrap.bind(new InetSocketAddress(ip, settings.Port)) + openChannels.add(channel) netty.notifyListeners(RemoteServerStarted(netty)) } @@ -132,6 +137,16 @@ class RemoteServerHandler( import netty.settings + private var addressToSet = true + + override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { + if (addressToSet) { + netty.setAddressFromChannel(event.getChannel) + addressToSet = false + } + super.handleUpstream(ctx, event) + } + /** * ChannelOpen overridden to store open channels for a clean postStop of a node. * If a channel is closed before, it is automatically removed from the open channels group. diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala index 4e31720b2b..123e2d53dc 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -38,14 +38,7 @@ class NettySettings(config: Config, val systemName: String) { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } - val Port = getInt("port") match { - case 0 ⇒ - try { - val s = new java.net.ServerSocket(0) - try s.getLocalPort finally s.close() - } catch { case e ⇒ throw new ConfigurationException("Unable to obtain random port", e) } - case other ⇒ other - } + val Port = getInt("port") val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala index 942ca0cd8d..392cee216b 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeathWatchSpec.scala @@ -18,7 +18,7 @@ akka { } remote.netty { hostname = "127.0.0.1" - port = 2665 + port = 0 } } """)) with ImplicitSender with DefaultTimeout with DeathWatchSpec { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index ac3833130c..26c28860f7 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -22,13 +22,13 @@ akka { actor.provider = "akka.remote.RemoteActorRefProvider" remote.netty { hostname = localhost - port = 12345 + port = 0 } actor.deployment { /blub { router = round-robin nr-of-instances = 2 - target.nodes = ["akka://remote_sys@localhost:12346"] + target.nodes = ["akka://remote_sys@localhost:12347"] } /elastic-blub { router = round-robin @@ -36,7 +36,7 @@ akka { lower-bound = 2 upper-bound = 3 } - target.nodes = ["akka://remote_sys@localhost:12346"] + target.nodes = ["akka://remote_sys@localhost:12347"] } } } @@ -44,7 +44,7 @@ akka { import RemoteRouterSpec._ - val conf = ConfigFactory.parseString("akka.remote.netty.port=12346").withFallback(system.settings.config) + val conf = ConfigFactory.parseString("akka.remote.netty.port=12347").withFallback(system.settings.config) val other = ActorSystem("remote_sys", conf) override def atTermination() { @@ -56,26 +56,26 @@ akka { "deploy its children on remote host driven by configuration" in { val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub") router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" } "deploy its children on remote host driven by programatic definition" in { val router = system.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2), - Seq("akka://remote_sys@localhost:12346"))), "blub2") + Seq("akka://remote_sys@localhost:12347"))), "blub2") router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c1" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub2/c2" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" } "deploy dynamic resizable number of children on remote host driven by configuration" in { val router = system.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub") router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c1" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" router ! "" - expectMsgType[ActorRef].path.toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/elastic-blub/c2" + expectMsgType[ActorRef].path.address.toString must be === "akka://remote_sys@localhost:12347" } } From 2bebf29c1c070c004ca13d810f22f3acc722e9ee Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 27 Jan 2012 15:39:29 +0100 Subject: [PATCH 06/13] implement ActorPath.fromString, see #1697 --- .../scala/akka/actor/ActorLookupSpec.scala | 22 +++++++++++++++++++ .../src/main/scala/akka/actor/ActorPath.scala | 6 +++++ 2 files changed, 28 insertions(+) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index 3b18c742ce..626e413ec8 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -7,6 +7,7 @@ import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await import akka.pattern.ask +import java.net.MalformedURLException object ActorLookupSpec { @@ -287,4 +288,25 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout { } + "An ActorPath" must { + + "support parsing its String rep" in { + val path = system.actorFor("user").path + ActorPath.fromString(path.toString) must be(path) + } + + "support parsing remote paths" in { + val remote = "akka://sys@host:1234/some/ref" + ActorPath.fromString(remote).toString must be(remote) + } + + "throw exception upon malformed paths" in { + intercept[MalformedURLException] { ActorPath.fromString("") } + intercept[MalformedURLException] { ActorPath.fromString("://hallo") } + intercept[MalformedURLException] { ActorPath.fromString("s://dd@:12") } + intercept[MalformedURLException] { ActorPath.fromString("s://dd@h:hd") } + } + + } + } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 99be8bae0e..5ba0ae4600 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -3,6 +3,7 @@ */ package akka.actor import scala.annotation.tailrec +import java.net.MalformedURLException object ActorPath { def split(s: String): List[String] = { @@ -16,6 +17,11 @@ object ActorPath { rec(s.length, Nil) } + def fromString(s: String): ActorPath = s match { + case ActorPathExtractor(addr, elems) ⇒ RootActorPath(addr) / elems + case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s) + } + val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r } From 43ec4c5895d1e5c4a5d33957385265b68dfe5cfc Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 07:44:10 +0100 Subject: [PATCH 07/13] Remove akka.conf from pi sample. --- .../akka-tutorial-first/src/main/resources/akka.conf | 6 ------ 1 file changed, 6 deletions(-) delete mode 100644 akka-tutorials/akka-tutorial-first/src/main/resources/akka.conf diff --git a/akka-tutorials/akka-tutorial-first/src/main/resources/akka.conf b/akka-tutorials/akka-tutorial-first/src/main/resources/akka.conf deleted file mode 100644 index 17517bccbe..0000000000 --- a/akka-tutorials/akka-tutorial-first/src/main/resources/akka.conf +++ /dev/null @@ -1,6 +0,0 @@ -akka.actor.deployment { - /user/master/pi { - router = round-robin - nr-of-instances = 10 - } -} From 7201a7e24b413ff9b14ea61f0da0a33f9c9dbbd3 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 07:44:38 +0100 Subject: [PATCH 08/13] Added name of the listener actor in pi sample --- .../src/main/java/akka/tutorial/first/java/Pi.java | 2 +- akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index d1ad063063..2eaddcd40c 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -180,7 +180,7 @@ public class Pi { ActorSystem system = ActorSystem.create("PiSystem"); // create the result listener, which will print the result and shutdown the system - final ActorRef listener = system.actorOf(new Props(Listener.class)); + final ActorRef listener = system.actorOf(new Props(Listener.class), "listener"); // create the master ActorRef master = system.actorOf(new Props(new UntypedActorFactory() { diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index 578a64445c..19fa23544c 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -95,7 +95,7 @@ object Pi extends App { val system = ActorSystem("PiSystem") // create the result listener, which will print the result and shutdown the system - val listener = system.actorOf(Props[Listener]) + val listener = system.actorOf(Props[Listener], name = "listener") // create the master val master = system.actorOf(Props(new Master( From 40c351e5bec8d657542ff7805a6dbbfba3723535 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 11:14:55 +0100 Subject: [PATCH 09/13] Describe akka-sbt-plugin AkkaKernelPlugin. See #1742 --- akka-docs/modules/microkernel.rst | 33 ++++++++++- akka-sbt-plugin/sample/project/Build.scala | 58 +++++++++++++++++++ .../sample/project/build.properties | 1 + akka-sbt-plugin/sample/project/plugins.sbt | 3 + .../sample/src/main/config/application.conf | 4 ++ .../sample/src/main/config/logback.xml | 15 +++++ .../sample/src/main/scala/HelloKernel.scala | 37 ++++++++++++ .../src/main/scala/AkkaKernelPlugin.scala | 4 +- 8 files changed, 152 insertions(+), 3 deletions(-) create mode 100644 akka-sbt-plugin/sample/project/Build.scala create mode 100644 akka-sbt-plugin/sample/project/build.properties create mode 100644 akka-sbt-plugin/sample/project/plugins.sbt create mode 100644 akka-sbt-plugin/sample/src/main/config/application.conf create mode 100644 akka-sbt-plugin/sample/src/main/config/logback.xml create mode 100644 akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala diff --git a/akka-docs/modules/microkernel.rst b/akka-docs/modules/microkernel.rst index 7325cefa70..ec6eabe3ef 100644 --- a/akka-docs/modules/microkernel.rst +++ b/akka-docs/modules/microkernel.rst @@ -24,7 +24,7 @@ command (on a unix-based system): .. code-block:: none - bin/akka sample.kernel.hello.HelloKernel + bin/start sample.kernel.hello.HelloKernel Use ``Ctrl-C`` to interrupt and exit the microkernel. @@ -34,3 +34,34 @@ The code for the Hello Kernel example (see the ``HelloKernel`` class for an exam of creating a Bootable): .. includecode:: ../../akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala + + +Distribution of microkernel application +--------------------------------------- + +To make a distribution package of the microkernel and your application the ``akka-sbt-plugin`` provides +``AkkaKernelPlugin``. It creates the directory structure, with jar files, configuration files and +start scripts. + +To use the sbt plugin you define it in your ``project/plugins.sbt``: + +.. includecode:: ../../akka-sbt-plugin/sample/project/plugins.sbt + +Then you add it to the settings of your ``project/Build.scala``. It is also important that you add the ``akka-kernel`` dependency. +This is an example of a complete sbt build file: + +.. includecode:: ../../akka-sbt-plugin/sample/project/Build.scala + +Run the plugin with sbt:: + + > dist + > dist:clean + +There are several settings that can be defined: + +* ``outputDirectory`` - destination directory of the package, default ``target/dist`` +* ``distJvmOptions`` - JVM parameters to be used in the start script +* ``configSourceDirs`` - Configuration files are copied from these directories, default ``src/config``, ``src/main/config``, ``src/main/resources`` +* ``distMainClass`` - Kernel main class to use in start script +* ``libFilter`` - Filter of dependency jar files +* ``additionalLibs`` - Additional dependency jar files diff --git a/akka-sbt-plugin/sample/project/Build.scala b/akka-sbt-plugin/sample/project/Build.scala new file mode 100644 index 0000000000..0fc391775b --- /dev/null +++ b/akka-sbt-plugin/sample/project/Build.scala @@ -0,0 +1,58 @@ + +import sbt._ +import Keys._ +import akka.sbt.AkkaKernelPlugin +import akka.sbt.AkkaKernelPlugin.{ Dist, outputDirectory, distJvmOptions} + +object HelloKernelBuild extends Build { + val Organization = "akka.sample" + val Version = "2.0-SNAPSHOT" + val ScalaVersion = "2.9.1" + + lazy val HelloKernel = Project( + id = "hello-kernel", + base = file("."), + settings = defaultSettings ++ AkkaKernelPlugin.distSettings ++ Seq( + libraryDependencies ++= Dependencies.helloKernel, + distJvmOptions in Dist := "-Xms256M -Xmx1024M", + outputDirectory in Dist := file("target/hello-dist") + ) + ) + + lazy val buildSettings = Defaults.defaultSettings ++ Seq( + organization := Organization, + version := Version, + scalaVersion := ScalaVersion, + crossPaths := false, + organizationName := "Typesafe Inc.", + organizationHomepage := Some(url("http://www.typesafe.com")) + ) + + lazy val defaultSettings = buildSettings ++ Seq( + resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/", + + // compile options + scalacOptions ++= Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"), + javacOptions ++= Seq("-Xlint:unchecked", "-Xlint:deprecation") + + ) +} + +object Dependencies { + import Dependency._ + + val helloKernel = Seq( + akkaKernel, akkaSlf4j, logback + ) +} + +object Dependency { + // Versions + object V { + val Akka = "2.0-M3" + } + + val akkaKernel = "com.typesafe.akka" % "akka-kernel" % V.Akka + val akkaSlf4j = "com.typesafe.akka" % "akka-slf4j" % V.Akka + val logback = "ch.qos.logback" % "logback-classic" % "1.0.0" +} diff --git a/akka-sbt-plugin/sample/project/build.properties b/akka-sbt-plugin/sample/project/build.properties new file mode 100644 index 0000000000..f4ff7a5afa --- /dev/null +++ b/akka-sbt-plugin/sample/project/build.properties @@ -0,0 +1 @@ +sbt.version=0.11.2 diff --git a/akka-sbt-plugin/sample/project/plugins.sbt b/akka-sbt-plugin/sample/project/plugins.sbt new file mode 100644 index 0000000000..afba4f3ad2 --- /dev/null +++ b/akka-sbt-plugin/sample/project/plugins.sbt @@ -0,0 +1,3 @@ +resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" + +addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.0-M3") diff --git a/akka-sbt-plugin/sample/src/main/config/application.conf b/akka-sbt-plugin/sample/src/main/config/application.conf new file mode 100644 index 0000000000..e8271a1081 --- /dev/null +++ b/akka-sbt-plugin/sample/src/main/config/application.conf @@ -0,0 +1,4 @@ +akka { + loglevel = INFO + event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] +} \ No newline at end of file diff --git a/akka-sbt-plugin/sample/src/main/config/logback.xml b/akka-sbt-plugin/sample/src/main/config/logback.xml new file mode 100644 index 0000000000..bddac0313d --- /dev/null +++ b/akka-sbt-plugin/sample/src/main/config/logback.xml @@ -0,0 +1,15 @@ + + + + + + + %date{ISO8601} %-5level %X{akkaSource} %X{sourceThread} - %msg%n + + + + + + + + diff --git a/akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala b/akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala new file mode 100644 index 0000000000..bef50fed72 --- /dev/null +++ b/akka-sbt-plugin/sample/src/main/scala/HelloKernel.scala @@ -0,0 +1,37 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package sample.kernel.hello + +import akka.actor.{ Actor, ActorSystem, ActorLogging, Props } +import akka.kernel.Bootable + +case object Start + +class HelloActor extends Actor with ActorLogging { + val worldActor = context.actorOf(Props[WorldActor], name = "world") + + def receive = { + case Start ⇒ worldActor ! "Hello" + case message: String ⇒ + log.info("Received message [{}]", message) + } +} + +class WorldActor extends Actor { + def receive = { + case message: String ⇒ sender ! (message.toUpperCase + " world!") + } +} + +class HelloKernel extends Bootable { + val system = ActorSystem("hellokernel") + + def startup = { + system.actorOf(Props[HelloActor], name = "hello") ! Start + } + + def shutdown = { + system.shutdown() + } +} \ No newline at end of file diff --git a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala index 5c1c5045af..08826fa5dd 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala @@ -126,7 +126,7 @@ object AkkaKernelPlugin extends Plugin { """|#!/bin/sh | |AKKA_HOME="$(cd "$(cd "$(dirname "$0")"; pwd -P)"/..; pwd)" - |AKKA_CLASSPATH="$AKKA_HOME/lib/*:$AKKA_HOME/config" + |AKKA_CLASSPATH="$AKKA_HOME/config:$AKKA_HOME/lib/*" |JAVA_OPTS="%s" | |java $JAVA_OPTS -cp "$AKKA_CLASSPATH" -Dakka.home="$AKKA_HOME" %s "$@" @@ -135,7 +135,7 @@ object AkkaKernelPlugin extends Plugin { private def distBatScript = """|@echo off |set AKKA_HOME=%%~dp0.. - |set AKKA_CLASSPATH=%%AKKA_HOME%%\lib\*;%%AKKA_HOME%%\config + |set AKKA_CLASSPATH=%%AKKA_HOME%%\config;%%AKKA_HOME%%\lib\* |set JAVA_OPTS=%s | |java %%JAVA_OPTS%% -cp "%%AKKA_CLASSPATH%%" -Dakka.home="%%AKKA_HOME%%" %s %%* From ebec79732222bc6fdf64bace9237ffe1cb6f3533 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 11:17:38 +0100 Subject: [PATCH 10/13] Describe akka-sbt-plugin AkkaKernelPlugin. Version 2.0-SNAPSHOT. See #1742 --- akka-sbt-plugin/sample/project/Build.scala | 2 +- akka-sbt-plugin/sample/project/plugins.sbt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-sbt-plugin/sample/project/Build.scala b/akka-sbt-plugin/sample/project/Build.scala index 0fc391775b..60c83740f8 100644 --- a/akka-sbt-plugin/sample/project/Build.scala +++ b/akka-sbt-plugin/sample/project/Build.scala @@ -49,7 +49,7 @@ object Dependencies { object Dependency { // Versions object V { - val Akka = "2.0-M3" + val Akka = "2.0-SNAPSHOT" } val akkaKernel = "com.typesafe.akka" % "akka-kernel" % V.Akka diff --git a/akka-sbt-plugin/sample/project/plugins.sbt b/akka-sbt-plugin/sample/project/plugins.sbt index afba4f3ad2..a44d2ee72f 100644 --- a/akka-sbt-plugin/sample/project/plugins.sbt +++ b/akka-sbt-plugin/sample/project/plugins.sbt @@ -1,3 +1,3 @@ resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" -addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.0-M3") +addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.0-SNAPSHOT") From c1178c9b1aba8cbf32082b4517f61e4fff2516c9 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 11:56:09 +0100 Subject: [PATCH 11/13] DOC: Better use of pipeTo in FaultHandlingDocSample --- .../actor/japi/FaultHandlingDocSample.java | 35 ++++++++++++++----- .../docs/actor/FaultHandlingDocSample.scala | 20 ++++++----- 2 files changed, 38 insertions(+), 17 deletions(-) diff --git a/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java b/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java index 8665f3777b..265f005059 100644 --- a/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java +++ b/akka-docs/java/code/akka/docs/actor/japi/FaultHandlingDocSample.java @@ -35,7 +35,7 @@ public class FaultHandlingDocSample { /** * Runs the sample */ - public static void main(String... args) { + public static void main(String[] args) { Config config = ConfigFactory.parseString("akka.loglevel = DEBUG \n" + "akka.actor.debug.lifecycle = on"); ActorSystem system = ActorSystem.create("FaultToleranceSample", config); @@ -62,11 +62,11 @@ public class FaultHandlingDocSample { public void onReceive(Object msg) { log.debug("received message {}", msg); - if (msg instanceof CurrentCount) { - CurrentCount current = (CurrentCount) msg; - log.info("Current count for [{}] is [{}]", current.key, current.count); - if (current.count > 50) { - log.info("That's enough, shutting down"); + if (msg instanceof Progress) { + Progress progress = (Progress) msg; + log.info("Current progress: {} %", progress.percent); + if (progress.percent >= 100.0) { + log.info("That's all, shutting down"); getContext().system().shutdown(); } } else if (msg == Actors.receiveTimeout()) { @@ -83,13 +83,25 @@ public class FaultHandlingDocSample { public interface WorkerApi { public static final Object Start = "Start"; public static final Object Do = "Do"; + + public static class Progress { + public final double percent; + + public Progress(double percent) { + this.percent = percent; + } + + public String toString() { + return String.format("%s(%s)", getClass().getSimpleName(), percent); + } + } } //#messages /** * Worker performs some work when it receives the Start message. It will - * continuously notify the sender of the Start message of current progress. + * continuously notify the sender of the Start message of current Progress. * The Worker supervise the CounterService. */ public static class Worker extends UntypedActor { @@ -99,6 +111,7 @@ public class FaultHandlingDocSample { // The sender of the initial Start message will continuously be notified about progress ActorRef progressListener; final ActorRef counterService = getContext().actorOf(new Props(CounterService.class), "counter"); + final int totalCount = 51; // Stop the CounterService child if it throws ServiceUnavailable private static SupervisorStrategy strategy = new OneForOneStrategy(-1, Duration.Inf(), @@ -128,8 +141,12 @@ public class FaultHandlingDocSample { counterService.tell(new Increment(1), getSelf()); counterService.tell(new Increment(1), getSelf()); - // Send current count to the initial sender - pipeTo(ask(counterService, GetCurrentCount, askTimeout), progressListener); + // Send current progress to the initial sender + pipeTo(ask(counterService, GetCurrentCount, askTimeout).map(new Function() { + public Progress apply(CurrentCount c) { + return new Progress(100.0 * c.count / totalCount); + } + }), progressListener); } else { unhandled(msg); } diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala index 99beab0cc3..fbdf3e25b9 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala @@ -43,15 +43,15 @@ object FaultHandlingDocSample extends App { * work has been done. */ class Listener extends Actor with ActorLogging { - import CounterService._ + import Worker._ // If we don't get any progress within 15 seconds then the service is unavailable context.setReceiveTimeout(15 seconds) def receive = { - case CurrentCount(key, count) ⇒ - log.info("Current count for [{}] is [{}]", key, count) - if (count > 50) { - log.info("That's enough, shutting down") + case Progress(percent) ⇒ + log.info("Current progress: {} %", percent) + if (percent >= 100.0) { + log.info("That's all, shutting down") context.system.shutdown() } @@ -66,13 +66,14 @@ class Listener extends Actor with ActorLogging { object Worker { case object Start case object Do + case class Progress(percent: Double) } //#messages /** * Worker performs some work when it receives the `Start` message. * It will continuously notify the sender of the `Start` message - * of current progress. The `Worker` supervise the `CounterService`. + * of current ``Progress``. The `Worker` supervise the `CounterService`. */ class Worker extends Actor with ActorLogging { import Worker._ @@ -87,6 +88,7 @@ class Worker extends Actor with ActorLogging { // The sender of the initial Start message will continuously be notified about progress var progressListener: Option[ActorRef] = None val counterService = context.actorOf(Props[CounterService], name = "counter") + val totalCount = 51 def receive = LoggingReceive { case Start if progressListener.isEmpty ⇒ @@ -98,8 +100,10 @@ class Worker extends Actor with ActorLogging { counterService ! Increment(1) counterService ! Increment(1) - // Send current count to the initial sender - counterService ? GetCurrentCount pipeTo progressListener.get + // Send current progress to the initial sender + counterService ? GetCurrentCount map { + case CurrentCount(_, count) ⇒ Progress(100.0 * count / totalCount) + } pipeTo progressListener.get } } From 4fb0858e557232e5f4a7e5b364d1697c0e21bdd1 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 30 Jan 2012 11:48:02 +0100 Subject: [PATCH 12/13] remote cleanup: include feedback from Viktor and Patrik MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - 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 --- .../scala/akka/actor/ActorRefProvider.scala | 5 ++- .../main/scala/akka/actor/ActorSystem.scala | 24 ++++++++---- .../src/main/scala/akka/actor/Deployer.scala | 7 ++-- .../akka/dispatch/AbstractDispatcher.scala | 3 +- .../scala/akka/dispatch/Dispatchers.scala | 8 ++-- .../src/main/scala/akka/event/Logging.scala | 2 +- akka-docs/general/configuration.rst | 14 +++++++ .../akka/actor/mailbox/DurableMailbox.scala | 4 +- .../actor/mailbox/BSONSerialization.scala | 10 ++--- akka-remote/src/main/resources/reference.conf | 37 +++++++++---------- .../akka/remote/RemoteActorRefProvider.scala | 7 ++-- .../main/scala/akka/remote/RemoteDaemon.scala | 4 +- .../scala/akka/remote/RemoteDeployer.scala | 2 +- .../main/scala/akka/remote/netty/Client.scala | 11 +++--- .../remote/netty/NettyRemoteSupport.scala | 19 ++++++---- .../main/scala/akka/remote/netty/Server.scala | 26 +++++++------ .../scala/akka/remote/netty/Settings.scala | 5 +-- ...rGatherRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../scala/akka/remote/RemoteConfigSpec.scala | 34 ++++++++++++++++- 19 files changed, 141 insertions(+), 83 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 369e1429db..56c3389072 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -298,12 +298,13 @@ class LocalActorRefProvider( def this(_systemName: String, settings: ActorSystem.Settings, eventStream: EventStream, - scheduler: Scheduler) = + scheduler: Scheduler, + classloader: ClassLoader) = this(_systemName, settings, eventStream, scheduler, - new Deployer(settings)) + new Deployer(settings, classloader)) val rootPath: ActorPath = RootActorPath(Address("akka", _systemName)) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 81e0761bb0..e3235a5cec 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -322,6 +322,14 @@ abstract class ExtendedActorSystem extends ActorSystem { */ 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 { @@ -386,16 +394,17 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten val scheduler: Scheduler = createScheduler() + val internalClassLoader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader + val provider: ActorRefProvider = { val arguments = Seq( classOf[String] -> name, classOf[Settings] -> settings, 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, loader) match { + ReflectiveAccess.createInstance[ActorRefProvider](ProviderClass, arguments, internalClassLoader) match { case Left(e) ⇒ throw e case Right(p) ⇒ p } @@ -416,7 +425,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten 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 def terminationFuture: Future[Unit] = provider.terminationFuture @@ -533,10 +544,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten private def loadExtensions() { import scala.collection.JavaConversions._ - val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse this.getClass.getClassLoader settings.config.getStringList("akka.extensions") foreach { fqcn ⇒ 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: ExtensionId[_]) ⇒ registerExtension(p); case Right(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index c8e780d5c2..36d82b2cec 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -23,7 +23,7 @@ case object LocalScope extends Scope * * @author Jonas Bonér */ -class Deployer(val settings: ActorSystem.Settings) { +class Deployer(val settings: ActorSystem.Settings, val classloader: ClassLoader) { import scala.collection.JavaConverters._ @@ -41,7 +41,6 @@ class Deployer(val settings: ActorSystem.Settings) { def deploy(d: Deploy): Unit = deployments.put(d.path, d) protected def parseConfig(key: String, config: Config): Option[Deploy] = { - import akka.util.ReflectiveAccess.getClassFor val deployment = config.withFallback(default) @@ -65,8 +64,8 @@ class Deployer(val settings: ActorSystem.Settings) { case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer) case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer) case fqn ⇒ - val constructorSignature = Array[Class[_]](classOf[Config]) - ReflectiveAccess.createInstance[RouterConfig](fqn, constructorSignature, Array[AnyRef](deployment)) match { + val args = Seq(classOf[Config] -> deployment) + ReflectiveAccess.createInstance[RouterConfig](fqn, args, classloader) match { case Right(router) ⇒ router case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 2f6b330cc8..3a788c0fd7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -322,8 +322,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit } case fqcn ⇒ val args = Seq(classOf[Config] -> config) - val loader = Option(Thread.currentThread.getContextClassLoader) getOrElse getClass.getClassLoader - ReflectiveAccess.createInstance[MailboxType](fqcn, args, loader) match { + ReflectiveAccess.createInstance[MailboxType](fqcn, args, prerequisites.classloader) match { case Right(instance) ⇒ instance case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 3871905905..17a2410784 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -19,13 +19,15 @@ trait DispatcherPrerequisites { def eventStream: EventStream def deadLetterMailbox: Mailbox def scheduler: Scheduler + def classloader: ClassLoader } case class DefaultDispatcherPrerequisites( val threadFactory: ThreadFactory, val eventStream: EventStream, val deadLetterMailbox: Mailbox, - val scheduler: Scheduler) extends DispatcherPrerequisites + val scheduler: Scheduler, + val classloader: ClassLoader) extends DispatcherPrerequisites object Dispatchers { /** @@ -134,8 +136,8 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites) case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites) case fqn ⇒ - val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites]) - ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, constructorSignature, Array[AnyRef](cfg, prerequisites)) match { + val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) + ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, args, prerequisites.classloader) match { case Right(configurator) ⇒ configurator case Left(exception) ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 35a84b511f..c8bbe5f9eb 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -101,7 +101,7 @@ trait LoggingBus extends ActorEventBus { if loggerName != StandardOutLoggerName } yield { try { - ReflectiveAccess.getClassFor[Actor](loggerName) match { + ReflectiveAccess.getClassFor[Actor](loggerName, system.internalClassLoader) match { case Right(actorClass) ⇒ addLogger(system, actorClass, level, logName) case Left(exception) ⇒ throw exception } diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index 8109e7a358..1a01ba24c9 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -217,6 +217,20 @@ and parsed by the actor system can be displayed like this: println(system.settings()); // 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 thread’s 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 ----------------------------- diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 6638c380bf..5e319dafac 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -3,7 +3,7 @@ */ package akka.actor.mailbox -import akka.actor.{ ActorContext, ActorRef } +import akka.actor.{ ActorContext, ActorRef, ExtendedActorSystem } import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox } import akka.remote.MessageSerializer import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol } @@ -15,7 +15,7 @@ private[akka] object DurableExecutableMailboxConfig { abstract class DurableMailbox(owner: ActorContext) extends CustomMailbox(owner) with DefaultSystemMessageQueue { import DurableExecutableMailboxConfig._ - def system = owner.system + def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem] def ownerPath = owner.self.path val ownerPathString = ownerPath.elements.mkString("/") val name = "mailbox_" + Name.replaceAllIn(ownerPathString, "_") diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 217f46d6ec..cc36286e36 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -16,11 +16,9 @@ import org.bson.DefaultBSONSerializer import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.MessageSerializer -import akka.actor.{ ActorSystem, ActorSystemImpl } +import akka.actor.ExtendedActorSystem -class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { - - val systemImpl = system.asInstanceOf[ActorSystemImpl] +class BSONSerializableMailbox(system: ExtendedActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { 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] system.log.debug("Deserializing a durable message from MongoDB: {}", doc) 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 senderPath = doc.as[String]("senderPath") - val sender = systemImpl.actorFor(senderPath) + val sender = system.actorFor(senderPath) MongoDurableMessage(ownerPath, msg, sender) } diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index 5b96b15d43..86de93527c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -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 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 # 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 # to client). 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? # 0 means "never backoff", any positive number will indicate time to block at most. 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' 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)? require-cookie = off - # (S) Reuse inbound connections for outbound messages + # (I) Reuse inbound connections for outbound messages use-passive-connections = on - # (C&S) Whether any Threds created by the remoting should be daemons or not - daemonic = on - - # (S) The hostname or ip to bind the remoting to, + # (I) The hostname or ip to bind the remoting to, # InetAddress.getLocalHost.getHostAddress is used if empty 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 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 - # (C) Timeout duration + # (O) Timeout duration connection-timeout = 120s - # (S) Sets the size of the connection backlog + # (I) Sets the size of the connection backlog 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 - # (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 - # (S) Maximum channel size, 0 for off + # (I) Maximum channel size, 0 for off 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 - # (C) Time between reconnect attempts for active clients + # (O) Time between reconnect attempts for active clients 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 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 } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 61b675fe55..6081372e6b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -22,11 +22,12 @@ class RemoteActorRefProvider( val systemName: String, val settings: ActorSystem.Settings, val eventStream: EventStream, - val scheduler: Scheduler) extends ActorRefProvider { + val scheduler: Scheduler, + val classloader: ClassLoader) extends ActorRefProvider { 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) @@ -87,7 +88,7 @@ class RemoteActorRefProvider( classOf[ActorSystemImpl] -> system, 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 Right(remote) ⇒ remote } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 1274e99416..bda71bcc00 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -17,8 +17,10 @@ case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMs * 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. + * + * 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) { /** diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index 6ce486b464..799bba13e3 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -10,7 +10,7 @@ import akka.config.ConfigurationException 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] = { import scala.collection.JavaConverters._ diff --git a/akka-remote/src/main/scala/akka/remote/netty/Client.scala b/akka-remote/src/main/scala/akka/remote/netty/Client.scala index 2d7d6218f7..2947d9db26 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Client.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -158,11 +158,12 @@ class ActiveRemoteClient private[akka] ( executionHandler = new ExecutionHandler(netty.executor) - bootstrap = new ClientBootstrap(netty.clientChannelFactory) - bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this)) - bootstrap.setOption("tcpNoDelay", true) - bootstrap.setOption("keepAlive", true) - bootstrap.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis) + val b = new ClientBootstrap(netty.clientChannelFactory) + b.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, b, executionHandler, remoteAddress, this)) + b.setOption("tcpNoDelay", true) + b.setOption("keepAlive", true) + b.setOption("connectTimeoutMillis", settings.ConnectionTimeout.toMillis) + bootstrap = b val remoteIP = InetAddress.getByName(remoteAddress.host.get) log.debug("Starting remote client connection to [{}|{}]", remoteAddress, remoteIP) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 4179beeb3d..e9fe83dd7e 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -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 threadFactory = new MonitorableThreadFactory("NettyRemoteTransport", settings.Daemonic, Some(getClass.getClassLoader)) - val timer: HashedWheelTimer = new HashedWheelTimer(threadFactory) + val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory) val executor = new OrderedMemoryAwareThreadPoolExecutor( settings.ExecutionPoolSize, settings.MaxChannelMemorySize, settings.MaxTotalMemorySize, - settings.ExecutionPoolKeepAlive.length, - settings.ExecutionPoolKeepAlive.unit, - threadFactory) + settings.ExecutionPoolKeepalive.length, + settings.ExecutionPoolKeepalive.unit, + system.threadFactory) val clientChannelFactory = new NioClientSocketChannelFactory( - Executors.newCachedThreadPool(threadFactory), - Executors.newCachedThreadPool(threadFactory)) + Executors.newCachedThreadPool(system.threadFactory), + Executors.newCachedThreadPool(system.threadFactory)) private val remoteClients = new HashMap[Address, RemoteClient] private val clientsLock = new ReentrantReadWriteLock @@ -79,7 +78,11 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor def shutdown(): Unit = { clientsLock.writeLock().lock() 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() } finally { clientsLock.writeLock().unlock() diff --git a/akka-remote/src/main/scala/akka/remote/netty/Server.scala b/akka-remote/src/main/scala/akka/remote/netty/Server.scala index 9ebeb8f3e8..a8bc6ef67b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -29,28 +29,29 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) { val ip = InetAddress.getByName(settings.Hostname) private val factory = new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(netty.threadFactory), - Executors.newCachedThreadPool(netty.threadFactory)) - - private val bootstrap = new ServerBootstrap(factory) + Executors.newCachedThreadPool(netty.system.threadFactory), + Executors.newCachedThreadPool(netty.system.threadFactory)) private val executionHandler = new ExecutionHandler(netty.executor) // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") - val pipelineFactory = new RemoteServerPipelineFactory(openChannels, executionHandler, netty) - bootstrap.setPipelineFactory(pipelineFactory) - bootstrap.setOption("backlog", settings.Backlog) - bootstrap.setOption("tcpNoDelay", true) - bootstrap.setOption("child.keepAlive", true) - bootstrap.setOption("reuseAddress", true) + private val bootstrap = { + val b = new ServerBootstrap(factory) + b.setPipelineFactory(new RemoteServerPipelineFactory(openChannels, executionHandler, netty)) + b.setOption("backlog", settings.Backlog) + b.setOption("tcpNoDelay", true) + b.setOption("child.keepAlive", true) + b.setOption("reuseAddress", true) + b + } @volatile private[akka] var channel: Channel = _ def start(): Unit = { - channel = bootstrap.bind(new InetSocketAddress(ip, settings.Port)) + channel = bootstrap.bind(new InetSocketAddress(ip, settings.DesiredPortFromConfig)) openChannels.add(channel) netty.notifyListeners(RemoteServerStarted(netty)) } @@ -62,7 +63,7 @@ class NettyRemoteServer(val netty: NettyRemoteTransport) { b.setOrigin(RemoteProtocol.AddressProtocol.newBuilder .setSystem(settings.systemName) .setHostname(settings.Hostname) - .setPort(settings.Port) + .setPort(settings.DesiredPortFromConfig) .build) if (settings.SecureCookie.nonEmpty) b.setCookie(settings.SecureCookie.get) @@ -139,6 +140,7 @@ class RemoteServerHandler( 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) = { if (addressToSet) { netty.setAddressFromChannel(event.getChannel) diff --git a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala index 123e2d53dc..3f7c8f83de 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -13,7 +13,6 @@ class NettySettings(config: Config, val systemName: String) { import config._ - val Daemonic = getBoolean("daemonic") val BackoffTimeout = Duration(getMilliseconds("backoff-timeout"), MILLISECONDS) val SecureCookie: Option[String] = getString("secure-cookie") match { @@ -38,13 +37,13 @@ class NettySettings(config: Config, val systemName: String) { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } - val Port = getInt("port") + val DesiredPortFromConfig = getInt("port") val ConnectionTimeout = Duration(getMilliseconds("connection-timeout"), MILLISECONDS) 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 { case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.netty.execution-pool-size is less than 1") diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala index 0eb3caa8e4..8a6bad79fa 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala @@ -89,7 +89,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter } 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)) { case (m, (n, c)) ⇒ m + (n -> (m(n) + c)) } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 837846d058..b60b90b900 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -6,6 +6,8 @@ package akka.remote import akka.testkit.AkkaSpec import akka.actor.ExtendedActorSystem import akka.util.duration._ +import akka.util.Duration +import akka.remote.netty.NettyRemoteTransport @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RemoteConfigSpec extends AkkaSpec( @@ -17,8 +19,9 @@ class RemoteConfigSpec extends AkkaSpec( } """) { - "RemoteExtension" must { - "be able to parse remote and cluster config elements" in { + "Remoting" must { + + "be able to parse generic remote config elements" in { val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings import settings._ @@ -33,5 +36,32 @@ class RemoteConfigSpec extends AkkaSpec( GossipFrequency must be(1 second) 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) + } + } } From 8c5d10df8e75aec760e39dd71658b963febd2428 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 30 Jan 2012 14:51:25 +0100 Subject: [PATCH 13/13] Fixed memory leak in doc tests. See #1743 --- .../code/akka/docs/actor/FSMDocTestBase.java | 82 ++++++++++++------- .../docs/actor/UntypedActorDocTestBase.java | 14 ++-- 2 files changed, 59 insertions(+), 37 deletions(-) diff --git a/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java b/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java index 981cac15b1..aeaca63f92 100644 --- a/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java @@ -18,45 +18,53 @@ import akka.actor.UntypedActor; import akka.actor.ActorSystem; import akka.actor.Props; import akka.testkit.TestProbe; +import akka.testkit.AkkaSpec; public class FSMDocTestBase { //#data public static final class SetTarget { final ActorRef ref; + public SetTarget(ActorRef ref) { this.ref = ref; } } - + public static final class Queue { final Object o; + public Queue(Object o) { this.o = o; } } - + public static final Object flush = new Object(); - + public static final class Batch { final List objects; + public Batch(List objects) { this.objects = objects; } } + //#data //#base static abstract class MyFSMBase extends UntypedActor { - + /* * This is the mutable state of this state machine. */ - protected enum State { IDLE, ACTIVE; } + protected enum State { + IDLE, ACTIVE; + } + private State state = State.IDLE; private ActorRef target; private List queue; - + /* * Then come all the mutator methods: */ @@ -64,21 +72,23 @@ public class FSMDocTestBase { this.target = target; queue = new ArrayList(); } - + protected void setState(State s) { if (state != s) { transition(state, s); state = s; } } - + protected void enqueue(Object o) { - if (queue != null) queue.add(o); + if (queue != null) + queue.add(o); } - + protected List drainQueue() { final List q = queue; - if (q == null) throw new IllegalStateException("drainQueue(): not yet initialized"); + if (q == null) + throw new IllegalStateException("drainQueue(): not yet initialized"); queue = new ArrayList(); return q; } @@ -89,13 +99,14 @@ public class FSMDocTestBase { protected boolean isInitialized() { return target != null; } - + protected State getState() { return state; } - + protected ActorRef getTarget() { - if (target == null) throw new IllegalStateException("getTarget(): not yet initialized"); + if (target == null) + throw new IllegalStateException("getTarget(): not yet initialized"); return target; } @@ -104,53 +115,62 @@ public class FSMDocTestBase { */ abstract protected void transition(State old, State next); } + //#base //#actor static public class MyFSM extends MyFSMBase { - + private final LoggingAdapter log = Logging.getLogger(getContext().system(), this); - + @Override public void onReceive(Object o) { - + if (getState() == State.IDLE) { - - if (o instanceof SetTarget) + + if (o instanceof SetTarget) init(((SetTarget) o).ref); - - else whenUnhandled(o); + + else + whenUnhandled(o); } else if (getState() == State.ACTIVE) { - if (o == flush) + if (o == flush) setState(State.IDLE); - else whenUnhandled(o); + else + whenUnhandled(o); } } - + @Override public void transition(State old, State next) { if (old == State.ACTIVE) { getTarget().tell(new Batch(drainQueue())); } } - + private void whenUnhandled(Object o) { if (o instanceof Queue && isInitialized()) { enqueue(((Queue) o).o); setState(State.ACTIVE); - + } else { log.warning("received unknown message {} in state {}", o, getState()); } } } + //#actor - - ActorSystem system = ActorSystem.create(); - + + ActorSystem system; + + @org.junit.Before + public void setUp() { + system = ActorSystem.create("FSMSystem", AkkaSpec.testConf()); + } + @org.junit.Test public void mustBunch() { final ActorRef buncher = system.actorOf(new Props(MyFSM.class)); @@ -165,10 +185,10 @@ public class FSMDocTestBase { assert b.objects.contains(1); assert b.objects.contains(2); } - + @org.junit.After public void cleanup() { system.shutdown(); } - + } diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index a72c828862..ff14c4e09b 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -211,16 +211,17 @@ public class UntypedActorDocTestBase { //#gracefulStop system.shutdown(); } - + class Result { final int x; final String s; + public Result(int x, String s) { this.x = x; this.s = s; } } - + @Test public void usePatternsAskPipeTo() { ActorSystem system = ActorSystem.create("MySystem"); @@ -229,13 +230,13 @@ public class UntypedActorDocTestBase { ActorRef actorC = system.actorOf(new Props(MyUntypedActor.class)); //#ask-pipeTo final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS)); - + final ArrayList> futures = new ArrayList>(); futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout futures.add(ask(actorB, "reqeest", t)); // using timeout from above - + final Future> aggregate = Futures.sequence(futures, system.dispatcher()); - + final Future transformed = aggregate.map(new akka.japi.Function, Result>() { public Result apply(Iterable coll) { final Iterator it = coll.iterator(); @@ -244,9 +245,10 @@ public class UntypedActorDocTestBase { return new Result(x, s); } }); - + pipeTo(transformed, actorC); //#ask-pipeTo + system.shutdown(); } public static class MyActor extends UntypedActor {