From b2ba6d4702958e3fe46c9e21cb0136627c4a4b90 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Endre=20S=C3=A1ndor=20Varga?= Date: Mon, 22 Oct 2012 17:13:50 +0200 Subject: [PATCH 01/62] Added remoting lifecycle event classes and event publisher --- .../scala/akka/cluster/LargeClusterSpec.scala | 1 - akka-remote/src/main/resources/reference.conf | 6 ++- .../main/scala/akka/remote/netty/Client.scala | 53 +++++++++++-------- .../main/scala/akka/remote/netty/Server.scala | 29 +++++----- 4 files changed, 51 insertions(+), 38 deletions(-) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala index 6465c5ead8..d074da2885 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala @@ -55,7 +55,6 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig { akka.remote.log-remote-lifecycle-events = off akka.remote.netty.execution-pool-size = 4 #akka.remote.netty.reconnection-time-window = 10s - akka.remote.netty.read-timeout = 5s akka.remote.netty.write-timeout = 5s akka.remote.netty.backoff-timeout = 500ms akka.remote.netty.connection-timeout = 500ms diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a70106a8b2..36bc91a02c 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -171,10 +171,12 @@ akka { # (O) Time between reconnect attempts for active clients reconnect-delay = 5s - # (O) Read inactivity period (lowest resolution is seconds) + # (O) Client read inactivity period (finest resolution is seconds) # after which active client connection is shutdown; - # will be re-established in case of new communication requests. + # Connection will be re-established in case of new communication requests. # A value of 0 will turn this feature off + # This value should be left to be 0 when use-passive-connections is off, or if + # no traffic is expected from the server side (i.e. it is a sink). read-timeout = 0s # (O) Write inactivity period (lowest resolution is seconds) 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 2015f82ac7..7568f859ec 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Client.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Client.scala @@ -21,6 +21,7 @@ import akka.util.Switch import scala.util.control.NonFatal import org.jboss.netty.handler.ssl.SslHandler import scala.concurrent.duration._ +import java.nio.channels.ClosedChannelException /** * This is the abstract baseclass for netty remote clients, currently there's only an @@ -63,21 +64,23 @@ private[akka] abstract class RemoteClient private[akka] (val netty: NettyRemoteT private def send(request: (Any, Option[ActorRef], ActorRef)): Unit = { try { val channel = currentChannel - val f = channel.write(request) - f.addListener( - new ChannelFutureListener { - import netty.system.deadLetters - def operationComplete(future: ChannelFuture): Unit = - if (future.isCancelled || !future.isSuccess) request match { - case (msg, sender, recipient) ⇒ deadLetters ! DeadLetter(msg, sender.getOrElse(deadLetters), recipient) - // We don't call notifyListeners here since we don't think failed message deliveries are errors - /// If the connection goes down we'll get the error reporting done by the pipeline. - } - }) - // 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 + if (channel.isOpen) { + val f = channel.write(request) + f.addListener( + new ChannelFutureListener { + import netty.system.deadLetters + def operationComplete(future: ChannelFuture): Unit = + if (future.isCancelled || !future.isSuccess) request match { + case (msg, sender, recipient) ⇒ deadLetters ! DeadLetter(msg, sender.getOrElse(deadLetters), recipient) + // We don't call notifyListeners here since we don't think failed message deliveries are errors + /// If the connection goes down we'll get the error reporting done by the pipeline. + } + }) + // 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 NonFatal(e) ⇒ netty.notifyListeners(RemoteClientError(e, netty, remoteAddress)) @@ -195,8 +198,11 @@ private[akka] class ActiveRemoteClient private[akka] ( notifyListeners(RemoteClientShutdown(netty, remoteAddress)) try { if ((connection ne null) && (connection.getChannel ne null)) { - ChannelAddress.remove(connection.getChannel) - connection.getChannel.close() + val channel = connection.getChannel + ChannelAddress.remove(channel) + // Try to disconnect first to reduce "connection reset by peer" events + if (channel.isConnected) channel.disconnect() + if (channel.isOpen) channel.close() } } finally { try { @@ -267,10 +273,8 @@ private[akka] class ActiveRemoteClientHandler( 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)) - case other ⇒ throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.netty, client.remoteAddress) } @@ -307,9 +311,14 @@ private[akka] class ActiveRemoteClientHandler( } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - val cause = if (event.getCause ne null) event.getCause else new Exception("Unknown cause") - client.notifyListeners(RemoteClientError(cause, client.netty, client.remoteAddress)) - event.getChannel.close() + val cause = if (event.getCause ne null) event.getCause else new AkkaException("Unknown cause") + cause match { + case _: ClosedChannelException ⇒ // Ignore + case NonFatal(e) ⇒ + client.notifyListeners(RemoteClientError(e, client.netty, client.remoteAddress)) + event.getChannel.close() + case e: Throwable ⇒ throw e // Rethrow fatals + } } } 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 16269a43a2..3de64ca0a9 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -3,21 +3,19 @@ */ package akka.remote.netty +import akka.actor.Address +import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } +import akka.remote._ +import java.net.InetAddress import java.net.InetSocketAddress +import java.nio.channels.ClosedChannelException 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._ import org.jboss.netty.channel.group.ChannelGroup import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory -import org.jboss.netty.handler.codec.frame.{ LengthFieldPrepender, LengthFieldBasedFrameDecoder } -import org.jboss.netty.handler.execution.ExecutionHandler -import akka.remote.RemoteProtocol.{ RemoteControlProtocol, CommandType, AkkaRemoteProtocol } -import akka.remote.{ RemoteServerShutdown, RemoteServerError, RemoteServerClientDisconnected, RemoteServerClientConnected, RemoteServerClientClosed, RemoteProtocol, RemoteMessage } -import akka.actor.Address -import java.net.InetAddress -import akka.actor.ActorSystemImpl -import org.jboss.netty.channel._ +import scala.util.control.NonFatal +import akka.AkkaException private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) { @@ -154,7 +152,6 @@ private[akka] class RemoteServerHandler( event.getMessage match { case remote: AkkaRemoteProtocol if remote.hasMessage ⇒ netty.receiveMessage(new RemoteMessage(remote.getMessage, netty.system)) - case remote: AkkaRemoteProtocol if remote.hasInstruction ⇒ val instruction = remote.getInstruction instruction.getCommandType match { @@ -179,8 +176,14 @@ private[akka] class RemoteServerHandler( } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - netty.notifyListeners(RemoteServerError(event.getCause, netty)) - event.getChannel.close() + val cause = if (event.getCause ne null) event.getCause else new AkkaException("Unknown cause") + cause match { + case _: ClosedChannelException ⇒ // Ignore + case NonFatal(e) ⇒ + netty.notifyListeners(RemoteServerError(e, netty)) + event.getChannel.close() + case e: Throwable ⇒ throw e // Rethrow fatals + } } } From 06199e3f8f6b27e63895495b9b358aee140d0239 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 26 Oct 2012 16:56:23 +0200 Subject: [PATCH 02/62] MatchError in RemoteDaemon case AddressTerminated, see #2660 --- akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index f6e000737a..ee8a6c5698 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -102,7 +102,10 @@ private[akka] class RemoteSystemDaemon( } case AddressTerminated(address) ⇒ - foreachChild { case a: InternalActorRef if a.getParent.path.address == address ⇒ system.stop(a) } + foreachChild { + case a: InternalActorRef if a.getParent.path.address == address ⇒ system.stop(a) + case _ ⇒ // skip, this child doesn't belong to the terminated address + } case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this) } From 7be1f4b55ede536c79f5b7df8b19da1e4b1ed113 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 29 Oct 2012 15:36:11 +0100 Subject: [PATCH 03/62] #2658 - Fixing todo sections of the documentation documentation ;-) --- akka-docs/rst/dev/documentation.rst | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/akka-docs/rst/dev/documentation.rst b/akka-docs/rst/dev/documentation.rst index ad8ff244ae..b5745db46a 100644 --- a/akka-docs/rst/dev/documentation.rst +++ b/akka-docs/rst/dev/documentation.rst @@ -17,13 +17,12 @@ built using `Sphinx`_. Sphinx ====== -More to come... - +For more details see `The Sphinx Documentation `_ reStructuredText ================ -More to come... +For more details see `The reST Quickref `_ Sections -------- From 8ee11349bcff8875ec7006beffcef03eb7caa733 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 29 Oct 2012 16:02:37 +0100 Subject: [PATCH 04/62] #2655 - Hardinging the LocalActorRefProviderSpec --- .../test/scala/akka/actor/LocalActorRefProviderSpec.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala index d9586c1f50..a768089415 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala @@ -43,9 +43,10 @@ class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.confi val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552" val a = system.actorOf(Props(new Actor { val child = context.actorOf(Props.empty, name = childName) - assert(childName == child.path.name) def receive = { - case "lookup" ⇒ sender ! context.actorFor(childName) + case "lookup" ⇒ + if (childName == child.path.name) sender ! context.actorFor(childName) + else sender ! s"§childName is not ${child.path.name}!" } })) a.tell("lookup", testActor) From d01ea366ce2805557d29044de01f73071bdab53b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 29 Oct 2012 23:07:06 +0100 Subject: [PATCH 05/62] #2661 - Changing ignoreMsg of TestProbe to take a PF from Any instead of AnyRef, to support primitives --- akka-testkit/src/main/scala/akka/testkit/TestKit.scala | 4 ++-- .../src/test/scala/akka/testkit/TestProbeSpec.scala | 7 +++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index b466b8a9d8..4f6744b74b 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -16,7 +16,7 @@ import scala.annotation.varargs import scala.reflect.ClassTag object TestActor { - type Ignore = Option[PartialFunction[AnyRef, Boolean]] + type Ignore = Option[PartialFunction[Any, Boolean]] abstract class AutoPilot { def run(sender: ActorRef, msg: Any): AutoPilot @@ -138,7 +138,7 @@ trait TestKitBase { * Ignore all messages in the test actor for which the given partial * function returns true. */ - def ignoreMsg(f: PartialFunction[AnyRef, Boolean]) { testActor ! TestActor.SetIgnore(Some(f)) } + def ignoreMsg(f: PartialFunction[Any, Boolean]) { testActor ! TestActor.SetIgnore(Some(f)) } /** * Stop ignoring messages in the test actor. diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index a413aa86a9..72e5b3a8c0 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -78,6 +78,13 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout { expectMsgAllClassOf(5 seconds, classOf[Int]) must be(Seq(42)) } + "be able to ignore primitive types" in { + ignoreMsg { case 42 ⇒ true } + testActor ! 42 + testActor ! "pigdog" + expectMsg("pigdog") + } + } } From 49500ab248e96c1a352c638dc205b895151985af Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 26 Oct 2012 11:42:18 +0200 Subject: [PATCH 06/62] Use full address when hashing routees, see #2657 * Problem: ConsistentHashingRouter used on different nodes with remote and local routees doesn't route to same targets. * Reason: Full address, with host and port was not used in the representation of the routees, which produced different hash rings on different nodes. * Solution: Fill in full address in the toString representation for LocalActorRef * IMPORTANT: Discovered that rootPath of the provider didn't include the full address. It is documented that it should and I needed that to be able to grab the address without depending on remoting. This caused changes in RemoteActorRefProvider. Initialization order is a bit scary there. --- .../scala/akka/routing/ConsistentHash.scala | 6 ++-- .../routing/ConsistentHashingRouter.scala | 33 +++++++++++++++---- .../akka/remote/RemoteActorRefProvider.scala | 22 ++++++++----- 3 files changed, 45 insertions(+), 16 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index d3bef92e6c..819fea2586 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -120,8 +120,10 @@ object ConsistentHash { apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]])) } - private def nodeHashFor(node: Any, vnode: Int): Int = - hashFor((node + ":" + vnode).getBytes("UTF-8")) + private def nodeHashFor(node: Any, vnode: Int): Int = { + val baseStr = node.toString + ":" + hashFor(baseStr + vnode) + } private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes) diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index 71a36be2b7..ca9027c8e8 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -12,6 +12,8 @@ import akka.dispatch.Dispatchers import akka.event.Logging import akka.serialization.SerializationExtension import java.util.concurrent.atomic.AtomicReference +import akka.actor.Address +import akka.actor.ExtendedActorSystem object ConsistentHashingRouter { /** @@ -238,20 +240,22 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ } val log = Logging(routeeProvider.context.system, routeeProvider.context.self) + val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.rootPath.address val vnodes = if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor else virtualNodesFactor // tuple of routees and the ConsistentHash, updated together in updateConsistentHash - val consistentHashRef = new AtomicReference[(IndexedSeq[ActorRef], ConsistentHash[ActorRef])]((null, null)) + val consistentHashRef = new AtomicReference[(IndexedSeq[ConsistentActorRef], ConsistentHash[ConsistentActorRef])]((null, null)) updateConsistentHash() // update consistentHash when routees has changed // changes to routees are rare and when no changes this is a quick operation - def updateConsistentHash(): ConsistentHash[ActorRef] = { + def updateConsistentHash(): ConsistentHash[ConsistentActorRef] = { val oldConsistentHashTuple = consistentHashRef.get val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple - val currentRoutees = routeeProvider.routees + val currentRoutees = routeeProvider.routees map { ConsistentActorRef(_, selfAddress) } + if (currentRoutees ne oldConsistentHashRoutees) { // when other instance, same content, no need to re-hash, but try to set routees val consistentHash = @@ -267,9 +271,9 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ val currentConsistenHash = updateConsistentHash() if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters else hashData match { - case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes) - case str: String ⇒ currentConsistenHash.nodeFor(str) - case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get) + case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes).actorRef + case str: String ⇒ currentConsistenHash.nodeFor(str).actorRef + case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get).actorRef } } catch { case NonFatal(e) ⇒ @@ -294,4 +298,21 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ } } +} + +/** + * INTERNAL API + * Important to use ActorRef with full address, with host and port, in the hash ring, + * so that same ring is produced on different nodes. + * The ConsistentHash uses toString of the ring nodes, and the ActorRef itself + * isn't a good representation, because LocalActorRef doesn't include the + * host and port. + */ +private[akka] case class ConsistentActorRef(actorRef: ActorRef, selfAddress: Address) { + override def toString: String = { + actorRef.path.address match { + case Address(_, _, None, None) ⇒ actorRef.path.toStringWithAddress(selfAddress) + case a ⇒ actorRef.path.toString + } + } } \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 1d9ad9edc2..4967978582 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -32,7 +32,9 @@ class RemoteActorRefProvider( private var _log = local.log def log: LoggingAdapter = _log - override def rootPath: ActorPath = local.rootPath + @volatile + private var _rootPath = local.rootPath + override def rootPath: ActorPath = _rootPath override def deadLetters: InternalActorRef = local.deadLetters // these are only available after init() @@ -61,7 +63,7 @@ class RemoteActorRefProvider( def init(system: ActorSystemImpl): Unit = { local.init(system) - _remoteDaemon = new RemoteSystemDaemon(system, rootPath / "remote", rootGuardian, log, untrustedMode = remoteSettings.UntrustedMode) + _remoteDaemon = new RemoteSystemDaemon(system, local.rootPath / "remote", rootGuardian, log, untrustedMode = remoteSettings.UntrustedMode) local.registerExtraNames(Map(("remote", remoteDaemon))) _serialization = SerializationExtension(system) @@ -82,6 +84,8 @@ class RemoteActorRefProvider( // this enables reception of remote requests _transport.start() + _rootPath = RootActorPath(local.rootPath.address.copy(host = transport.address.host, port = transport.address.port)) + val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor { def receive = { case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address) @@ -154,7 +158,7 @@ class RemoteActorRefProvider( Iterator(props.deploy) ++ deployment.iterator reduce ((a, b) ⇒ b withFallback a) match { case d @ Deploy(_, _, _, RemoteScope(addr)) ⇒ - if (addr == rootPath.address || addr == transport.address) { + if (isSelfAddress(addr)) { local.actorOf(system, props, supervisor, path, false, deployment.headOption, false, async) } else { val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements @@ -167,12 +171,12 @@ class RemoteActorRefProvider( } def actorFor(path: ActorPath): InternalActorRef = - if (path.address == rootPath.address || path.address == transport.address) actorFor(rootGuardian, path.elements) + if (isSelfAddress(path.address)) actorFor(rootGuardian, path.elements) else new RemoteActorRef(this, transport, path, Nobody, props = None, deploy = None) def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match { case ActorPathExtractor(address, elems) ⇒ - if (address == rootPath.address || address == transport.address) actorFor(rootGuardian, elems) + if (isSelfAddress(address)) actorFor(rootGuardian, elems) else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody, props = None, deploy = None) case _ ⇒ local.actorFor(ref, path) } @@ -190,14 +194,16 @@ class RemoteActorRefProvider( } def getExternalAddressFor(addr: Address): Option[Address] = { - val ta = transport.address - val ra = rootPath.address addr match { - case `ta` | `ra` ⇒ Some(rootPath.address) + case _ if isSelfAddress(addr) ⇒ Some(local.rootPath.address) case Address("akka", _, Some(_), Some(_)) ⇒ Some(transport.address) case _ ⇒ None } } + + private def isSelfAddress(address: Address): Boolean = + address == local.rootPath.address || address == rootPath.address || address == transport.address + } private[akka] trait RemoteRef extends ActorRefScope { From d46f6264794a91fbc58337c434448aba04b65b7e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Mon, 29 Oct 2012 14:37:39 +0100 Subject: [PATCH 07/62] Limit the number of netty worker threads based on configuration. #2659 --- .../src/main/resources/reference.conf | 32 +++++++++++++++++++ .../akka/remote/testconductor/Conductor.scala | 2 +- .../akka/remote/testconductor/Extension.scala | 26 +++++++++++---- .../akka/remote/testconductor/Player.scala | 6 ++-- .../testconductor/RemoteConnection.scala | 8 +++-- akka-remote/src/main/resources/reference.conf | 30 +++++++++++++++++ .../remote/netty/NettyRemoteSupport.scala | 3 +- .../main/scala/akka/remote/netty/Server.scala | 3 +- .../scala/akka/remote/netty/Settings.scala | 11 +++++++ .../scala/akka/remote/RemoteConfigSpec.scala | 25 +++++++++++++++ 10 files changed, 131 insertions(+), 15 deletions(-) diff --git a/akka-remote-tests/src/main/resources/reference.conf b/akka-remote-tests/src/main/resources/reference.conf index 40c16c4ccd..7ad9bf6e76 100644 --- a/akka-remote-tests/src/main/resources/reference.conf +++ b/akka-remote-tests/src/main/resources/reference.conf @@ -29,5 +29,37 @@ akka { # minimum time interval which is to be inserted between reconnect attempts reconnect-backoff = 1s + + netty { + # (I&O) Used to configure the number of I/O worker threads on server sockets + server-socket-worker-pool { + # Min number of threads to cap factor-based number to + pool-size-min = 1 + + # The pool size factor is used to determine thread pool size + # using the following formula: ceil(available processors * factor). + # Resulting size is then bounded by the pool-size-min and + # pool-size-max values. + pool-size-factor = 1.0 + + # Max number of threads to cap factor-based number to + pool-size-max = 2 + } + + # (I&O) Used to configure the number of I/O worker threads on client sockets + client-socket-worker-pool { + # Min number of threads to cap factor-based number to + pool-size-min = 1 + + # The pool size factor is used to determine thread pool size + # using the following formula: ceil(available processors * factor). + # Resulting size is then bounded by the pool-size-min and + # pool-size-max values. + pool-size-factor = 1.0 + + # Max number of threads to cap factor-based number to + pool-size-max = 2 + } + } } } \ No newline at end of file diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala index eae82b6e6d..25837cbb71 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Conductor.scala @@ -391,7 +391,7 @@ private[akka] class Controller(private var initialParticipants: Int, controllerP import BarrierCoordinator._ val settings = TestConductor().Settings - val connection = RemoteConnection(Server, controllerPort, + val connection = RemoteConnection(Server, controllerPort, settings.ServerSocketWorkerPoolSize, new ConductorHandler(settings.QueryTimeout, self, Logging(context.system, "ConductorHandler"))) /* diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala index 07a6c9c22f..62eca3128d 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Extension.scala @@ -6,6 +6,8 @@ import akka.util.Timeout import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.ConcurrentHashMap import scala.concurrent.duration.Duration +import com.typesafe.config.Config +import akka.dispatch.ThreadPoolConfig /** * Access to the [[akka.remote.testconductor.TestConductorExt]] extension: @@ -41,15 +43,25 @@ object TestConductor extends ExtensionKey[TestConductorExt] { class TestConductorExt(val system: ExtendedActorSystem) extends Extension with Conductor with Player { object Settings { - val config = system.settings.config + val config = system.settings.config.getConfig("akka.testconductor") - val ConnectTimeout = Duration(config.getMilliseconds("akka.testconductor.connect-timeout"), MILLISECONDS) - val ClientReconnects = config.getInt("akka.testconductor.client-reconnects") - val ReconnectBackoff = Duration(config.getMilliseconds("akka.testconductor.reconnect-backoff"), MILLISECONDS) + val ConnectTimeout = Duration(config.getMilliseconds("connect-timeout"), MILLISECONDS) + val ClientReconnects = config.getInt("client-reconnects") + val ReconnectBackoff = Duration(config.getMilliseconds("reconnect-backoff"), MILLISECONDS) - implicit val BarrierTimeout = Timeout(Duration(config.getMilliseconds("akka.testconductor.barrier-timeout"), MILLISECONDS)) - implicit val QueryTimeout = Timeout(Duration(config.getMilliseconds("akka.testconductor.query-timeout"), MILLISECONDS)) - val PacketSplitThreshold = Duration(config.getMilliseconds("akka.testconductor.packet-split-threshold"), MILLISECONDS) + implicit val BarrierTimeout = Timeout(Duration(config.getMilliseconds("barrier-timeout"), MILLISECONDS)) + implicit val QueryTimeout = Timeout(Duration(config.getMilliseconds("query-timeout"), MILLISECONDS)) + val PacketSplitThreshold = Duration(config.getMilliseconds("packet-split-threshold"), MILLISECONDS) + + private def computeWPS(config: Config): Int = + ThreadPoolConfig.scaledPoolSize( + config.getInt("pool-size-min"), + config.getDouble("pool-size-factor"), + config.getInt("pool-size-max")) + + val ServerSocketWorkerPoolSize = computeWPS(config.getConfig("netty.server-socket-worker-pool")) + + val ClientSocketWorkerPoolSize = computeWPS(config.getConfig("netty.client-socket-worker-pool")) } /** diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala index 1442558362..2a3957d146 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala @@ -143,7 +143,8 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress) val settings = TestConductor().Settings val handler = new PlayerHandler(controllerAddr, settings.ClientReconnects, settings.ReconnectBackoff, - self, Logging(context.system, "PlayerHandler"), context.system.scheduler)(context.dispatcher) + settings.ClientSocketWorkerPoolSize, self, Logging(context.system, "PlayerHandler"), + context.system.scheduler)(context.dispatcher) startWith(Connecting, Data(None, None)) @@ -254,6 +255,7 @@ private[akka] class PlayerHandler( server: InetSocketAddress, private var reconnects: Int, backoff: FiniteDuration, + poolSize: Int, fsm: ActorRef, log: LoggingAdapter, scheduler: Scheduler)(implicit executor: ExecutionContext) @@ -283,7 +285,7 @@ private[akka] class PlayerHandler( private def reconnect(): Unit = { nextAttempt = Deadline.now + backoff - RemoteConnection(Client, server, this) + RemoteConnection(Client, server, poolSize, this) } override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala index 1979857bf0..db212e7cbf 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/RemoteConnection.scala @@ -45,16 +45,18 @@ private[akka] case object Server extends Role * INTERNAL API. */ private[akka] object RemoteConnection { - def apply(role: Role, sockaddr: InetSocketAddress, handler: ChannelUpstreamHandler): Channel = { + def apply(role: Role, sockaddr: InetSocketAddress, poolSize: Int, handler: ChannelUpstreamHandler): Channel = { role match { case Client ⇒ - val socketfactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool) + val socketfactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool, + poolSize) val bootstrap = new ClientBootstrap(socketfactory) bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler)) bootstrap.setOption("tcpNoDelay", true) bootstrap.connect(sockaddr).getChannel case Server ⇒ - val socketfactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool) + val socketfactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool, + poolSize) val bootstrap = new ServerBootstrap(socketfactory) bootstrap.setPipelineFactory(new TestConductorPipelineFactory(handler)) bootstrap.setOption("reuseAddress", true) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a70106a8b2..cc8818a3be 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -244,6 +244,36 @@ akka { # suite (see enabled-algorithms section above) random-number-generator = "" } + + # (I&O) Used to configure the number of I/O worker threads on server sockets + server-socket-worker-pool { + # Min number of threads to cap factor-based number to + pool-size-min = 2 + + # The pool size factor is used to determine thread pool size + # using the following formula: ceil(available processors * factor). + # Resulting size is then bounded by the pool-size-min and + # pool-size-max values. + pool-size-factor = 1.0 + + # Max number of threads to cap factor-based number to + pool-size-max = 8 + } + + # (I&O) Used to configure the number of I/O worker threads on client sockets + client-socket-worker-pool { + # Min number of threads to cap factor-based number to + pool-size-min = 2 + + # The pool size factor is used to determine thread pool size + # using the following formula: ceil(available processors * factor). + # Resulting size is then bounded by the pool-size-min and + # pool-size-max values. + pool-size-factor = 1.0 + + # Max number of threads to cap factor-based number to + pool-size-max = 8 + } } } } 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 6e36c63024..de95ad1e8d 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -45,7 +45,8 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider val d = system.dispatchers.lookup(id) new NioClientSocketChannelFactory(d, d) case None ⇒ - new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()) + new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), + settings.ClientSocketWorkerPoolSize) } /** 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 16269a43a2..1f083bcab5 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -31,7 +31,8 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) { val d = netty.system.dispatchers.lookup(id) new NioServerSocketChannelFactory(d, d) case None ⇒ - new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()) + new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), + settings.ServerSocketWorkerPoolSize) } // group of open channels, used for clean-up 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 c9fb4aff9a..7c1ca16540 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -10,6 +10,7 @@ import java.net.InetAddress import akka.ConfigurationException import scala.collection.JavaConverters.iterableAsScalaIterableConverter import scala.concurrent.duration.FiniteDuration +import akka.dispatch.ThreadPoolConfig private[akka] class NettySettings(config: Config, val systemName: String) { @@ -139,4 +140,14 @@ private[akka] class NettySettings(config: Config, val systemName: String) { } enableSSL } + + private def computeWPS(config: Config): Int = + ThreadPoolConfig.scaledPoolSize( + config.getInt("pool-size-min"), + config.getDouble("pool-size-factor"), + config.getInt("pool-size-max")) + + val ServerSocketWorkerPoolSize = computeWPS(config.getConfig("server-socket-worker-pool")) + + val ClientSocketWorkerPoolSize = computeWPS(config.getConfig("client-socket-worker-pool")) } diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 5355433d69..45b6ad5610 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -62,7 +62,32 @@ class RemoteConfigSpec extends AkkaSpec( WriteBufferLowWaterMark must be(None) SendBufferSize must be(None) ReceiveBufferSize must be(None) + ServerSocketWorkerPoolSize must be >= (2) + ServerSocketWorkerPoolSize must be <= (8) + ClientSocketWorkerPoolSize must be >= (2) + ClientSocketWorkerPoolSize must be <= (8) } + "contain correct configuration values in reference.conf" in { + val c = system.asInstanceOf[ExtendedActorSystem]. + provider.asInstanceOf[RemoteActorRefProvider]. + remoteSettings.config.getConfig("akka.remote.netty") + + // server-socket-worker-pool + { + val pool = c.getConfig("server-socket-worker-pool") + pool.getInt("pool-size-min") must equal(2) + pool.getDouble("pool-size-factor") must equal(1.0) + pool.getInt("pool-size-max") must equal(8) + } + + // client-socket-worker-pool + { + val pool = c.getConfig("client-socket-worker-pool") + pool.getInt("pool-size-min") must equal(2) + pool.getDouble("pool-size-factor") must equal(1.0) + pool.getInt("pool-size-max") must equal(8) + } + } } } From 0d9d877b0cda932a012eb513b3ea251221d5a15a Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 30 Oct 2012 07:33:00 +0100 Subject: [PATCH 08/62] Correction of remote-tests-experimental proj name --- scripts/multi-node-log-replace.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/multi-node-log-replace.sh b/scripts/multi-node-log-replace.sh index 8e8af7112a..3bee844c2d 100755 --- a/scripts/multi-node-log-replace.sh +++ b/scripts/multi-node-log-replace.sh @@ -22,4 +22,4 @@ # check for an sbt command type -P sbt &> /dev/null || fail "sbt command not found" -sbt "project akka-remote-tests" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2" \ No newline at end of file +sbt "project akka-remote-tests-experimental" "test:run-main akka.remote.testkit.LogRoleReplace $1 $2" From 545becacfe2a0b6e98af583f74fcf90b5c890ec2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 30 Oct 2012 09:28:45 +0100 Subject: [PATCH 09/62] Use withDefaultValue instead of withDefault * Just minor boy scouting cleanup --- akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala | 2 +- .../main/scala/akka/cluster/routing/ClusterRouterConfig.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala index ae82da6407..565d50e636 100644 --- a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala +++ b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala @@ -55,7 +55,7 @@ private[akka] object SubclassifiedIndex { } private[SubclassifiedIndex] def emptyMergeMap[K, V] = internalEmptyMergeMap.asInstanceOf[Map[K, Set[V]]] - private[this] val internalEmptyMergeMap = Map[AnyRef, Set[AnyRef]]().withDefault(_ ⇒ Set[AnyRef]()) + private[this] val internalEmptyMergeMap = Map[AnyRef, Set[AnyRef]]().withDefaultValue(Set[AnyRef]()) } /** diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index bdbb1297fb..ef99ced2d8 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -202,7 +202,7 @@ private[akka] class ClusterRouteeProvider( } else { // find the node with least routees val numberOfRouteesPerNode: Map[Address, Int] = - currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefault(_ ⇒ 0)) { (acc, x) ⇒ + currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefaultValue(0)) { (acc, x) ⇒ val address = fullAddress(x) acc + (address -> (acc(address) + 1)) } From 0b2d3df67e2fea0935880b9650cb937d6c8233a0 Mon Sep 17 00:00:00 2001 From: "Kaspar Fischer (hbf)" Date: Wed, 31 Oct 2012 13:17:59 +0100 Subject: [PATCH 10/62] contribution module throttle --- akka-contrib/docs/index.rst | 1 + akka-contrib/docs/throttle.rst | 60 ++++ akka-contrib/docs/throttler.png | Bin 0 -> 4036 bytes .../throttle/TimerBasedThrottler.scala | 294 ++++++++++++++++++ .../throttle/TimerBasedThrottlerSpec.scala | 144 +++++++++ 5 files changed, 499 insertions(+) create mode 100644 akka-contrib/docs/throttle.rst create mode 100644 akka-contrib/docs/throttler.png create mode 100644 akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala create mode 100644 akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala diff --git a/akka-contrib/docs/index.rst b/akka-contrib/docs/index.rst index 9f5b57c513..5303c21c6d 100644 --- a/akka-contrib/docs/index.rst +++ b/akka-contrib/docs/index.rst @@ -29,6 +29,7 @@ The Current List of Modules .. toctree:: reliable-proxy + throttle Suggested Way of Using these Contributions ------------------------------------------ diff --git a/akka-contrib/docs/throttle.rst b/akka-contrib/docs/throttle.rst new file mode 100644 index 0000000000..ab60fb6b96 --- /dev/null +++ b/akka-contrib/docs/throttle.rst @@ -0,0 +1,60 @@ +Throttling Actor Messages +========================= + +Introduction +------------ + +Suppose you are writing an application that makes HTTP requests to an external +web service and that this web service has a restriction in place: you may not +make more than 10 requests in 1 minute. You will get blocked or need to pay if +you don’t stay under this limit. In such a scenario you will want to employ +a *message throttler*. + +This extension module provides a simple implementation of a throttling actor, +the :class:`TimerBasedThrottler`. + + +How to use it +------------- + +You can use a :class:`TimerBasedThrottler` as follows: + +.. includecode:: @contribSrc@/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala#demo-code + +Please refer to the ScalaDoc documentation for the details. + + +The guarantees +-------------- + +:class:`TimerBasedThrottler` uses a timer internally. When the throttler’s rate is 3 msg/s, +for example, the throttler will start a timer that triggers +every second and each time will give the throttler exactly three "vouchers"; +each voucher gives the throttler a right to deliver a message. In this way, +at most 3 messages will be sent out by the throttler in each interval. + +It should be noted that such timer-based throttlers provide relatively **weak guarantees**: + +* Only *start times* are taken into account. This may be a problem if, for example, the + throttler is used to throttle requests to an external web service. If a web request + takes very long on the server then the rate *observed on the server* may be higher. +* A timer-based throttler only makes guarantees for the intervals of its own timer. In + our example, no more than 3 messages are delivered within such intervals. Other + intervals on the timeline, however, may contain more calls. + +The two cases are illustrated in the two figures below, each showing a timeline and three +intervals of the timer. The message delivery times chosen by the throttler are indicated +by dots, and as you can see, each interval contains at most 3 point, so the throttler +works correctly. Still, there is in each example an interval (the red one) that is +problematic. In the first scenario, this is because the delivery times are merely the +start times of longer requests (indicated by the four bars above the timeline that start +at the dots), so that the server observes four requests during the red interval. In the +second scenario, the messages are centered around one of the points in time where the +timer triggers, causing the red interval to contain too many messages. + +.. image:: throttler.png + +For some application scenarios, the guarantees provided by a timer-based throttler might +be too weak. Charles Cordingley’s `blog post `_ +discusses a throttler with stronger guarantees (it solves problem 2 from above). +Future versions of this module may feature throttlers with better guarantees. \ No newline at end of file diff --git a/akka-contrib/docs/throttler.png b/akka-contrib/docs/throttler.png new file mode 100644 index 0000000000000000000000000000000000000000..eab1a52a34e7435e1e6cafcf3e6c511dab45f078 GIT binary patch literal 4036 zcmeAS@N?(olHy`uVBq!ia0y~yV6V=e zG&%VkzHsT1)8xie3?lpj*Pc5R{4na8>)4^_P?Q-cIpySu6rUOW2FZOa%+3FU*x07o za~MvJV{2oz;^FxdESQwl#>2z(i-+g;GX{pk>jUxw3M6c`&)d_*F{I+w+c})!BFCj1?w=QObu483&Z=y{wMbvXyQ!0_LrA&lb*EIz3J$In z9f8f z-E_m`HjUo$(xvSeMK6e6xcYO&`t|w}MqJ$7FB3~x8A@0k!WqD*ed)4gYI&cZefNL* z^l6FOu0t0-1qWVSP{UsQ@{y~Ra4cgiW9-7|eSLkm4ldZcckVeCs6GaeDW>9g@7`Ug zsQ7+|$a7qXpUw3tq2K zH#RQ*x2Mu?(kdPg2->+kRG?q0R=dfktMUwY4;J{{bfn~`Bq zo6Ik3rJ{RxU+wObrC0ZVj@g;i`cgkDF>zs5&Z&dV%kp!Qk``sXSsT6G=l+eQjEt9p zt`|POP}(w4YMy<4U2I55KtRBkU8UkhMfZN1FD)xwyK?2t&FTIb9v%@H8Hc9FU%P(& zd;E!BX>+sV$JTLh{YtsIN?B6Uf4<$@{hywno7>&h6EgMxw?nOoza8IXT-dsFsj97R z?$uSHMde42AHBP$a`XHLn>KCAjo!|-K1TB2$Kyqfa<)}1>-d&s2M13+nzXUXQUAdG z|B1z)7rfB&ZYWGXF2z0F``_12;j+#Dzg~}jy?F6jt;d$Rw_exmsQT^Jx|5irgh5wG2!;{@w?Yw`K2HE ziSO`~sq1#`+_}Cw_51Jl`?sH!lb6pozma@tN%i-4H+Q{#{W#e?PR{(W!LhzI>x?`F z@4qptf8Hk>c5r>Qbj_aC*-sDDy=qveeBOMy|LR?p`Vj$PQCgw8nwm51^XFMOo_53QGI{l;L@zampWak)F7CpGM)Ol8#&-``gRHs+0IG;Mz zMXoT&&-U)=)qD5Gn*aas@bD_HBE^yob^p{IM$_gMSX>E4fu2M_*bIqiHwLZ{p0`A^;j zQc=BUrx)&<|6q0adZ}jn?oP=yuV1g&wQTwF)5UjZs8v_22$2_HI_3ZjLHw;T-UZq8!|NHyy!}fn)3tT~)wQ*~ubzGMe0kolSFf_>?^~^*efxXlZm+|sUTfqJY!aKw8Ig7W zg4Xw$JuS!fgocJjoqm0Ny?5%rKcCNcY|XxUa0ajRnscQ##=*5)ZX~T-xiWC-_3a5| zW!rXMUpH~$#8+=FUc9KQt9$9spFcJIo|CUQz7&wv)DC@myXE6!{n$Mkckf-hCMGVv zT&cOa`DRuTPwdl!Ys3w*n8W@@UrSE9Uj2?Iz5a7=;VIqi-Tyu}?Om&|w|3F7I9r_) zJ=?c!ySD8U_p|fcqbn;P?Qwo8>m6Qc_VVA9hk@s_-{hTs`|kF`;JGW^y-!AjMRl#? z`@AnC_|%sp-=gQ-xf8Rmyi_H&_4)bv>1k9MWxbw5*!bJqNyq{{g3-^aqjzyIUIQ-aEWeoCFm{QKvx zf7L%ft)DdMSN(HGEq(577bZ*ypL+gO)$>!&w~sx^;W!ofofe z{5gJm_l>(}C4+t@aZi_yy(k*X7~9@@OZVf|iGt@QyefP5qzI|h2N{CRV3bM$q6l2i zKxzqm0uVXWS_L9TqyZRKKy>3Z@pVGrrPFh*%YE+eT)TGds*S$BzRU98?5+MDl6?F2 zZQuJhR;|*?%*@POd9t+j=clEoXBbWjTNjb|t2-(pVujS~q7MsRO3$7#V}(@p>ub7~ z&C4REf?lXYH!5>1al#PnSMq_Uy91-NcEGM^n<$ zZn1d?t)A<3{ajyPYGdQJzXsQis{TJEx;kOz@*8g>=DT*^x^?Tq)s|hya&9b;GD`pU z=JE0V(+%5pK}}N@_tl6AD>I&-Z$JOt)1`eU&Q6>+ZCmp1 zvf9aWtxCO;UcGvCtXFom?70&sI{Nyqna}s>nd^T2%9W7PZztvC+Jk~qYXz^FfBg9A zi^;pXy%ism?%mn>YB9h4pATnd8cV)>X`~wH{_V}p$9r~7n=(b_pZc!orKkCGv4LXWG)6|H;|)VC{h$rEOF zzKoAyt?j3c{f^$*QRpl?$FO$Sl+8*S9)EY5D=u8#zu3KRMYGYZITCJL*VYL~%Z^utR;*cCKDk}O_xaQ}l)Z~JYlqU?}U^;KPT4^y^#q?fZAj{^a@d;cGvA z{<`26u{P4Lu zmc>_YKE1z6{?@Hqe}0>Bq{OaO4*soo$|?W7y8pZt4-Pdk?Oks-;j-ud)h9Q+b>F&m zYmRBJ)wg$dS4$b)yBeC))7?G$&BKL#YhpJ)>y7sP^=w+qBy%-2w^J9tPh%-5DLFIC zGFUb;XJZHRZ1enU(Y`lt-V{>S)~j`y)>ovw?yYv%q#y5XqJ1qbEq6W44qJb3Tj}a| zw)0)Pzqsv6`nf4}ec8f2d$#4>+@zbpyiT+s~9 zi7(I1$++-YA$d~!+^IKSWX-#-a%A=TNt2fO%v*Cmv7^6#dg>Cs7ynH;XLfq-iGIH% zYSI+trJ3QPMov=Sr>XhQ(U6PUt2S}^^yT|!PB1Ax*7N!KdH(wRo*nDz^S{ouSjZk) zsG0Qk*7d#XuH3n^C-HBU?c$H0zWDfiOD|ac;?)6xfB5IT`>|U> T<$x{&0|SGntDnm{r-UW|g*xXE literal 0 HcmV?d00001 diff --git a/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala new file mode 100644 index 0000000000..12e98e89b2 --- /dev/null +++ b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala @@ -0,0 +1,294 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.contrib.throttle + +import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.util.control.NonFatal +import scala.collection.immutable.{ Queue ⇒ Q } +import akka.actor.{ ActorRef, Actor, FSM } +import Throttler._ +import TimerBasedThrottler._ +import java.util.concurrent.TimeUnit +import akka.AkkaException + +/** + * Marker trait for throttlers. + * + * == Throttling == + * A throttler is an actor that is defined through a target actor and a rate + * (of type [[akka.contrib.throttle.Throttler.Rate]]). You set or change the target and rate at any time through the `SetTarget(target)` + * and `SetRate(rate)` messages, respectively. When you send the throttler any other message `msg`, it will + * put the message `msg` into an internal queue and eventually send all queued messages to the target, at + * a speed that respects the given rate. If no target is currently defined then the messages will be queued + * and will be delivered as soon as a target gets set. + * + * A [[akka.contrib.throttle.Throttler]] understands actor messages of type + * [[akka.contrib.throttle.Throttler.SetTarget]], [[akka.contrib.throttle.Throttler.SetRate]], in + * addition to any other messages, which the throttler will consider as messages to be sent to + * the target. + * + * == Transparency == + * Notice that the throttler `forward`s messages, i.e., the target will see the original message sender (and not the throttler) as the sender of the message. + * + * == Persistence == + * Throttlers usually use an internal queue to keep the messages that need to be sent to the target. + * You therefore cannot rely on the throttler's inbox size in order to learn how much messages are + * outstanding. + * + * It is left to the implementation whether the internal queue is persisted over application restarts or + * actor failure. + * + * == Processing messages == + * The target should process messages as fast as possible. If the target requires substantial time to + * process messages, it should distribute its work to other actors (using for example something like + * a `BalancingDispatcher`), otherwise the resulting system will always work below + * the threshold rate. + * + * Example: Suppose the throttler has a rate of 3msg/s and the target requires 1s to process a message. + * This system will only process messages at a rate of 1msg/s: the target will receive messages at at most 3msg/s + * but as it handles them synchronously and each of them takes 1s, its inbox will grow and grow. In such + * a situation, the target should distribute its messages to a set of worker actors so that individual messages + * can be handled in parallel. + * + * @see [[akka.contrib.throttle.TimerBasedThrottler]] + */ +trait Throttler { self: Actor ⇒ } + +/** + * Message types understood by [[akka.contrib.throttle.Throttler]]'s. + * + * @see [[akka.contrib.throttle.Throttler]] + * @see [[akka.contrib.throttle.Throttler.Rate]] + */ +object Throttler { + /** + * A rate used for throttling. + * + * There are some shorthands available to construct rates: + * {{{ + * import java.util.concurrent.TimeUnit._ + * import scala.concurrent.duration.{ Duration, FiniteDuration } + * + * val rate1 = 1 msgsPer (1, SECONDS) + * val rate2 = 1 msgsPer Duration(1, SECONDS) + * val rate3 = 1 msgsPer (1 seconds) + * val rate4 = 1 msgsPerSecond + * val rate5 = 1 msgsPerMinute + * val rate6 = 1 msgsPerHour + * }}} + * + * @param numberOfCalls the number of calls that may take place in a period + * @param duration the length of the period + * @see [[akka.contrib.throttle.Throttler]] + */ + case class Rate(val numberOfCalls: Int, val duration: FiniteDuration) { + /** + * The duration in milliseconds. + */ + def durationInMillis(): Long = duration.toMillis + } + + /** + * Set the target of a [[akka.contrib.throttle.Throttler]]. + * + * You may change a throttler's target at any time. + * + * Notice that the messages sent by the throttler to the target will have the original sender (and + * not the throttler) as the sender. (In Akka terms, the throttler `forward`s the message.) + * + * @param target if `target` is `None`, the throttler will stop delivering messages and the messages already received + * but not yet delivered, as well as any messages received in the future will be queued + * and eventually be delivered when a new target is set. If `target` is not `None`, the currently queued messages + * as well as any messages received in the the future will be delivered to the new target at a rate not exceeding the current throttler's rate. + */ + case class SetTarget(target: Option[ActorRef]) + + /** + * Set the rate of a [[akka.contrib.throttle.Throttler]]. + * + * You may change a throttler's rate at any time. + * + * @param rate the rate at which messages will be delivered to the target of the throttler + */ + case class SetRate(rate: Rate) + + import language.implicitConversions + + /** + * Helper for some syntactic sugar. + * + * @see [[akka.contrib.throttle.Throttler.Rate]] + */ + implicit class RateInt(numberOfCalls: Int) { + def msgsPer(duration: Int, timeUnit: TimeUnit) = Rate(numberOfCalls, Duration(duration, timeUnit)) + def msgsPer(duration: FiniteDuration) = Rate(numberOfCalls, duration) + def msgsPerSecond = Rate(numberOfCalls, Duration(1, TimeUnit.SECONDS)) + def msgsPerMinute = Rate(numberOfCalls, Duration(1, TimeUnit.MINUTES)) + def msgsPerHour = Rate(numberOfCalls, Duration(1, TimeUnit.HOURS)) + } + +} + +/** + * Implementation-specific internals. + */ +object TimerBasedThrottler { + private[throttle] case object Tick + + // States of the FSM + private[throttle] sealed trait State + // Idle means we don't deliver messages, either because there are none, or because no target was set. + private[throttle] case object Idle extends State + // Active means we the target is set and we have a message queue that is non-empty. + private[throttle] case object Active extends State + + // Messages, as we queue them to be sent later + private[throttle] case class Message(message: Any, sender: ActorRef) + + // The data of the FSM + private[throttle] sealed case class Data(target: Option[ActorRef], + callsLeftInThisPeriod: Int, + queue: Q[Message]) +} + +/** + * A [[akka.contrib.throttle.Throttler]] that uses a timer to control the message delivery rate. + * + * ==Example== + * For example, if you set a rate like "3 messages in 1 second", the throttler + * will send the first three messages immediately to the target actor but will need to impose a delay before + * sending out further messages: + * {{{ + * // A simple actor that prints whatever it receives + * val printer = system.actorOf(Props(new Actor { + * def receive = { + * case x => println(x) + * } + * })) + * // The throttler for this example, setting the rate + * val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second)))) + * // Set the target + * throttler ! SetTarget(Some(printer)) + * // These three messages will be sent to the printer immediately + * throttler ! "1" + * throttler ! "2" + * throttler ! "3" + * // These two will wait at least until 1 second has passed + * throttler ! "4" + * throttler ! "5" + * }}} + * + * ==Implementation notes== + * This throttler implementation internally installs a timer that repeats every `rate.durationInMillis` and enables `rate.numberOfCalls` + * additional calls to take place. A `TimerBasedThrottler` uses very few system resources, provided the rate's duration is not too + * fine-grained (which would cause a lot of timer invocations); for example, it does not store the calling history + * as other throttlers may need to do. + * + * However, a `TimerBasedThrottler` only provides ''weak guarantees'' on the rate (see also + * this blog post): + * + * - Only ''delivery'' times are taken into account: if, for example, the throttler is used to throttle + * requests to an external web service then only the start times of the web requests are considered. + * If a web request takes very long on the server then more than `rate.numberOfCalls`-many requests + * may be observed on the server in an interval of duration `rate.durationInMillis()`. + * - There may be intervals of duration `rate.durationInMillis()` that contain more than `rate.numberOfCalls` + * message deliveries: a `TimerBasedThrottler` only makes guarantees for the intervals + * of its ''own'' timer, namely that no more than `rate.numberOfCalls`-many messages are delivered within such intervals. Other intervals on the + * timeline may contain more calls. + * + * For some applications, these guarantees may not be sufficient. + * + * ==Known issues== + * + * - If you change the rate using `SetRate(rate)`, the actual rate may in fact be higher for the + * overlapping period (i.e., `durationInMillis()`) of the new and old rate. Therefore, + * changing the rate frequently is not recommended with the current implementation. + * - The queue of messages to be delivered is not persisted in any way; actor or system failure will + * cause the queued messages to be lost. + * + * @see [[akka.contrib.throttle.Throttler]] + */ +class TimerBasedThrottler(var rate: Rate) extends Actor with Throttler with FSM[State, Data] { + + startWith(Idle, Data(None, rate.numberOfCalls, Q())) + + // Idle: no messages, or target not set + when(Idle) { + // Set the rate + case Event(SetRate(rate), d) ⇒ + this.rate = rate + stay using d.copy(callsLeftInThisPeriod = rate.numberOfCalls) + + // Set the target + case Event(SetTarget(t @ Some(_)), d) if !d.queue.isEmpty ⇒ + goto(Active) using deliverMessages(d.copy(target = t)) + case Event(SetTarget(t), d) ⇒ + stay using d.copy(target = t) + + // Queuing + case Event(msg, d @ Data(None, _, queue)) ⇒ + stay using d.copy(queue = queue.enqueue(Message(msg, context.sender))) + case Event(msg, d @ Data(Some(_), _, Seq())) ⇒ + goto(Active) using deliverMessages(d.copy(queue = Q(Message(msg, context.sender)))) + // Note: The case Event(msg, t @ Data(Some(_), _, _, Seq(_*))) should never happen here. + } + + when(Active) { + // Set the rate + case Event(SetRate(rate), d) ⇒ + this.rate = rate + // Note: this should be improved (see "Known issues" in class comments) + stopTimer() + startTimer(rate) + stay using d.copy(callsLeftInThisPeriod = rate.numberOfCalls) + + // Set the target (when the new target is None) + case Event(SetTarget(None), d) ⇒ + goto(Idle) using d.copy(target = None) + + // Set the target (when the new target is not None) + case Event(SetTarget(t @ Some(_)), d) ⇒ + stay using d.copy(target = t) + + // Period ends and we have no more messages + case Event(Tick, d @ Data(_, _, Seq())) ⇒ + goto(Idle) + + // Period ends and we get more occasions to send messages + case Event(Tick, d @ Data(_, _, _)) ⇒ + stay using deliverMessages(d.copy(callsLeftInThisPeriod = rate.numberOfCalls)) + + // Queue a message (when we cannot send messages in the current period anymore) + case Event(msg, d @ Data(_, 0, queue)) ⇒ + stay using d.copy(queue = queue.enqueue(Message(msg, context.sender))) + + // Queue a message (when we can send some more messages in the current period) + case Event(msg, d @ Data(_, _, queue)) ⇒ + stay using deliverMessages(d.copy(queue = queue.enqueue(Message(msg, context.sender)))) + } + + onTransition { + case Idle -> Active ⇒ startTimer(rate) + case Active -> Idle ⇒ stopTimer() + } + + initialize + + private def startTimer(rate: Rate) = setTimer("morePermits", Tick, rate.duration, true) + private def stopTimer() = cancelTimer("morePermits") + + /** + * Send as many messages as we can (while respecting the rate) to the target and + * return the state data (with the queue containing the remaining ones). + */ + private def deliverMessages(data: Data): Data = { + val queue = data.queue + val nrOfMsgToSend = scala.math.min(queue.length, data.callsLeftInThisPeriod) + + queue.take(nrOfMsgToSend).foreach(x ⇒ data.target.get.tell(x.message, x.sender)) + + data.copy(queue = queue.drop(nrOfMsgToSend), callsLeftInThisPeriod = data.callsLeftInThisPeriod - nrOfMsgToSend) + } +} \ No newline at end of file diff --git a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala new file mode 100644 index 0000000000..4f8f422cb6 --- /dev/null +++ b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala @@ -0,0 +1,144 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.contrib.throttle + +import language.postfixOps +import scala.concurrent.duration._ +import akka.actor.ActorSystem +import akka.actor.Actor +import akka.actor.Props +import akka.testkit.TestKit +import akka.testkit.ImplicitSender +import akka.contrib.throttle.Throttler._ +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import org.scalatest.WordSpec +import org.scalatest.matchers.MustMatchers +import org.scalatest.BeforeAndAfterAll + +object TimerBasedThrottlerSpec { + class EchoActor extends Actor { + def receive = { + case x ⇒ sender ! x + } + } +} + +@RunWith(classOf[JUnitRunner]) +class TimerBasedThrottlerSpec(_system: ActorSystem) extends TestKit(_system) with ImplicitSender + with WordSpec with MustMatchers with BeforeAndAfterAll { + + def this() = this(ActorSystem("TimerBasedThrottlerSpec")) + + override def afterAll { + system.shutdown() + } + + "A throttler" must { + + "must pass the ScalaDoc class documentation example prgoram" in { + //#demo-code + // A simple actor that prints whatever it receives + val printer = system.actorOf(Props(new Actor { + def receive = { + case x ⇒ println(x) + } + })) + // The throttler for this example, setting the rate + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + // Set the target + throttler ! SetTarget(Some(printer)) + // These three messages will be sent to the echoer immediately + throttler ! "1" + throttler ! "2" + throttler ! "3" + // These two will wait until a second has passed + throttler ! "4" + throttler ! "5" + //#demo-code + } + + "keep messages until a target is set" in { + val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + throttler ! "1" + throttler ! "2" + throttler ! "3" + throttler ! "4" + throttler ! "5" + throttler ! "6" + expectNoMsg(1 second) + throttler ! SetTarget(Some(echo)) + within(2.seconds) { + expectMsg("1") + expectMsg("2") + expectMsg("3") + expectMsg("4") + expectMsg("5") + expectMsg("6") + } + } + + "respect the rate (3 msg/s)" in { + val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + throttler ! SetTarget(Some(echo)) + throttler ! "1" + throttler ! "2" + throttler ! "3" + throttler ! "4" + throttler ! "5" + throttler ! "6" + throttler ! "7" + within(1 second) { + expectMsg("1") + expectMsg("2") + expectMsg("3") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("4") + expectMsg("5") + expectMsg("6") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("7") + } + } + + "respect the rate (4 msg/s)" in { + val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) + val throttler = system.actorOf(Props(new TimerBasedThrottler(4 msgsPer (1 second)))) + throttler ! SetTarget(Some(echo)) + throttler ! "1" + throttler ! "2" + throttler ! "3" + throttler ! "4" + throttler ! "5" + throttler ! "6" + throttler ! "7" + throttler ! "8" + throttler ! "9" + within(1 second) { + expectMsg("1") + expectMsg("2") + expectMsg("3") + expectMsg("4") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("5") + expectMsg("6") + expectMsg("7") + expectMsg("8") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("9") + } + } + } +} \ No newline at end of file From 0615ba78970a52fc0b3b374c2e95a6a2278592e1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 31 Oct 2012 14:20:20 +0100 Subject: [PATCH 11/62] #2669 - Fixing the docs about how to generate the docs --- akka-docs/rst/dev/documentation.rst | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/akka-docs/rst/dev/documentation.rst b/akka-docs/rst/dev/documentation.rst index ad8ff244ae..18b6707f43 100644 --- a/akka-docs/rst/dev/documentation.rst +++ b/akka-docs/rst/dev/documentation.rst @@ -75,16 +75,17 @@ First install `Sphinx`_. See below. Building -------- -:: +For the html version of the docs:: - cd akka-docs + sbt sphinx:generate-html - make html - open _build/html/index.html + open /akka-docs/target/sphinx/html/index.html - make pdf - open _build/latex/Akka.pdf +For the pdf version of the docs:: + sbt sphinx:generate-pdf + + open /akka-docs/target/sphinx/latex/Akka.pdf Installing Sphinx on OS X ------------------------- From 977194ff8e7b895ebcfcd827b1f2e31e92b44a9a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Wed, 31 Oct 2012 16:37:03 +0100 Subject: [PATCH 12/62] Improve MultiNodeSpec ifNode syntax. #2126 --- .../scala/akka/cluster/LargeClusterSpec.scala | 6 +++--- .../scala/akka/cluster/MultiNodeClusterSpec.scala | 2 +- .../scala/akka/remote/testkit/MultiNodeSpec.scala | 14 +++++++------- .../remote/testconductor/TestConductorSpec.scala | 7 ++----- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala index 1ebb930df6..06e3caeac8 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LargeClusterSpec.scala @@ -148,7 +148,7 @@ abstract class LargeClusterSpec runOn(runOnRoles: _*) { systems.size must be(nodesPerDatacenter) // make sure it is initialized - val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet) + val clusterNodes = if(isNode(from)) joiningClusterNodes else systems.map(Cluster(_)).toSet val startGossipCounts = Map.empty[Cluster, Long] ++ clusterNodes.map(c ⇒ (c -> c.readView.latestStats.receivedGossipCount)) def gossipCount(c: Cluster): Long = { @@ -260,7 +260,7 @@ abstract class LargeClusterSpec if (bulk.nonEmpty) { val totalNodes = nodesPerDatacenter * 4 + bulk.size within(expectedMaxDuration(totalNodes)) { - val joiningClusters = ifNode(fifthDatacenter)(bulk.map(Cluster(_)).toSet)(Set.empty) + val joiningClusters = if(isNode(fifthDatacenter)) bulk.map(Cluster(_)).toSet else Set.empty[Cluster] join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) enterBarrier("fifth-datacenter-joined-" + bulk.size) @@ -270,7 +270,7 @@ abstract class LargeClusterSpec for (i ← 0 until oneByOne.size) { val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1 within(expectedMaxDuration(totalNodes)) { - val joiningClusters = ifNode(fifthDatacenter)(Set(Cluster(oneByOne(i))))(Set.empty) + val joiningClusters = if(isNode(fifthDatacenter)) Set(Cluster(oneByOne(i))) else Set.empty[Cluster] join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes, runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) enterBarrier("fifth-datacenter-joined-" + (bulk.size + i)) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala index 5ff931f1c2..12fc8ebbc6 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -210,7 +210,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS val expectedLeader = roleOfLeader(nodesInCluster) val leader = clusterView.leader val isLeader = leader == Some(clusterView.selfAddress) - assert(isLeader == ifNode(expectedLeader)(true)(false), + assert(isLeader == isNode(expectedLeader), "expectedLeader [%s], got leader [%s], members [%s]".format(expectedLeader, leader, clusterView.members)) clusterView.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving)) } diff --git a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala index 9d5fd4b55e..c40a4e5a7b 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala @@ -42,7 +42,10 @@ abstract class MultiNodeConfig { /** * Register a config override for a specific participant. */ - def nodeConfig(role: RoleName, config: Config): Unit = _nodeConf += role -> config + def nodeConfig(roles: RoleName*)(configs: Config*): Unit = { + val c = configs.reduceLeft(_ withFallback _) + _nodeConf ++= roles map { _ -> c } + } /** * Include for verbose debug logging @@ -318,18 +321,15 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: * to the `roleMap`). */ def runOn(nodes: RoleName*)(thunk: ⇒ Unit): Unit = { - if (nodes exists (_ == myself)) { + if (isNode(nodes: _*)) { thunk } } /** - * Execute the `yes` block of code only on the given nodes (names according - * to the `roleMap`) else execute the `no` block of code. + * Verify that the running node matches one of the given nodes */ - def ifNode[T](nodes: RoleName*)(yes: ⇒ T)(no: ⇒ T): T = { - if (nodes exists (_ == myself)) yes else no - } + def isNode(nodes: RoleName*): Boolean = nodes contains myself /** * Enter the named barriers in the order given. Use the remaining duration from diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala index 1b5ac90772..544ee03ead 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/testconductor/TestConductorSpec.scala @@ -89,11 +89,8 @@ class TestConductorSpec extends MultiNodeSpec(TestConductorMultiJvmSpec) with ST } val (min, max) = - ifNode(master) { - (0 seconds, 500 millis) - } { - (0.6 seconds, 2 seconds) - } + if(isNode(master))(0 seconds, 500 millis) + else (0.6 seconds, 2 seconds) within(min, max) { expectMsg(500 millis, 10) From 58c0b2beae6a94f5f0a3a6631a105401447861e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Thu, 1 Nov 2012 11:04:05 +0100 Subject: [PATCH 13/62] Correct and update the SbtMultiJvm documentation. #2671 --- akka-docs/rst/dev/multi-jvm-testing.rst | 107 +++++++++++++++--------- project/plugins.sbt | 2 +- 2 files changed, 67 insertions(+), 42 deletions(-) diff --git a/akka-docs/rst/dev/multi-jvm-testing.rst b/akka-docs/rst/dev/multi-jvm-testing.rst index 8157e6bd84..92a4e9c6b1 100644 --- a/akka-docs/rst/dev/multi-jvm-testing.rst +++ b/akka-docs/rst/dev/multi-jvm-testing.rst @@ -18,52 +18,65 @@ You can add it as a plugin by adding the following to your project/plugins.sbt: .. includecode:: ../../../project/plugins.sbt#sbt-multi-jvm You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm`` -settings and config. For example, here is an example of how the akka-remote-tests project adds -multi-JVM testing (Simplified for clarity): +settings and config. Please note that MultiJvm test sources are located in ``src/multi-jvm/...``, +and not in ``src/test/...``. + +Here is an example Build.scala file that uses the MultiJvm plugin: .. parsed-literal:: import sbt._ import Keys._ import com.typesafe.sbt.SbtMultiJvm - import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.{ MultiJvm, extraOptions } + import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.{ MultiJvm } - object AkkaBuild extends Build { + object ExampleBuild extends Build { - lazy val remoteTests = Project( - id = "akka-remote-tests", - base = file("akka-remote-tests"), - dependencies = Seq(remote, actorTests % "test->test", - testkit % "test->test"), - settings = defaultSettings ++ Seq( - // disable parallel tests - parallelExecution in Test := false, - extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => - (name: String) => (src ** (name + ".conf")).get. - headOption.map("-Dakka.config=" + _.absolutePath).toSeq - }, - executeTests in Test <<= ((executeTests in Test), - (executeTests in MultiJvm)) map { - case ((_, testResults), (_, multiJvmResults)) => - val results = testResults ++ multiJvmResults - (Tests.overall(results.values), results) - } - ) - ) configs (MultiJvm) + lazy val buildSettings = Defaults.defaultSettings ++ multiJvmSettings ++ Seq( + organization := "example", + version := "1.0", + scalaVersion := "@scalaVersion@", + // make sure that the artifacts don't have the scala version in the name + crossPaths := false + ) - lazy val buildSettings = Defaults.defaultSettings ++ - SbtMultiJvm.multiJvmSettings ++ Seq( - organization := "com.typesafe.akka", - version := "@version@", - scalaVersion := "@scalaVersion@", - crossPaths := false - ) + lazy val example = Project( + id = "example", + base = file("."), + settings = buildSettings ++ + Seq(libraryDependencies ++= Dependencies.example) + ) configs(MultiJvm) - lazy val defaultSettings = buildSettings ++ Seq( - resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" - ) + lazy val multiJvmSettings = SbtMultiJvm.multiJvmSettings ++ Seq( + // make sure that MultiJvm test are compiled by the default test compilation + compile in MultiJvm <<= (compile in MultiJvm) triggeredBy (compile in Test), + // disable parallel tests + parallelExecution in Test := false, + // make sure that MultiJvm tests are executed by the default test target + executeTests in Test <<= + ((executeTests in Test), (executeTests in MultiJvm)) map { + case ((_, testResults), (_, multiJvmResults)) => + val results = testResults ++ multiJvmResults + (Tests.overall(results.values), results) + } + ) - } + object Dependencies { + val example = Seq( + // ---- application dependencies ---- + "com.typesafe.akka" %% "akka-actor" % "@version@" @crossString@, + "com.typesafe.akka" %% "akka-remote" % "@version@" @crossString@, + + // ---- test dependencies ---- + "com.typesafe.akka" %% "akka-testkit" % "@version@" % + "test" cross CrossVersion.full, + "com.typesafe.akka" %% "akka-remote-tests-experimental" % "@version@" % + "test" cross CrossVersion.full, + "org.scalatest" %% "scalatest" % "1.8" % "test" cross CrossVersion.full, + "junit" % "junit" % "4.5" % "test" + ) + } + } You can specify JVM options for the forked JVMs:: @@ -73,7 +86,7 @@ You can specify JVM options for the forked JVMs:: Running tests ============= -The multi-jvm tasks are similar to the normal tasks: ``test``, ``test-only``, +The multi-JVM tasks are similar to the normal tasks: ``test``, ``test-only``, and ``run``, but are under the ``multi-jvm`` configuration. So in Akka, to run all the multi-JVM tests in the akka-remote project use (at @@ -111,8 +124,8 @@ options after the test names and ``--``. For example: Creating application tests ========================== -The tests are discovered, and combined, through a naming convention. MultiJvm tests are -located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern: +The tests are discovered, and combined, through a naming convention. MultiJvm test sources +are located in ``src/multi-jvm/...``. A test is named with the following pattern: .. code-block:: none @@ -162,14 +175,26 @@ spawned, one for each node. It will look like this: [success] Total time: ... -Naming -====== +Changing Defaults +================= + +You can chenge the name of the multi-JVM test source directory by adding the following +configuration to your project: + +.. code-block:: none + + unmanagedSourceDirectories in MultiJvm <<= + Seq(baseDirectory(_ / "src/some_directory_here")).join + You can change what the ``MultiJvm`` identifier is. For example, to change it to -``ClusterTest`` use the ``multiJvmMarker`` setting:: +``ClusterTest`` use the ``multiJvmMarker`` setting: + +.. code-block:: none multiJvmMarker in MultiJvm := "ClusterTest" + Your tests should now be named ``{TestName}ClusterTest{NodeName}``. diff --git a/project/plugins.sbt b/project/plugins.sbt index 6f68c66496..8e57ed4e2d 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,7 +3,7 @@ resolvers += Classpaths.typesafeResolver // these comment markers are for including code into the docs //#sbt-multi-jvm -addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.3") +addSbtPlugin("com.typesafe.sbt" % "sbt-multi-jvm" % "0.3.4") //#sbt-multi-jvm addSbtPlugin("com.typesafe.sbt" % "sbt-scalariform" % "1.0.0") From 2e2d367ae6136f2e4561a6eeb91f55120c05ce54 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 1 Nov 2012 19:27:53 +0100 Subject: [PATCH 14/62] Missing variable assignment, see #2677 * Credit goes to Eric Pederson --- .../sample/cluster/transformation/TransformationSample.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala index 5deb2d4a40..159e17f94b 100644 --- a/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala +++ b/akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/transformation/TransformationSample.scala @@ -68,7 +68,8 @@ class TransformationFrontend extends Actor { context watch sender backends = backends :+ sender - case Terminated(a) ⇒ backends.filterNot(_ == a) + case Terminated(a) ⇒ + backends = backends.filterNot(_ == a) } } //#frontend From 7918a6f72b66e6aae8d3cf69e98dbe700cf13256 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 2 Nov 2012 18:48:37 +0100 Subject: [PATCH 15/62] Adding pool size for both use with dispatcher as without. --- .../main/scala/akka/remote/netty/NettyRemoteSupport.scala | 8 ++++---- akka-remote/src/main/scala/akka/remote/netty/Server.scala | 5 ++--- 2 files changed, 6 insertions(+), 7 deletions(-) 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 de95ad1e8d..7c27a82e6d 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -40,13 +40,13 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider // TODO replace by system.scheduler val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory) - val clientChannelFactory = settings.UseDispatcherForIO match { + val clientChannelFactory = { + settings.UseDispatcherForIO match { case Some(id) ⇒ val d = system.dispatchers.lookup(id) - new NioClientSocketChannelFactory(d, d) + new NioClientSocketChannelFactory(d, d, settings.ClientSocketWorkerPoolSize) case None ⇒ - new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), - settings.ClientSocketWorkerPoolSize) + new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), settings.ClientSocketWorkerPoolSize) } /** 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 1f083bcab5..2f76b324cd 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -29,10 +29,9 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) { settings.UseDispatcherForIO match { case Some(id) ⇒ val d = netty.system.dispatchers.lookup(id) - new NioServerSocketChannelFactory(d, d) + new NioServerSocketChannelFactory(d, d, settings.ServerSocketWorkerPoolSize) case None ⇒ - new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), - settings.ServerSocketWorkerPoolSize) + new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), settings.ServerSocketWorkerPoolSize) } // group of open channels, used for clean-up From 8eec825f18e776ff3b96ca688b15644175d25fd2 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 4 Nov 2012 02:57:30 +0100 Subject: [PATCH 16/62] paragraph is not a dollar. I promise. --- .../src/test/scala/akka/actor/LocalActorRefProviderSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala index a768089415..4cb432aa23 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala @@ -46,7 +46,7 @@ class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.confi def receive = { case "lookup" ⇒ if (childName == child.path.name) sender ! context.actorFor(childName) - else sender ! s"§childName is not ${child.path.name}!" + else sender ! s"$childName is not ${child.path.name}!" } })) a.tell("lookup", testActor) From 29cf71d6bc2e05ce65ae0e51b613aef475abb87a Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 5 Nov 2012 13:33:14 +0100 Subject: [PATCH 17/62] make timing requirements for lenient for ReliableProxySpec, see #2637 --- .../scala/akka/contrib/pattern/ReliableProxySpec.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-contrib/src/multi-jvm/scala/akka/contrib/pattern/ReliableProxySpec.scala b/akka-contrib/src/multi-jvm/scala/akka/contrib/pattern/ReliableProxySpec.scala index 3595b9887a..f71bb0116b 100644 --- a/akka-contrib/src/multi-jvm/scala/akka/contrib/pattern/ReliableProxySpec.scala +++ b/akka-contrib/src/multi-jvm/scala/akka/contrib/pattern/ReliableProxySpec.scala @@ -122,8 +122,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod enterBarrier("test2b") runOn(local) { - testConductor.throttle(local, remote, Direction.Send, -1) - expectTransition(Active, Idle) + testConductor.throttle(local, remote, Direction.Send, -1).await + within(5 seconds) { expectTransition(Active, Idle) } } runOn(remote) { within(1 second) { @@ -152,8 +152,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod enterBarrier("test3a") runOn(local) { - testConductor.throttle(local, remote, Direction.Receive, -1) - expectTransition(Active, Idle) + testConductor.throttle(local, remote, Direction.Receive, -1).await + within(5 seconds) { expectTransition(Active, Idle) } } enterBarrier("test3b") From edc9fd733d7d57e95654c4fc1fefdb9d7eb52d74 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 5 Nov 2012 17:11:07 +0100 Subject: [PATCH 18/62] Allow negative Load average in test, see #2664 * According to documentation it can be negative for some OS --- .../src/main/scala/akka/cluster/ClusterMetricsCollector.scala | 2 +- .../src/test/scala/akka/cluster/MetricsCollectorSpec.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala index 7edd0974c0..eb01e76e73 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala @@ -463,7 +463,7 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe /** * (SIGAR / JMX) Returns the OS-specific average system load on the CPUs in the system, for the past 1 minute. - * On some systems the JMX OS system load average may not be available, in which case a -1 is returned. + * On some systems the JMX OS system load average may not be available, in which case a negative value is returned. * Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default. */ def systemLoadAverage: Metric = Metric("system-load-average", Some(BigDecimal(Try( diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala index f00869d92f..e1044000e8 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala @@ -78,11 +78,12 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl val metrics = sample.metrics.collect { case m if m.isDefined ⇒ (m.name, m.value.get) } val used = metrics collectFirst { case ("heap-memory-used", b) ⇒ b } val committed = metrics collectFirst { case ("heap-memory-committed", b) ⇒ b } + metrics foreach { case ("total-cores", b) ⇒ b.intValue must be > (0) case ("network-max-rx", b) ⇒ b.longValue must be > (0L) case ("network-max-tx", b) ⇒ b.longValue must be > (0L) - case ("system-load-average", b) ⇒ b.doubleValue must be >= (0.0) + case ("system-load-average", b) ⇒ // not possible to assert b, allowed to be negative or positive case ("processors", b) ⇒ b.intValue must be >= (0) case ("heap-memory-used", b) ⇒ b.longValue must be >= (0L) case ("heap-memory-committed", b) ⇒ b.longValue must be > (0L) From 4917680f7348d9abf4962f860f25b71cde75af17 Mon Sep 17 00:00:00 2001 From: "Kaspar Fischer (hbf)" Date: Mon, 5 Nov 2012 17:38:40 +0100 Subject: [PATCH 19/62] more tests and bug fixes --- .../throttle/TimerBasedThrottler.scala | 26 +++---- .../throttle/TimerBasedThrottlerSpec.scala | 68 +++++++++++++++++-- 2 files changed, 78 insertions(+), 16 deletions(-) diff --git a/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala index 12e98e89b2..4917be2579 100644 --- a/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala +++ b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala @@ -1,7 +1,7 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + package akka.contrib.throttle import scala.concurrent.duration.{ Duration, FiniteDuration } @@ -121,7 +121,7 @@ object Throttler { * * @see [[akka.contrib.throttle.Throttler.Rate]] */ - implicit class RateInt(numberOfCalls: Int) { + implicit class RateInt(val numberOfCalls: Int) extends AnyVal { def msgsPer(duration: Int, timeUnit: TimeUnit) = Rate(numberOfCalls, Duration(duration, timeUnit)) def msgsPer(duration: FiniteDuration) = Rate(numberOfCalls, duration) def msgsPerSecond = Rate(numberOfCalls, Duration(1, TimeUnit.SECONDS)) @@ -137,11 +137,9 @@ object Throttler { object TimerBasedThrottler { private[throttle] case object Tick - // States of the FSM + // States of the FSM: A `TimerBasedThrottler` is in state `Active` iff the timer is running. private[throttle] sealed trait State - // Idle means we don't deliver messages, either because there are none, or because no target was set. private[throttle] case object Idle extends State - // Active means we the target is set and we have a message queue that is non-empty. private[throttle] case object Active extends State // Messages, as we queue them to be sent later @@ -211,7 +209,6 @@ object TimerBasedThrottler { * @see [[akka.contrib.throttle.Throttler]] */ class TimerBasedThrottler(var rate: Rate) extends Actor with Throttler with FSM[State, Data] { - startWith(Idle, Data(None, rate.numberOfCalls, Q())) // Idle: no messages, or target not set @@ -246,15 +243,20 @@ class TimerBasedThrottler(var rate: Rate) extends Actor with Throttler with FSM[ // Set the target (when the new target is None) case Event(SetTarget(None), d) ⇒ - goto(Idle) using d.copy(target = None) + // Note: We do not yet switch to state `Inactive` because we need the timer to tick once more before + stay using d.copy(target = None) // Set the target (when the new target is not None) case Event(SetTarget(t @ Some(_)), d) ⇒ stay using d.copy(target = t) - // Period ends and we have no more messages + // Tick after a `SetTarget(None)`: take the additional permits and go to `Idle` + case Event(Tick, d @ Data(None, _, _)) ⇒ + goto(Idle) using d.copy(callsLeftInThisPeriod = rate.numberOfCalls) + + // Period ends and we have no more messages: take the additional permits and go to `Idle` case Event(Tick, d @ Data(_, _, Seq())) ⇒ - goto(Idle) + goto(Idle) using d.copy(callsLeftInThisPeriod = rate.numberOfCalls) // Period ends and we get more occasions to send messages case Event(Tick, d @ Data(_, _, _)) ⇒ diff --git a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala index 4f8f422cb6..c78384a2dd 100644 --- a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala +++ b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala @@ -27,11 +27,9 @@ object TimerBasedThrottlerSpec { } @RunWith(classOf[JUnitRunner]) -class TimerBasedThrottlerSpec(_system: ActorSystem) extends TestKit(_system) with ImplicitSender +class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSpec")) with ImplicitSender with WordSpec with MustMatchers with BeforeAndAfterAll { - def this() = this(ActorSystem("TimerBasedThrottlerSpec")) - override def afterAll { system.shutdown() } @@ -71,7 +69,7 @@ class TimerBasedThrottlerSpec(_system: ActorSystem) extends TestKit(_system) wit throttler ! "6" expectNoMsg(1 second) throttler ! SetTarget(Some(echo)) - within(2.seconds) { + within(2 seconds) { expectMsg("1") expectMsg("2") expectMsg("3") @@ -81,6 +79,68 @@ class TimerBasedThrottlerSpec(_system: ActorSystem) extends TestKit(_system) wit } } + "send messages after a `SetTarget(None)` pause" in { + val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + throttler ! SetTarget(Some(echo)) + throttler ! "1" + throttler ! "2" + throttler ! "3" + throttler ! SetTarget(None) + within(1 second) { + expectMsg("1") + expectMsg("2") + expectMsg("3") + expectNoMsg(remaining) + } + expectNoMsg(1 second) + throttler ! SetTarget(Some(echo)) + throttler ! "4" + throttler ! "5" + throttler ! "6" + throttler ! "7" + within(1 seconds) { + expectMsg("4") + expectMsg("5") + expectMsg("6") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("7") + } + } + + "keep messages when the target is set to None" in { + val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + throttler ! SetTarget(Some(echo)) + throttler ! "1" + throttler ! "2" + throttler ! "3" + throttler ! "4" + throttler ! "5" + throttler ! "6" + throttler ! "7" + throttler ! SetTarget(None) + within(1 second) { + expectMsg("1") + expectMsg("2") + expectMsg("3") + expectNoMsg(remaining) + } + expectNoMsg(1 second) + throttler ! SetTarget(Some(echo)) + within(1 seconds) { + expectMsg("4") + expectMsg("5") + expectMsg("6") + expectNoMsg(remaining) + } + within(1 second) { + expectMsg("7") + } + } + "respect the rate (3 msg/s)" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) From 99cf420a376597b7340a37180d0ea89a6b3004fb Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 5 Nov 2012 20:07:57 +0100 Subject: [PATCH 20/62] Use undefined metrics value for negative load average, see #2664 --- .../scala/akka/cluster/ClusterMetricsCollector.scala | 11 ++++++++--- .../scala/akka/cluster/MetricsCollectorSpec.scala | 6 ++---- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala index eb01e76e73..76c01a6381 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala @@ -463,11 +463,16 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe /** * (SIGAR / JMX) Returns the OS-specific average system load on the CPUs in the system, for the past 1 minute. - * On some systems the JMX OS system load average may not be available, in which case a negative value is returned. + * On some systems the JMX OS system load average may not be available, in which case a Metric with + * undefined value is returned. * Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default. */ - def systemLoadAverage: Metric = Metric("system-load-average", Some(BigDecimal(Try( - LoadAverage.get.invoke(sigar.get).asInstanceOf[Array[Double]].toSeq.head) getOrElse osMBean.getSystemLoadAverage))) + def systemLoadAverage: Metric = Metric("system-load-average", + Try(LoadAverage.get.invoke(sigar.get).asInstanceOf[Array[Double]].toSeq.head).getOrElse( + osMBean.getSystemLoadAverage) match { + case x if x < 0 ⇒ None // load average may be unavailable on some platform + case x ⇒ Some(BigDecimal(x)) + }) /** * (JMX) Returns the number of available processors diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala index e1044000e8..6589ba0efc 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala @@ -78,12 +78,11 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl val metrics = sample.metrics.collect { case m if m.isDefined ⇒ (m.name, m.value.get) } val used = metrics collectFirst { case ("heap-memory-used", b) ⇒ b } val committed = metrics collectFirst { case ("heap-memory-committed", b) ⇒ b } - metrics foreach { case ("total-cores", b) ⇒ b.intValue must be > (0) case ("network-max-rx", b) ⇒ b.longValue must be > (0L) case ("network-max-tx", b) ⇒ b.longValue must be > (0L) - case ("system-load-average", b) ⇒ // not possible to assert b, allowed to be negative or positive + case ("system-load-average", b) ⇒ b.doubleValue must be >= (0.0) case ("processors", b) ⇒ b.intValue must be >= (0) case ("heap-memory-used", b) ⇒ b.longValue must be >= (0L) case ("heap-memory-committed", b) ⇒ b.longValue must be > (0L) @@ -110,8 +109,7 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl "collect JMX metrics" in { // heap max may be undefined depending on the OS - // systemLoadAverage is JMX is SIGAR not present - collector.systemLoadAverage.isDefined must be(true) + // systemLoadAverage is JMX if SIGAR not present, but not available on all OS collector.used.isDefined must be(true) collector.committed.isDefined must be(true) collector.processors.isDefined must be(true) From 35493b6c584f4ea4a91cd7de22206a4f6237357b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 3 Nov 2012 20:20:48 +0100 Subject: [PATCH 21/62] Unbreaking poolsize for everybody. --- .../scala/akka/remote/netty/NettyRemoteSupport.scala | 8 ++------ .../src/main/scala/akka/remote/netty/Server.scala | 12 ++++-------- 2 files changed, 6 insertions(+), 14 deletions(-) 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 7c27a82e6d..a65bac1a66 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -41,12 +41,8 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory) val clientChannelFactory = { - settings.UseDispatcherForIO match { - case Some(id) ⇒ - val d = system.dispatchers.lookup(id) - new NioClientSocketChannelFactory(d, d, settings.ClientSocketWorkerPoolSize) - case None ⇒ - new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), settings.ClientSocketWorkerPoolSize) + val boss, worker = settings.UseDispatcherForIO.map(system.dispatchers.lookup) getOrElse Executors.newCachedThreadPool() + new NioClientSocketChannelFactory(boss, worker, settings.ClientSocketWorkerPoolSize) } /** 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 2f76b324cd..83c9b6026b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Server.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Server.scala @@ -25,14 +25,10 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) { val ip = InetAddress.getByName(settings.Hostname) - private val factory = - settings.UseDispatcherForIO match { - case Some(id) ⇒ - val d = netty.system.dispatchers.lookup(id) - new NioServerSocketChannelFactory(d, d, settings.ServerSocketWorkerPoolSize) - case None ⇒ - new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(), settings.ServerSocketWorkerPoolSize) - } + private val factory = { + val boss, worker = settings.UseDispatcherForIO.map(netty.system.dispatchers.lookup) getOrElse Executors.newCachedThreadPool() + new NioServerSocketChannelFactory(boss, worker, settings.ServerSocketWorkerPoolSize) + } // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") From 480f5d5683569d324dad7cf0f883c47fcad86320 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 5 Nov 2012 22:11:57 +0100 Subject: [PATCH 22/62] make FSM.receive overridable, see #2680 --- akka-actor/src/main/scala/akka/actor/FSM.scala | 2 +- akka-docs/rst/scala/fsm.rst | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 273ab765c3..069691ce67 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -521,7 +521,7 @@ trait FSM[S, D] extends Listeners with ActorLogging { * Main actor receive() method * ******************************************* */ - override final def receive: Receive = { + override def receive: Receive = { case TimeoutMarker(gen) ⇒ if (generation == gen) { processMsg(StateTimeout, "state timeout") diff --git a/akka-docs/rst/scala/fsm.rst b/akka-docs/rst/scala/fsm.rst index bcccce0197..4ace396a14 100644 --- a/akka-docs/rst/scala/fsm.rst +++ b/akka-docs/rst/scala/fsm.rst @@ -124,6 +124,14 @@ obvious that an actor is actually created: :include: simple-fsm :exclude: fsm-body +.. note:: + + The FSM trait defines a ``receive`` method which handles internal messages + and passes everything else through to the FSM logic (according to the + current state). When overriding the ``receive`` method, keep in mind that + e.g. state timeout handling depends on actually passing the messages through + the FSM logic. + The :class:`FSM` trait takes two type parameters: #. the supertype of all state names, usually a sealed trait with case objects From 94060d61326f15c6576c16dc7afd430673fe2838 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 6 Nov 2012 12:30:31 +1300 Subject: [PATCH 23/62] Update build and sbt plugin to sbt 0.12.1 --- akka-sbt-plugin/sample/project/Build.scala | 6 +++--- akka-sbt-plugin/sample/project/build.properties | 2 +- akka-sbt-plugin/sample/project/plugins.sbt | 2 -- akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala | 4 ++-- project/AkkaBuild.scala | 2 +- project/build.properties | 2 +- 6 files changed, 8 insertions(+), 10 deletions(-) diff --git a/akka-sbt-plugin/sample/project/Build.scala b/akka-sbt-plugin/sample/project/Build.scala index b7c5122da3..6fbf075359 100644 --- a/akka-sbt-plugin/sample/project/Build.scala +++ b/akka-sbt-plugin/sample/project/Build.scala @@ -7,7 +7,7 @@ import akka.sbt.AkkaKernelPlugin.{ Dist, outputDirectory, distJvmOptions} object HelloKernelBuild extends Build { val Organization = "akka.sample" val Version = "2.2-SNAPSHOT" - val ScalaVersion = "2.10.0-M6" + val ScalaVersion = "2.10.0-RC1" lazy val HelloKernel = Project( id = "hello-kernel", @@ -52,7 +52,7 @@ object Dependency { val Akka = "2.2-SNAPSHOT" } - val akkaKernel = "com.typesafe.akka" % "akka-kernel" % V.Akka - val akkaSlf4j = "com.typesafe.akka" % "akka-slf4j" % V.Akka + val akkaKernel = "com.typesafe.akka" %% "akka-kernel" % V.Akka cross CrossVersion.full + val akkaSlf4j = "com.typesafe.akka" %% "akka-slf4j" % V.Akka cross CrossVersion.full 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 index f4ff7a5afa..4474a03e1a 100644 --- a/akka-sbt-plugin/sample/project/build.properties +++ b/akka-sbt-plugin/sample/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.2 +sbt.version=0.12.1 diff --git a/akka-sbt-plugin/sample/project/plugins.sbt b/akka-sbt-plugin/sample/project/plugins.sbt index b34eb00adb..6200abdd63 100644 --- a/akka-sbt-plugin/sample/project/plugins.sbt +++ b/akka-sbt-plugin/sample/project/plugins.sbt @@ -1,3 +1 @@ -resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" - addSbtPlugin("com.typesafe.akka" % "akka-sbt-plugin" % "2.2-SNAPSHOT") diff --git a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala index 835a596a4a..f000763a70 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala @@ -59,7 +59,7 @@ object AkkaKernelPlugin extends Plugin { (distConfig, sourceDirectory, crossTarget, dependencyClasspath, projectDependencies, allDependencies, buildStructure, state) map { (conf, src, tgt, cp, projDeps, allDeps, buildStruct, st) ⇒ if (isKernelProject(allDeps)) { - val log = logger(st) + val log = st.log val distBinPath = conf.outputDirectory / "bin" val distConfigPath = conf.outputDirectory / "config" val distDeployPath = conf.outputDirectory / "deploy" @@ -201,7 +201,7 @@ object AkkaKernelPlugin extends Plugin { def setting[A](key: SettingKey[A], errorMessage: ⇒ String) = { optionalSetting(key) getOrElse { - logger(state).error(errorMessage); + state.log.error(errorMessage); throw new IllegalArgumentException() } } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 000456ebdf..8c1a4a6335 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -274,7 +274,7 @@ object AkkaBuild extends Build { publishMavenStyle := false, // SBT Plugins should be published as Ivy publishTo <<= Publish.akkaPluginPublishTo, scalacOptions in Compile := Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"), - scalaVersion := "2.9.1", + scalaVersion := "2.9.2", scalaBinaryVersion <<= scalaVersion ) ) diff --git a/project/build.properties b/project/build.properties index a8c2f849be..4474a03e1a 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.12.0 +sbt.version=0.12.1 From 645e713362db2966c73e914b531a2f8210304bad Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Nov 2012 09:34:45 +0100 Subject: [PATCH 24/62] align Act.become with context.become, see #2682 - become uses discardOld=true - becomeStacked uses discardOld=false - update docs and demonstrate this difference --- .../test/scala/akka/actor/ActorDSLSpec.scala | 28 ++++++++++++++++++- .../main/scala/akka/actor/dsl/Creators.scala | 13 +++++++-- akka-docs/rst/scala/actors.rst | 11 ++++++++ 3 files changed, 49 insertions(+), 3 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala index 578f557871..097912601a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala @@ -103,6 +103,32 @@ class ActorDSLSpec extends AkkaSpec { i.receive() must be("hi") } + "support becomeStacked" in { + //#becomeStacked + val a = actor(new Act { + become { // this will replace the initial (empty) behavior + case "info" ⇒ sender ! "A" + case "switch" ⇒ + becomeStacked { // this will stack upon the "A" behavior + case "info" ⇒ sender ! "B" + case "switch" ⇒ unbecome() // return to the "A" behavior + } + case "lobotomize" => unbecome() // OH NOES: Actor.emptyBehavior + } + }) + //#becomeStacked + + implicit def sender = testActor + a ! "info" + expectMsg("A") + a ! "switch" + a ! "info" + expectMsg("B") + a ! "switch" + a ! "info" + expectMsg("A") + } + "support setup/teardown" in { //#simple-start-stop val a = actor(new Act { @@ -188,7 +214,7 @@ class ActorDSLSpec extends AkkaSpec { become { case 1 ⇒ stash() case 2 ⇒ - testActor ! 2; unstashAll(); become { + testActor ! 2; unstashAll(); becomeStacked { case 1 ⇒ testActor ! 1; unbecome() } } diff --git a/akka-actor/src/main/scala/akka/actor/dsl/Creators.scala b/akka-actor/src/main/scala/akka/actor/dsl/Creators.scala index 98ab610502..a9515f3000 100644 --- a/akka-actor/src/main/scala/akka/actor/dsl/Creators.scala +++ b/akka-actor/src/main/scala/akka/actor/dsl/Creators.scala @@ -29,7 +29,9 @@ trait Creators { this: ActorDSL.type ⇒ * for quickly trying things out in the REPL. It makes the following keywords * available: * - * - `become` mapped to `context.become(_, discardOld = false)` + * - `become` mapped to `context.become(_, discardOld = true)` + * + * - `becomeStacked` mapped to `context.become(_, discardOld = false)` * * - `unbecome` mapped to `context.unbecome` * @@ -87,7 +89,14 @@ trait Creators { this: ActorDSL.type ⇒ * stack is cleared upon restart. Use `unbecome()` to pop an element off * this stack. */ - def become(r: Receive) = context.become(r, discardOld = false) + def becomeStacked(r: Receive) = context.become(r, discardOld = false) + + /** + * Replace the behavior at the top of the behavior stack for this actor. The + * stack is cleared upon restart. Use `unbecome()` to pop an element off + * this stack or `becomeStacked()` to push a new element on top of it. + */ + def become(r: Receive) = context.become(r, discardOld = true) /** * Pop the active behavior from the behavior stack of this actor. This stack diff --git a/akka-docs/rst/scala/actors.rst b/akka-docs/rst/scala/actors.rst index e81f744952..5a4be2cc58 100644 --- a/akka-docs/rst/scala/actors.rst +++ b/akka-docs/rst/scala/actors.rst @@ -174,6 +174,17 @@ form of the ``implicit val context: ActorContext``. Outside of an actor, you have to either declare an implicit :class:`ActorSystem`, or you can give the factory explicitly (see further below). +The two possible ways of issuing a ``context.become`` (replacing or adding the +new behavior) are offered separately to enable a clutter-free notation of +nested receives: + +.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#becomeStacked + +Please note that calling ``unbecome`` more often than ``becomeStacked`` results +in the original behavior being installed, which in case of the :class:`Act` +trait is the empty behavior (the outer ``become`` just replaces it during +construction). + Life-cycle hooks are also exposed as DSL elements (see `Start Hook`_ and `Stop Hook`_ below), where later invocations of the methods shown below will replace the contents of the respective hooks: From f9eb59e8833d68c008f9273679a279b75558c551 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Nov 2012 11:00:27 +0100 Subject: [PATCH 25/62] better document become semantics, see #2683 --- .../src/main/scala/akka/actor/ActorCell.scala | 22 +++++++--- .../src/main/scala/akka/actor/Stash.scala | 4 +- .../code/docs/actor/UntypedActorSwapper.java | 4 +- akka-docs/rst/java/untyped-actors.rst | 29 ++++++------- akka-docs/rst/scala/actors.rst | 43 +++++++------------ .../scala/code/docs/actor/ActorDocSpec.scala | 4 +- akka-docs/rst/scala/typed-actors.rst | 5 ++- 7 files changed, 54 insertions(+), 57 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 8108626ab4..14d280eb98 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -76,8 +76,14 @@ trait ActorContext extends ActorRefFactory { /** * Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler. - * Puts the behavior on top of the hotswap stack. - * If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack + * This method acts upon the behavior stack as follows: + * + * - if `discardOld = true` it will replace the top element (i.e. the current behavior) + * - if `discardOld = false` it will keep the current behavior and push the given one atop + * + * The default of replacing the current behavior has been chosen to avoid memory leaks in + * case client code is written without consulting this documentation first (i.e. always pushing + * new closures and never issuing an `unbecome()`) */ def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit @@ -167,14 +173,20 @@ trait UntypedActorContext extends ActorContext { /** * Changes the Actor's behavior to become the new 'Procedure' handler. - * Puts the behavior on top of the hotswap stack. + * Replaces the current behavior at the top of the hotswap stack. */ def become(behavior: Procedure[Any]): Unit /** * Changes the Actor's behavior to become the new 'Procedure' handler. - * Puts the behavior on top of the hotswap stack. - * If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack + * This method acts upon the behavior stack as follows: + * + * - if `discardOld = true` it will replace the top element (i.e. the current behavior) + * - if `discardOld = false` it will keep the current behavior and push the given one atop + * + * The default of replacing the current behavior has been chosen to avoid memory leaks in + * case client code is written without consulting this documentation first (i.e. always pushing + * new closures and never issuing an `unbecome()`) */ def become(behavior: Procedure[Any], discardOld: Boolean): Unit diff --git a/akka-actor/src/main/scala/akka/actor/Stash.scala b/akka-actor/src/main/scala/akka/actor/Stash.scala index 05b618d03a..cdf4ef6d5b 100644 --- a/akka-actor/src/main/scala/akka/actor/Stash.scala +++ b/akka-actor/src/main/scala/akka/actor/Stash.scala @@ -16,13 +16,13 @@ import akka.AkkaException * def receive = { * case "open" ⇒ * unstashAll() - * context.become { + * context.become({ * case "write" ⇒ // do writing... * case "close" ⇒ * unstashAll() * context.unbecome() * case msg ⇒ stash() - * } + * }, discardOld = false) * case "done" ⇒ // done * case msg ⇒ stash() * } diff --git a/akka-docs/rst/java/code/docs/actor/UntypedActorSwapper.java b/akka-docs/rst/java/code/docs/actor/UntypedActorSwapper.java index c882ac015a..5098278a38 100644 --- a/akka-docs/rst/java/code/docs/actor/UntypedActorSwapper.java +++ b/akka-docs/rst/java/code/docs/actor/UntypedActorSwapper.java @@ -32,9 +32,9 @@ public class UntypedActorSwapper { @Override public void apply(Object message) { log.info("Ho"); - getContext().unbecome(); // resets the latest 'become' (just for fun) + getContext().unbecome(); // resets the latest 'become' } - }); + }, false); // this signals stacking of the new behavior } else { unhandled(message); } diff --git a/akka-docs/rst/java/untyped-actors.rst b/akka-docs/rst/java/untyped-actors.rst index 2ee8bc397f..c708188d07 100644 --- a/akka-docs/rst/java/untyped-actors.rst +++ b/akka-docs/rst/java/untyped-actors.rst @@ -549,7 +549,8 @@ Upgrade Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at runtime. Use the ``getContext().become`` method from within the Actor. -The hotswapped code is kept in a Stack which can be pushed and popped. +The hotswapped code is kept in a Stack which can be pushed (replacing or adding +at the top) and popped. .. warning:: @@ -563,26 +564,20 @@ To hotswap the Actor using ``getContext().become``: .. includecode:: code/docs/actor/UntypedActorDocTestBase.java :include: hot-swap-actor -The ``become`` method is useful for many different things, such as to implement -a Finite State Machine (FSM). +This variant of the :meth:`become` method is useful for many different things, +such as to implement a Finite State Machine (FSM). It will replace the current +behavior (i.e. the top of the behavior stack), which means that +:meth:`unbecome` is not called, instead always the next behavior is explicitly +installed. -Here is another little cute example of ``become`` and ``unbecome`` in action: +The other way of using :meth:`become` does not replace but add to the top of +the behavior stack. In this case care must be taken to ensure that the number +of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones +in the long run, otherwise this amounts to a memory leak (which is why this +behavior is not the default). .. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper -Downgrade ---------- - -Since the hotswapped code is pushed to a Stack you can downgrade the code as -well. Use the ``getContext().unbecome`` method from within the Actor. - -.. code-block:: java - - public void onReceive(Object message) { - if (message.equals("revert")) getContext().unbecome(); - } - - Stash ===== diff --git a/akka-docs/rst/scala/actors.rst b/akka-docs/rst/scala/actors.rst index e81f744952..6e90d589ae 100644 --- a/akka-docs/rst/scala/actors.rst +++ b/akka-docs/rst/scala/actors.rst @@ -653,11 +653,10 @@ Upgrade ------- Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at -runtime: Invoke the ``context.become`` method from within the Actor. - -Become takes a ``PartialFunction[Any, Unit]`` that implements -the new message handler. The hotswapped code is kept in a Stack which can be -pushed and popped. +runtime: invoke the ``context.become`` method from within the Actor. +:meth:`become` takes a ``PartialFunction[Any, Unit]`` that implements the new +message handler. The hotswapped code is kept in a Stack which can be pushed and +popped. .. warning:: @@ -667,38 +666,26 @@ To hotswap the Actor behavior using ``become``: .. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor -The ``become`` method is useful for many different things, but a particular nice -example of it is in example where it is used to implement a Finite State Machine -(FSM): `Dining Hakkers`_. +This variant of the :meth:`become` method is useful for many different things, +such as to implement a Finite State Machine (FSM, for an example see `Dining +Hakkers`_). It will replace the current behavior (i.e. the top of the behavior +stack), which means that :meth:`unbecome` is not called, instead always the +next behavior is explicitly installed. .. _Dining Hakkers: @github@/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala -Here is another little cute example of ``become`` and ``unbecome`` in action: +The other way of using :meth:`become` does not replace but add to the top of +the behavior stack. In this case care must be taken to ensure that the number +of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones +in the long run, otherwise this amounts to a memory leak (which is why this +behavior is not the default). .. includecode:: code/docs/actor/ActorDocSpec.scala#swapper Encoding Scala Actors nested receives without accidentally leaking memory ------------------------------------------------------------------------- -See this `Unnested receive example `_. - - -Downgrade ---------- - -Since the hotswapped code is pushed to a Stack you can downgrade the code as -well, all you need to do is to: Invoke the ``context.unbecome`` method from within the Actor. - -This will pop the Stack and replace the Actor's implementation with the -``PartialFunction[Any, Unit]`` that is at the top of the Stack. - -Here's how you use the ``unbecome`` method: - -.. code-block:: scala - - def receive = { - case "revert" => context.unbecome() - } +See this `Unnested receive example <@github@/akka-docs/scala/code/docs/actor/UnnestedReceives.scala>`_. Stash diff --git a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala index 611b7a43f5..6642df3b28 100644 --- a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala @@ -96,11 +96,11 @@ class Swapper extends Actor { def receive = { case Swap ⇒ log.info("Hi") - become { + become({ case Swap ⇒ log.info("Ho") unbecome() // resets the latest 'become' (just for fun) - } + }, discardOld = false) // push on top instead of replace } } diff --git a/akka-docs/rst/scala/typed-actors.rst b/akka-docs/rst/scala/typed-actors.rst index ce9c608e4e..0a0597cf0d 100644 --- a/akka-docs/rst/scala/typed-actors.rst +++ b/akka-docs/rst/scala/typed-actors.rst @@ -7,7 +7,10 @@ Essentially turning method invocations into asynchronous dispatch instead of syn Typed Actors consist of 2 "parts", a public interface and an implementation, and if you've done any work in "enterprise" Java, this will be very familiar to you. As with normal Actors you have an external API (the public interface instance) that will delegate methodcalls asynchronously to a private instance of the implementation. -The advantage of Typed Actors vs. Actors is that with TypedActors you have a static contract, and don't need to define your own messages, the downside is that it places some limitations on what you can do and what you can't, i.e. you can't use become/unbecome. +The advantage of Typed Actors vs. Actors is that with TypedActors you have a +static contract, and don't need to define your own messages, the downside is +that it places some limitations on what you can do and what you can't, i.e. you +cannot use :meth:`become`/:meth:`unbecome`. Typed Actors are implemented using `JDK Proxies `_ which provide a pretty easy-worked API to intercept method calls. From 657a24dbc80e8830072bcd031f7bd56c74985f6d Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Nov 2012 11:41:50 +0100 Subject: [PATCH 26/62] change Java stash sample to become(), see #2642 --- .../docs/actor/UntypedActorDocTestBase.java | 31 +++++++++---------- akka-docs/rst/java/untyped-actors.rst | 8 +++-- akka-docs/rst/scala/actors.rst | 8 +++-- 3 files changed, 25 insertions(+), 22 deletions(-) diff --git a/akka-docs/rst/java/code/docs/actor/UntypedActorDocTestBase.java b/akka-docs/rst/java/code/docs/actor/UntypedActorDocTestBase.java index d825858239..fded2ddb3b 100644 --- a/akka-docs/rst/java/code/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/rst/java/code/docs/actor/UntypedActorDocTestBase.java @@ -351,24 +351,23 @@ public class UntypedActorDocTestBase { static //#stash public class ActorWithProtocol extends UntypedActorWithStash { - private Boolean isOpen = false; public void onReceive(Object msg) { - if (isOpen) { - if (msg.equals("write")) { - // do writing... - } else if (msg.equals("close")) { - unstashAll(); - isOpen = false; - } else { - stash(); - } + if (msg.equals("open")) { + unstashAll(); + getContext().become(new Procedure() { + public void apply(Object msg) throws Exception { + if (msg.equals("write")) { + // do writing... + } else if (msg.equals("close")) { + unstashAll(); + getContext().unbecome(); + } else { + stash(); + } + } + }, false); // add behavior on top instead of replacing } else { - if (msg.equals("open")) { - unstashAll(); - isOpen = true; - } else { - stash(); - } + stash(); } } } diff --git a/akka-docs/rst/java/untyped-actors.rst b/akka-docs/rst/java/untyped-actors.rst index 2ee8bc397f..6c3e08f377 100644 --- a/akka-docs/rst/java/untyped-actors.rst +++ b/akka-docs/rst/java/untyped-actors.rst @@ -627,9 +627,11 @@ The stash is backed by a ``scala.collection.immutable.Vector``. As a result, even a very large number of messages may be stashed without a major impact on performance. -Note that the stash is not persisted across restarts of an actor, -unlike the actor's mailbox. Therefore, it should be managed like other -parts of the actor's state which have the same property. +Note that the stash is part of the ephemeral actor state, unlike the +mailbox. Therefore, it should be managed like other parts of the +actor's state which have the same property. The :class:`Stash` trait’s +implementation of :meth:`preRestart` will call ``unstashAll()``, which is +usually the desired behavior. Killing an Actor diff --git a/akka-docs/rst/scala/actors.rst b/akka-docs/rst/scala/actors.rst index e81f744952..313d8643b2 100644 --- a/akka-docs/rst/scala/actors.rst +++ b/akka-docs/rst/scala/actors.rst @@ -752,9 +752,11 @@ major impact on performance. callback. This means it's not possible to write ``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``. -Note that the stash is not persisted across restarts of an actor, -unlike the actor's mailbox. Therefore, it should be managed like other -parts of the actor's state which have the same property. +Note that the stash is part of the ephemeral actor state, unlike the +mailbox. Therefore, it should be managed like other parts of the +actor's state which have the same property. The :class:`Stash` trait’s +implementation of :meth:`preRestart` will call ``unstashAll()``, which is +usually the desired behavior. Killing an Actor From fb237b859e886aa651294303d9665065222a8bc2 Mon Sep 17 00:00:00 2001 From: "Kaspar Fischer (hbf)" Date: Tue, 6 Nov 2012 11:55:09 +0100 Subject: [PATCH 27/62] removed `remaining` argument in `expectNoMsg` calls in tests --- .../throttle/TimerBasedThrottlerSpec.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala index c78384a2dd..cfb51554ad 100644 --- a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala +++ b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala @@ -91,7 +91,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("1") expectMsg("2") expectMsg("3") - expectNoMsg(remaining) + expectNoMsg() } expectNoMsg(1 second) throttler ! SetTarget(Some(echo)) @@ -103,7 +103,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("4") expectMsg("5") expectMsg("6") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("7") @@ -126,7 +126,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("1") expectMsg("2") expectMsg("3") - expectNoMsg(remaining) + expectNoMsg() } expectNoMsg(1 second) throttler ! SetTarget(Some(echo)) @@ -134,7 +134,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("4") expectMsg("5") expectMsg("6") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("7") @@ -156,13 +156,13 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("1") expectMsg("2") expectMsg("3") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("4") expectMsg("5") expectMsg("6") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("7") @@ -187,14 +187,14 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp expectMsg("2") expectMsg("3") expectMsg("4") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("5") expectMsg("6") expectMsg("7") expectMsg("8") - expectNoMsg(remaining) + expectNoMsg() } within(1 second) { expectMsg("9") From ebd47bcdf6deecd1732f66f3e906104b129f2c31 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Nov 2012 11:59:11 +0100 Subject: [PATCH 28/62] correct small confusion (short: confuso) --- akka-docs/rst/java/untyped-actors.rst | 5 ++--- akka-docs/rst/scala/actors.rst | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/akka-docs/rst/java/untyped-actors.rst b/akka-docs/rst/java/untyped-actors.rst index c708188d07..a7575148c4 100644 --- a/akka-docs/rst/java/untyped-actors.rst +++ b/akka-docs/rst/java/untyped-actors.rst @@ -566,9 +566,8 @@ To hotswap the Actor using ``getContext().become``: This variant of the :meth:`become` method is useful for many different things, such as to implement a Finite State Machine (FSM). It will replace the current -behavior (i.e. the top of the behavior stack), which means that -:meth:`unbecome` is not called, instead always the next behavior is explicitly -installed. +behavior (i.e. the top of the behavior stack), which means that you do not use +:meth:`unbecome`, instead always the next behavior is explicitly installed. The other way of using :meth:`become` does not replace but add to the top of the behavior stack. In this case care must be taken to ensure that the number diff --git a/akka-docs/rst/scala/actors.rst b/akka-docs/rst/scala/actors.rst index 6e90d589ae..21556eeefa 100644 --- a/akka-docs/rst/scala/actors.rst +++ b/akka-docs/rst/scala/actors.rst @@ -669,7 +669,7 @@ To hotswap the Actor behavior using ``become``: This variant of the :meth:`become` method is useful for many different things, such as to implement a Finite State Machine (FSM, for an example see `Dining Hakkers`_). It will replace the current behavior (i.e. the top of the behavior -stack), which means that :meth:`unbecome` is not called, instead always the +stack), which means that you do not use :meth:`unbecome`, instead always the next behavior is explicitly installed. .. _Dining Hakkers: @github@/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala From d98c4db1f981fcfa05be733503b9c17276da5cca Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 6 Nov 2012 14:15:57 +0100 Subject: [PATCH 29/62] fix up two oversights in previous doc change - use discardOld=false also in Scala stash sample code - correct link to UnnestedReceives sample --- akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala | 2 +- akka-docs/rst/scala/actors.rst | 2 +- akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala index 097912601a..2aba0e18d4 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala @@ -113,7 +113,7 @@ class ActorDSLSpec extends AkkaSpec { case "info" ⇒ sender ! "B" case "switch" ⇒ unbecome() // return to the "A" behavior } - case "lobotomize" => unbecome() // OH NOES: Actor.emptyBehavior + case "lobotomize" ⇒ unbecome() // OH NOES: Actor.emptyBehavior } }) //#becomeStacked diff --git a/akka-docs/rst/scala/actors.rst b/akka-docs/rst/scala/actors.rst index 85f5c7ae8e..e52d9ba973 100644 --- a/akka-docs/rst/scala/actors.rst +++ b/akka-docs/rst/scala/actors.rst @@ -696,7 +696,7 @@ behavior is not the default). Encoding Scala Actors nested receives without accidentally leaking memory ------------------------------------------------------------------------- -See this `Unnested receive example <@github@/akka-docs/scala/code/docs/actor/UnnestedReceives.scala>`_. +See this `Unnested receive example <@github@/akka-docs/rst/scala/code/docs/actor/UnnestedReceives.scala>`_. Stash diff --git a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala index 6642df3b28..fc936ff13b 100644 --- a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala @@ -316,13 +316,13 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { def receive = { case "open" ⇒ unstashAll() - context.become { + context.become({ case "write" ⇒ // do writing... case "close" ⇒ unstashAll() context.unbecome() case msg ⇒ stash() - } + }, discardOld = false) // stack on top instead of replacing case msg ⇒ stash() } } From aeb4e471415b2f757d4065cc5baed8a940e78cc4 Mon Sep 17 00:00:00 2001 From: "Kaspar Fischer (hbf)" Date: Tue, 6 Nov 2012 15:35:40 +0100 Subject: [PATCH 30/62] fixes to accomodate time dilation in tests and minor issues from reviews by @rkuhn and @patriknw --- .../contrib/throttle/TimerBasedThrottler.scala | 2 +- .../throttle/TimerBasedThrottlerSpec.scala | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala index 4917be2579..de614619f7 100644 --- a/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala +++ b/akka-contrib/src/main/scala/akka/contrib/throttle/TimerBasedThrottler.scala @@ -54,7 +54,7 @@ import akka.AkkaException * * @see [[akka.contrib.throttle.TimerBasedThrottler]] */ -trait Throttler { self: Actor ⇒ } +trait Throttler { this: Actor ⇒ } /** * Message types understood by [[akka.contrib.throttle.Throttler]]'s. diff --git a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala index cfb51554ad..be077489ee 100644 --- a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala +++ b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala @@ -17,6 +17,7 @@ import org.scalatest.junit.JUnitRunner import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import org.scalatest.BeforeAndAfterAll +import akka.testkit._ object TimerBasedThrottlerSpec { class EchoActor extends Actor { @@ -36,7 +37,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "A throttler" must { - "must pass the ScalaDoc class documentation example prgoram" in { + "must pass the ScalaDoc class documentation example program" in { //#demo-code // A simple actor that prints whatever it receives val printer = system.actorOf(Props(new Actor { @@ -45,7 +46,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp } })) // The throttler for this example, setting the rate - val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated)))) // Set the target throttler ! SetTarget(Some(printer)) // These three messages will be sent to the echoer immediately @@ -60,7 +61,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "keep messages until a target is set" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) - val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated)))) throttler ! "1" throttler ! "2" throttler ! "3" @@ -81,7 +82,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "send messages after a `SetTarget(None)` pause" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) - val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated)))) throttler ! SetTarget(Some(echo)) throttler ! "1" throttler ! "2" @@ -112,7 +113,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "keep messages when the target is set to None" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) - val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated)))) throttler ! SetTarget(Some(echo)) throttler ! "1" throttler ! "2" @@ -143,7 +144,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "respect the rate (3 msg/s)" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) - val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated)))) throttler ! SetTarget(Some(echo)) throttler ! "1" throttler ! "2" @@ -171,7 +172,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp "respect the rate (4 msg/s)" in { val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor]) - val throttler = system.actorOf(Props(new TimerBasedThrottler(4 msgsPer (1 second)))) + val throttler = system.actorOf(Props(new TimerBasedThrottler(4 msgsPer (1.second.dilated)))) throttler ! SetTarget(Some(echo)) throttler ! "1" throttler ! "2" From 1f61c293ec1ceb7c5adb18f939bdaba391e01bb4 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 7 Nov 2012 09:49:56 +0100 Subject: [PATCH 31/62] fix two old Duration references --- akka-docs/rst/common/duration.rst | 2 +- akka-docs/rst/scala/testing.rst | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-docs/rst/common/duration.rst b/akka-docs/rst/common/duration.rst index c159c99a8c..97136d48b3 100644 --- a/akka-docs/rst/common/duration.rst +++ b/akka-docs/rst/common/duration.rst @@ -5,7 +5,7 @@ Duration ######## Durations are used throughout the Akka library, wherefore this concept is -represented by a special data type, :class:`scala.concurrent.util.Duration`. +represented by a special data type, :class:`scala.concurrent.duration.Duration`. Values of this type may represent infinite (:obj:`Duration.Inf`, :obj:`Duration.MinusInf`) or finite durations, or be :obj:`Duration.Undefined`. diff --git a/akka-docs/rst/scala/testing.rst b/akka-docs/rst/scala/testing.rst index 9a80ab0e59..a57305ce30 100644 --- a/akka-docs/rst/scala/testing.rst +++ b/akka-docs/rst/scala/testing.rst @@ -713,8 +713,8 @@ Some `Specs2 `_ users have contributed examples of how to wor actually beneficial also for the third point—is to apply the TestKit together with :class:`org.specs2.specification.Scope`. * The Specification traits provide a :class:`Duration` DSL which uses partly - the same method names as :class:`scala.concurrent.util.Duration`, resulting in ambiguous - implicits if ``akka.util.duration._`` is imported. There are two work-arounds: + the same method names as :class:`scala.concurrent.duration.Duration`, resulting in ambiguous + implicits if ``scala.concurrent.duration._`` is imported. There are two work-arounds: * either use the Specification variant of Duration and supply an implicit conversion to the Akka Duration. This conversion is not supplied with the From eee616c6350df9239d367950e3232547f5637d55 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 7 Nov 2012 16:38:40 +0100 Subject: [PATCH 32/62] #2688 - Switching to s.c.u.Unsafe.instance for akka.util.Unsafe.instance --- .../src/main/scala/akka/util/Unsafe.java | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/akka-actor/src/main/scala/akka/util/Unsafe.java b/akka-actor/src/main/scala/akka/util/Unsafe.java index ace3c1baac..005d1b3441 100644 --- a/akka-actor/src/main/scala/akka/util/Unsafe.java +++ b/akka-actor/src/main/scala/akka/util/Unsafe.java @@ -5,27 +5,9 @@ package akka.util; -import java.lang.reflect.Field; - /** * INTERNAL API */ public final class Unsafe { - public final static sun.misc.Unsafe instance; - static { - try { - sun.misc.Unsafe found = null; - for(Field field : sun.misc.Unsafe.class.getDeclaredFields()) { - if (field.getType() == sun.misc.Unsafe.class) { - field.setAccessible(true); - found = (sun.misc.Unsafe) field.get(null); - break; - } - } - if (found == null) throw new IllegalStateException("Can't find instance of sun.misc.Unsafe"); - else instance = found; - } catch(Throwable t) { - throw new ExceptionInInitializerError(t); - } - } + public final static sun.misc.Unsafe instance = scala.concurrent.util.Unsafe.instance; } From 8f131c680fb089f4e1a8be78cfe4ad72b0a1123a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 30 Oct 2012 15:08:41 +0100 Subject: [PATCH 33/62] Switching to immutable.Seq instead of Seq --- .../akka/actor/RelativeActorPathSpec.scala | 12 +++-- .../scala/akka/actor/TypedActorSpec.scala | 13 +++--- .../workbench/BenchResultRepository.scala | 29 ++++++------ .../workbench/GoogleChartBuilder.scala | 23 +++++----- .../akka/performance/workbench/Report.scala | 18 ++++---- .../src/main/scala/akka/actor/ActorPath.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 2 +- .../main/scala/akka/actor/ActorSystem.scala | 34 +++++++------- .../src/main/scala/akka/actor/Address.scala | 5 ++- .../src/main/scala/akka/actor/Deployer.scala | 2 +- .../main/scala/akka/actor/DynamicAccess.scala | 10 ++--- .../src/main/scala/akka/actor/Extension.scala | 2 +- .../main/scala/akka/actor/FaultHandling.scala | 38 +++++++--------- akka-actor/src/main/scala/akka/actor/IO.scala | 19 ++++---- .../main/scala/akka/actor/TypedActor.scala | 22 +++++----- .../akka/dispatch/AbstractDispatcher.scala | 4 +- .../scala/akka/dispatch/Dispatchers.scala | 2 +- .../src/main/scala/akka/event/EventBus.scala | 5 ++- .../src/main/scala/akka/event/Logging.scala | 9 ++-- .../src/main/scala/akka/japi/JavaAPI.scala | 7 +-- .../scala/akka/routing/ConsistentHash.scala | 6 +-- .../akka/serialization/Serialization.scala | 21 ++++----- .../scala/akka/util/SubclassifiedIndex.scala | 6 ++- .../akka/camel/ConcurrentActivationTest.scala | 11 +++-- .../src/main/scala/akka/cluster/Cluster.scala | 2 +- .../cluster/ClusterMetricsCollector.scala | 2 +- ...ientDowningNodeThatIsUnreachableSpec.scala | 5 ++- .../ClientDowningNodeThatIsUpSpec.scala | 5 ++- ...aderDowningNodeThatIsUnreachableSpec.scala | 9 ++-- .../akka/cluster/LeaderElectionSpec.scala | 3 +- .../akka/cluster/MultiNodeClusterSpec.scala | 35 +++++++-------- .../akka/cluster/SingletonClusterSpec.scala | 3 +- .../scala/akka/cluster/SplitBrainSpec.scala | 7 +-- .../UnreachableNodeRejoinsClusterSpec.scala | 7 +-- .../cluster/AccrualFailureDetectorSpec.scala | 6 +-- .../akka/cluster/MetricsCollectorSpec.scala | 13 +++--- .../docs/actor/FaultHandlingTestBase.java | 4 +- .../jrouting/CustomRouterDocTestBase.java | 3 +- .../scala/code/docs/actor/FSMDocSpec.scala | 11 ++--- .../code/docs/testkit/TestKitUsageSpec.scala | 7 +-- .../code/docs/zeromq/ZeromqDocSpec.scala | 12 ++--- .../src/main/scala/akka/kernel/Main.scala | 7 +-- .../blueprint/NamespaceHandlerTest.scala | 6 +-- .../akka/osgi/ActorSystemActivatorTest.scala | 6 ++- .../scala/akka/osgi/PojoSRTestSupport.scala | 16 +++---- .../NetworkFailureInjector.scala | 18 ++++---- .../akka/remote/testconductor/Player.scala | 20 ++++----- .../akka/remote/testkit/MultiNodeSpec.scala | 24 +++++----- .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../remote/netty/NettyRemoteSupport.scala | 11 ++--- .../provider/InternetSeedGenerator.scala | 5 ++- .../scala/akka/testkit/TestActorRef.scala | 2 +- .../akka/testkit/TestEventListener.scala | 19 ++++---- .../src/main/scala/akka/testkit/TestKit.scala | 44 +++++++++---------- .../src/main/scala/akka/testkit/package.scala | 8 ++-- .../UntypedCoordinatedIncrementTest.java | 4 +- .../transactor/UntypedTransactorTest.java | 4 +- .../transactor/CoordinatedIncrementSpec.scala | 7 +-- .../akka/transactor/FickleFriendsSpec.scala | 15 ++++--- .../akka/transactor/TransactorSpec.scala | 5 ++- .../akka/zeromq/ConcurrentSocketActor.scala | 13 +++--- .../main/scala/akka/zeromq/SocketOption.scala | 32 +++++++------- .../akka/zeromq/ZMQMessageDeserializer.scala | 9 ++-- .../scala/akka/zeromq/ZeroMQExtension.scala | 4 +- .../zeromq/ConcurrentSocketActorSpec.scala | 8 ++-- 65 files changed, 375 insertions(+), 350 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RelativeActorPathSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RelativeActorPathSpec.scala index 179f4aa92c..6870a36125 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RelativeActorPathSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RelativeActorPathSpec.scala @@ -6,24 +6,22 @@ package akka.actor import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import java.net.URLEncoder +import scala.collection.immutable class RelativeActorPathSpec extends WordSpec with MustMatchers { - def elements(path: String): Seq[String] = path match { - case RelativeActorPath(elem) ⇒ elem.toSeq - case _ ⇒ Nil - } + def elements(path: String): immutable.Seq[String] = RelativeActorPath.unapply(path).getOrElse(Nil) "RelativeActorPath" must { "match single name" in { - elements("foo") must be(Seq("foo")) + elements("foo") must be(List("foo")) } "match path separated names" in { - elements("foo/bar/baz") must be(Seq("foo", "bar", "baz")) + elements("foo/bar/baz") must be(List("foo", "bar", "baz")) } "match url encoded name" in { val name = URLEncoder.encode("akka://ClusterSystem@127.0.0.1:2552", "UTF-8") - elements(name) must be(Seq(name)) + elements(name) must be(List(name)) } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 040c8e6211..201b6c6949 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -5,18 +5,19 @@ package akka.actor import language.postfixOps import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } -import akka.util.Timeout +import scala.annotation.tailrec +import scala.collection.immutable import scala.concurrent.{ Await, Future, Promise } import scala.concurrent.duration._ -import java.util.concurrent.atomic.AtomicReference -import annotation.tailrec import akka.testkit.{ EventFilter, filterEvents, AkkaSpec } +import akka.util.Timeout import akka.japi.{ Option ⇒ JOption } import akka.testkit.DefaultTimeout -import akka.dispatch.{ Dispatchers } +import akka.dispatch.Dispatchers import akka.pattern.ask import akka.serialization.JavaSerializer import akka.actor.TypedActor._ +import java.util.concurrent.atomic.AtomicReference import java.lang.IllegalStateException import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } @@ -35,9 +36,9 @@ object TypedActorSpec { } """ - class CyclicIterator[T](val items: Seq[T]) extends Iterator[T] { + class CyclicIterator[T](val items: immutable.Seq[T]) extends Iterator[T] { - private[this] val current: AtomicReference[Seq[T]] = new AtomicReference(items) + private[this] val current = new AtomicReference(items) def hasNext = items != Nil diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala index 1cccd19417..7bc3fec9d1 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/BenchResultRepository.scala @@ -12,17 +12,18 @@ import java.io.PrintWriter import java.text.SimpleDateFormat import java.util.Date import scala.collection.mutable.{ Map ⇒ MutableMap } +import scala.collection.immutable import akka.actor.ActorSystem import akka.event.Logging trait BenchResultRepository { def add(stats: Stats) - def get(name: String): Seq[Stats] + def get(name: String): immutable.Seq[Stats] def get(name: String, load: Int): Option[Stats] - def getWithHistorical(name: String, load: Int): Seq[Stats] + def getWithHistorical(name: String, load: Int): immutable.Seq[Stats] def isBaseline(stats: Stats): Boolean @@ -38,9 +39,9 @@ object BenchResultRepository { } class FileBenchResultRepository extends BenchResultRepository { - private val statsByName = MutableMap[String, Seq[Stats]]() + private val statsByName = MutableMap[String, immutable.Seq[Stats]]() private val baselineStats = MutableMap[Key, Stats]() - private val historicalStats = MutableMap[Key, Seq[Stats]]() + private val historicalStats = MutableMap[Key, immutable.Seq[Stats]]() private def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir") private val serDir = resultDir + "/ser" private def serDirExists: Boolean = new File(serDir).exists @@ -51,13 +52,13 @@ class FileBenchResultRepository extends BenchResultRepository { case class Key(name: String, load: Int) def add(stats: Stats): Unit = synchronized { - val values = statsByName.getOrElseUpdate(stats.name, IndexedSeq.empty) + val values = statsByName.getOrElseUpdate(stats.name, Vector.empty) statsByName(stats.name) = values :+ stats save(stats) } - def get(name: String): Seq[Stats] = synchronized { - statsByName.getOrElse(name, IndexedSeq.empty) + def get(name: String): immutable.Seq[Stats] = synchronized { + statsByName.getOrElse(name, Vector.empty) } def get(name: String, load: Int): Option[Stats] = synchronized { @@ -68,13 +69,13 @@ class FileBenchResultRepository extends BenchResultRepository { baselineStats.get(Key(stats.name, stats.load)) == Some(stats) } - def getWithHistorical(name: String, load: Int): Seq[Stats] = synchronized { + def getWithHistorical(name: String, load: Int): immutable.Seq[Stats] = synchronized { val key = Key(name, load) - val historical = historicalStats.getOrElse(key, IndexedSeq.empty) + val historical = historicalStats.getOrElse(key, Vector.empty) val baseline = baselineStats.get(key) val current = get(name, load) - val limited = (IndexedSeq.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical) + val limited = (Vector.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical) limited.sortBy(_.timestamp) } @@ -94,7 +95,7 @@ class FileBenchResultRepository extends BenchResultRepository { } val historical = load(historicalFiles) for (h ← historical) { - val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), IndexedSeq.empty) + val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), Vector.empty) historicalStats(Key(h.name, h.load)) = values :+ h } } @@ -120,7 +121,7 @@ class FileBenchResultRepository extends BenchResultRepository { } } - private def load(files: Iterable[File]): Seq[Stats] = { + private def load(files: Iterable[File]): immutable.Seq[Stats] = { val result = for (f ← files) yield { var in: ObjectInputStream = null @@ -132,11 +133,11 @@ class FileBenchResultRepository extends BenchResultRepository { case e: Throwable ⇒ None } finally { - if (in ne null) try { in.close() } catch { case ignore: Exception ⇒ } + if (in ne null) try in.close() catch { case ignore: Exception ⇒ } } } - result.flatten.toSeq.sortBy(_.timestamp) + result.flatten.toVector.sortBy(_.timestamp) } loadFiles() diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/GoogleChartBuilder.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/GoogleChartBuilder.scala index 52b30ceee7..66b634d47f 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/GoogleChartBuilder.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/GoogleChartBuilder.scala @@ -3,7 +3,7 @@ package akka.performance.workbench import java.io.UnsupportedEncodingException import java.net.URLEncoder -import scala.collection.immutable.TreeMap +import scala.collection.immutable /** * Generates URLs to Google Chart API http://code.google.com/apis/chart/ @@ -16,7 +16,7 @@ object GoogleChartBuilder { /** * Builds a bar chart for tps in the statistics. */ - def tpsChartUrl(statsByTimestamp: TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = { + def tpsChartUrl(statsByTimestamp: immutable.TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = { if (statsByTimestamp.isEmpty) "" else { val loads = statsByTimestamp.values.head.map(_.load) @@ -46,7 +46,7 @@ object GoogleChartBuilder { //sb.append("&") // legend - val legendStats = statsByTimestamp.values.map(_.head).toSeq + val legendStats = statsByTimestamp.values.toVector.map(_.head) appendLegend(legendStats, sb, legend) sb.append("&") // bar spacing @@ -60,10 +60,7 @@ object GoogleChartBuilder { val loadStr = loads.mkString(",") sb.append("chd=t:") val maxValue = allStats.map(_.tps).max - val tpsSeries: Iterable[String] = - for (statsSeq ← statsByTimestamp.values) yield { - statsSeq.map(_.tps).mkString(",") - } + val tpsSeries: Iterable[String] = for (statsSeq ← statsByTimestamp.values) yield statsSeq.map(_.tps).mkString(",") sb.append(tpsSeries.mkString("|")) // y range @@ -83,7 +80,7 @@ object GoogleChartBuilder { /** * Builds a bar chart for all percentiles and the mean in the statistics. */ - def percentilesAndMeanChartUrl(statistics: Seq[Stats], title: String, legend: Stats ⇒ String): String = { + def percentilesAndMeanChartUrl(statistics: immutable.Seq[Stats], title: String, legend: Stats ⇒ String): String = { if (statistics.isEmpty) "" else { val current = statistics.last @@ -146,13 +143,13 @@ object GoogleChartBuilder { } } - private def percentileLabels(percentiles: TreeMap[Int, Long], sb: StringBuilder) { + private def percentileLabels(percentiles: immutable.TreeMap[Int, Long], sb: StringBuilder) { sb.append("chxl=1:|") val s = percentiles.keys.toList.map(_ + "%").mkString("|") sb.append(s) } - private def appendLegend(statistics: Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) { + private def appendLegend(statistics: immutable.Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) { val legends = statistics.map(legend(_)) sb.append("chdl=") val s = legends.map(urlEncode(_)).mkString("|") @@ -166,7 +163,7 @@ object GoogleChartBuilder { sb.append(s) } - private def dataSeries(allPercentiles: Seq[TreeMap[Int, Long]], meanValues: Seq[Double], sb: StringBuilder) { + private def dataSeries(allPercentiles: immutable.Seq[immutable.TreeMap[Int, Long]], meanValues: immutable.Seq[Double], sb: StringBuilder) { val percentileSeries = for { percentiles ← allPercentiles @@ -181,7 +178,7 @@ object GoogleChartBuilder { sb.append(series.mkString("|")) } - private def dataSeries(values: Seq[Double], sb: StringBuilder) { + private def dataSeries(values: immutable.Seq[Double], sb: StringBuilder) { val series = values.map(formatDouble(_)) sb.append(series.mkString("|")) } @@ -198,7 +195,7 @@ object GoogleChartBuilder { } } - def latencyAndThroughputChartUrl(statistics: Seq[Stats], title: String): String = { + def latencyAndThroughputChartUrl(statistics: immutable.Seq[Stats], title: String): String = { if (statistics.isEmpty) "" else { val sb = new StringBuilder diff --git a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala index 18f87702f3..f7974e6784 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/workbench/Report.scala @@ -5,7 +5,7 @@ import java.text.SimpleDateFormat import java.util.Date import akka.actor.ActorSystem import akka.event.Logging -import scala.collection.immutable.TreeMap +import scala.collection.immutable class Report( system: ActorSystem, @@ -19,7 +19,7 @@ class Report( val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss") - def html(statistics: Seq[Stats]) { + def html(statistics: immutable.Seq[Stats]) { val current = statistics.last val sb = new StringBuilder @@ -80,13 +80,13 @@ class Report( chartUrl } - def comparePercentilesAndMeanChart(stats: Stats): Seq[String] = { + def comparePercentilesAndMeanChart(stats: Stats): immutable.Seq[String] = { for { - compareName ← compareResultWith.toSeq + compareName ← compareResultWith.to[immutable.Seq] compareStats ← resultRepository.get(compareName, stats.load) } yield { val chartTitle = stats.name + " vs. " + compareName + ", " + stats.load + " clients" + ", Percentiles and Mean (microseconds)" - val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(Seq(compareStats, stats), chartTitle, _.name) + val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(List(compareStats, stats), chartTitle, _.name) chartUrl } } @@ -102,17 +102,17 @@ class Report( } } - def compareWithHistoricalTpsChart(statistics: Seq[Stats]): Option[String] = { + def compareWithHistoricalTpsChart(statistics: immutable.Seq[Stats]): Option[String] = { if (statistics.isEmpty) { None } else { val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp) - val statsByTimestamp = TreeMap[Long, Seq[Stats]]() ++ + val statsByTimestamp = immutable.TreeMap[Long, Seq[Stats]]() ++ (for (ts ← histTimestamps) yield { val seq = for (stats ← statistics) yield { - val withHistorical: Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load) + val withHistorical: immutable.Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load) val cell = withHistorical.find(_.timestamp == ts) cell.getOrElse(Stats(stats.name, stats.load, ts)) } @@ -131,7 +131,7 @@ class Report( chartUrl } - def formatResultsTable(statsSeq: Seq[Stats]): String = { + def formatResultsTable(statsSeq: immutable.Seq[Stats]): String = { val name = statsSeq.head.name diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index cc21e0de16..a20e8220b1 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -121,7 +121,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act else addr + name override def compareTo(other: ActorPath): Int = other match { - case r: RootActorPath ⇒ toString compareTo r.toString + case r: RootActorPath ⇒ toString compareTo r.toString // FIXME make this cheaper by comparing address and name in isolation case c: ChildActorPath ⇒ 1 } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 8fa84b2978..a11da0c150 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -480,7 +480,7 @@ class LocalActorRefProvider( def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras private def guardianSupervisorStrategyConfigurator = - dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Seq()).get + dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Nil).get /** * Overridable supervision strategy to be used by the “/user” guardian. diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 212ee9372d..b4309bcb5c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -9,17 +9,17 @@ import akka.dispatch._ import akka.pattern.ask import com.typesafe.config.{ Config, ConfigFactory } import scala.annotation.tailrec -import scala.concurrent.duration.Duration -import java.io.Closeable +import scala.collection.immutable +import scala.concurrent.duration.{ FiniteDuration, Duration } import scala.concurrent.{ Await, Awaitable, CanAwait, Future } +import scala.util.{ Failure, Success } import scala.util.control.NonFatal import akka.util._ +import java.io.Closeable import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap } import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException } import java.util.concurrent.TimeUnit.MILLISECONDS import akka.actor.dungeon.ChildrenContainer -import scala.concurrent.duration.FiniteDuration -import util.{ Failure, Success } object ActorSystem { @@ -144,7 +144,7 @@ object ActorSystem { final val LogLevel: String = getString("akka.loglevel") final val StdoutLogLevel: String = getString("akka.stdout-loglevel") - final val EventHandlers: Seq[String] = getStringList("akka.event-handlers").asScala + final val EventHandlers: immutable.Seq[String] = getStringList("akka.event-handlers").asScala.to[Vector] final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS)) final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start") @@ -273,10 +273,8 @@ abstract class ActorSystem extends ActorRefFactory { /** * ''Java API'': Recursively create a descendant’s path by appending all child names. */ - def descendant(names: java.lang.Iterable[String]): ActorPath = { - import scala.collection.JavaConverters._ - /(names.asScala) - } + def descendant(names: java.lang.Iterable[String]): ActorPath = + /(scala.collection.JavaConverters.iterableAsScalaIterableConverter(names).asScala) /** * Start-up time in milliseconds since the epoch. @@ -536,7 +534,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, val scheduler: Scheduler = createScheduler() val provider: ActorRefProvider = { - val arguments = Seq( + val arguments = Vector( classOf[String] -> name, classOf[Settings] -> settings, classOf[EventStream] -> eventStream, @@ -676,15 +674,15 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null private def loadExtensions() { - import scala.collection.JavaConverters.collectionAsScalaIterableConverter - settings.config.getStringList("akka.extensions").asScala foreach { fqcn ⇒ - dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()) } match { - case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()) - case Success(p: ExtensionId[_]) ⇒ registerExtension(p) - case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) - case Failure(problem) ⇒ log.error(problem, "While trying to load extension [{}], skipping...", fqcn) + scala.collection.JavaConverters.collectionAsScalaIterableConverter( + settings.config.getStringList("akka.extensions")).asScala foreach { fqcn ⇒ + dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match { + case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()) + case Success(p: ExtensionId[_]) ⇒ registerExtension(p) + case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) + case Failure(problem) ⇒ log.error(problem, "While trying to load extension [{}], skipping...", fqcn) + } } - } } override def toString: String = lookupRoot.path.root.address.toString diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index 7c87d696a4..b8f8a52e45 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -5,7 +5,8 @@ package akka.actor import java.net.URI import java.net.URISyntaxException import java.net.MalformedURLException -import annotation.tailrec +import scala.annotation.tailrec +import scala.collection.immutable /** * The address specifies the physical location under which an Actor can be @@ -71,7 +72,7 @@ private[akka] trait PathUtils { } object RelativeActorPath extends PathUtils { - def unapply(addr: String): Option[Iterable[String]] = { + def unapply(addr: String): Option[immutable.Seq[String]] = { try { val uri = new URI(addr) if (uri.isAbsolute) None diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 06d3b01a1b..e74b54c320 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -160,7 +160,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce val vnodes = deployment.getInt("virtual-nodes-factor") ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes) case fqn ⇒ - val args = Seq(classOf[Config] -> deployment) + val args = List(classOf[Config] -> deployment) dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({ case exception ⇒ throw new IllegalArgumentException( ("Cannot instantiate router [%s], defined in [%s], " + diff --git a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala index 7a73eb3b15..af891bc483 100644 --- a/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala +++ b/akka-actor/src/main/scala/akka/actor/DynamicAccess.scala @@ -3,7 +3,7 @@ */ package akka.actor -import scala.util.control.NonFatal +import scala.collection.immutable import java.lang.reflect.InvocationTargetException import scala.reflect.ClassTag import scala.util.Try @@ -25,7 +25,7 @@ abstract class DynamicAccess { * val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name)) * }}} */ - def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T] + def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] /** * Obtain a `Class[_]` object loaded with the right class loader (i.e. the one @@ -40,7 +40,7 @@ abstract class DynamicAccess { * `args` argument. The exact usage of args depends on which type is requested, * see the relevant requesting code for details. */ - def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T] + def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] /** * Obtain the Scala “object” instance for the given fully-qualified class name, if there is one. @@ -70,7 +70,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c) }) - override def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T] = + override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] = Try { val types = args.map(_._1).toArray val values = args.map(_._2).toArray @@ -81,7 +81,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t) } recover { case i: InvocationTargetException if i.getTargetException ne null ⇒ throw i.getTargetException } - override def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T] = + override def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] = getClassFor(fqcn) flatMap { c ⇒ createInstanceFor(c, args) } override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = { diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index 6fab4ceb07..707c07982a 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -98,5 +98,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext def this(clazz: Class[T]) = this()(ClassTag(clazz)) override def lookup(): ExtensionId[T] = this - def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, Seq(classOf[ExtendedActorSystem] -> system)).get + def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] -> system)).get } diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index d72389ae5e..444618df00 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -5,9 +5,10 @@ package akka.actor import language.implicitConversions +import java.lang.{ Iterable ⇒ JIterable } import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer -import java.lang.{ Iterable ⇒ JIterable } +import scala.collection.immutable import scala.concurrent.duration.Duration /** * INTERNAL API @@ -170,7 +171,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Implicit conversion from `Seq` of Throwables to a `Decider`. * This maps the given Throwables to restarts, otherwise escalates. */ - implicit def seqThrowable2Decider(trapExit: Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit) + implicit def seqThrowable2Decider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeImmutableDecider(trapExit) type Decider = PartialFunction[Throwable, Directive] type JDecider = akka.japi.Function[Throwable, Directive] @@ -180,23 +181,21 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: Array[Class[_]]): Decider = - { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } - + def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeImmutableDecider(trapExit) /** * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: Seq[Class[_ <: Throwable]]): Decider = - { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } + def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = + makeImmutableDecider(scala.collection.JavaConverters.iterableAsScalaIterableConverter(trapExit).asScala) - /** - * Decider builder which just checks whether one of - * the given Throwables matches the cause and restarts, otherwise escalates. - */ - def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = { - import scala.collection.JavaConverters.iterableAsScalaIterableConverter - makeDecider(trapExit.asScala.toSeq) + private[this] def makeImmutableDecider(trapExit: Iterable[Class[_]]): Decider = { + val traps = trapExit match { // This is the sad, awkward, truth + case s: immutable.Seq[_] ⇒ s.asInstanceOf[immutable.Seq[Class[_]]] + case other ⇒ other.to[immutable.Seq] + } + + { case x ⇒ if (traps exists (_ isInstance x)) Restart else Escalate } } /** @@ -222,14 +221,14 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * * INTERNAL API */ - private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] = + private[akka] def sort(in: Iterable[CauseDirective]): immutable.Seq[CauseDirective] = (new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒ buf.indexWhere(_._1 isAssignableFrom ca._1) match { case -1 ⇒ buf append ca case x ⇒ buf insert (x, ca) } buf - } + }.to[immutable.Seq] private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] = if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None @@ -338,10 +337,6 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - - def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) = - this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - /* * this is a performance optimization to avoid re-allocating the pairs upon * every call to requestRestartPermission, assuming that strategies are shared @@ -380,9 +375,6 @@ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) = - this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - /* * this is a performance optimization to avoid re-allocating the pairs upon * every call to requestRestartPermission, assuming that strategies are shared diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 635afe2a58..e1dedb3ba2 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -6,6 +6,7 @@ package akka.actor import language.higherKinds import language.postfixOps +import scala.collection.immutable import scala.concurrent.{ ExecutionContext, Future } import scala.concurrent.duration.Duration import scala.util.control.NonFatal @@ -122,7 +123,7 @@ object IO { * @return a new SocketHandle that can be used to perform actions on the * new connection's SocketChannel. */ - def accept(options: Seq[SocketOption] = Seq.empty)(implicit socketOwner: ActorRef): SocketHandle = { + def accept(options: immutable.Seq[SocketOption] = Nil)(implicit socketOwner: ActorRef): SocketHandle = { val socket = SocketHandle(socketOwner, ioManager) ioManager ! Accept(socket, this, options) socket @@ -250,7 +251,7 @@ object IO { * * Normally sent using IOManager.listen() */ - case class Listen(server: ServerHandle, address: SocketAddress, options: Seq[ServerSocketOption] = Seq.empty) extends IOMessage + case class Listen(server: ServerHandle, address: SocketAddress, options: immutable.Seq[ServerSocketOption] = Nil) extends IOMessage /** * Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is @@ -272,7 +273,7 @@ object IO { * * Normally sent using [[akka.actor.IO.ServerHandle]].accept() */ - case class Accept(socket: SocketHandle, server: ServerHandle, options: Seq[SocketOption] = Seq.empty) extends IOMessage + case class Accept(socket: SocketHandle, server: ServerHandle, options: immutable.Seq[SocketOption] = Nil) extends IOMessage /** * Message to an [[akka.actor.IOManager]] to create a SocketChannel connected @@ -280,7 +281,7 @@ object IO { * * Normally sent using IOManager.connect() */ - case class Connect(socket: SocketHandle, address: SocketAddress, options: Seq[SocketOption] = Seq.empty) extends IOMessage + case class Connect(socket: SocketHandle, address: SocketAddress, options: immutable.Seq[SocketOption] = Nil) extends IOMessage /** * Message from an [[akka.actor.IOManager]] that the SocketChannel has @@ -832,7 +833,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension { * @param option Seq of [[akka.actor.IO.ServerSocketOptions]] to setup on socket * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket */ - def listen(address: SocketAddress, options: Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = { + def listen(address: SocketAddress, options: immutable.Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = { val server = IO.ServerHandle(owner, actor) actor ! IO.Listen(server, address, options) server @@ -847,7 +848,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension { * @param owner the ActorRef that will receive messages from the IOManagerActor * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket */ - def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Seq.empty) + def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Nil) /** * Create a ServerSocketChannel listening on a host and port. Messages will @@ -860,7 +861,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension { * @param owner the ActorRef that will receive messages from the IOManagerActor * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket */ - def listen(host: String, port: Int, options: Seq[IO.ServerSocketOption] = Seq.empty)(implicit owner: ActorRef): IO.ServerHandle = + def listen(host: String, port: Int, options: immutable.Seq[IO.ServerSocketOption] = Nil)(implicit owner: ActorRef): IO.ServerHandle = listen(new InetSocketAddress(host, port), options)(owner) /** @@ -873,7 +874,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension { * @param owner the ActorRef that will receive messages from the IOManagerActor * @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket */ - def connect(address: SocketAddress, options: Seq[IO.SocketOption] = Seq.empty)(implicit owner: ActorRef): IO.SocketHandle = { + def connect(address: SocketAddress, options: immutable.Seq[IO.SocketOption] = Nil)(implicit owner: ActorRef): IO.SocketHandle = { val socket = IO.SocketHandle(owner, actor) actor ! IO.Connect(socket, address, options) socket @@ -991,7 +992,7 @@ final class IOManagerActor(val settings: Settings) extends Actor with ActorLoggi private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) } - private def setSocketOptions(socket: java.net.Socket, options: Seq[IO.SocketOption]) { + private def setSocketOptions(socket: java.net.Socket, options: immutable.Seq[IO.SocketOption]) { options foreach { case IO.KeepAlive(on) ⇒ forwardFailure(socket.setKeepAlive(on)) case IO.OOBInline(on) ⇒ forwardFailure(socket.setOOBInline(on)) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 6dbe48ba40..f7b0e853ef 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -4,22 +4,24 @@ package akka.actor import language.existentials -import akka.japi.{ Creator, Option ⇒ JOption } -import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } -import akka.util.Timeout + import scala.util.control.NonFatal +import scala.util.{ Try, Success, Failure } +import scala.collection.immutable +import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.Duration +import scala.reflect.ClassTag import scala.concurrent.{ Await, Future } +import akka.japi.{ Creator, Option ⇒ JOption } +import akka.util.Timeout import akka.util.Reflect.instantiator +import akka.serialization.{ JavaSerializer, SerializationExtension } import akka.dispatch._ import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar } import java.util.concurrent.TimeoutException import java.util.concurrent.TimeUnit.MILLISECONDS -import scala.reflect.ClassTag -import akka.serialization.{ JavaSerializer, SerializationExtension } import java.io.ObjectStreamException -import scala.util.{ Try, Success, Failure } -import scala.concurrent.duration.FiniteDuration +import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } /** * A TypedActorFactory is something that can created TypedActor instances. @@ -439,8 +441,8 @@ object TypedProps { * @return a sequence of interfaces that the specified class implements, * or a sequence containing only itself, if itself is an interface. */ - def extractInterfaces(clazz: Class[_]): Seq[Class[_]] = - if (clazz.isInterface) Seq[Class[_]](clazz) else clazz.getInterfaces.toList + def extractInterfaces(clazz: Class[_]): immutable.Seq[Class[_]] = + if (clazz.isInterface) List[Class[_]](clazz) else clazz.getInterfaces.to[List] /** * Uses the supplied class as the factory for the TypedActor implementation, @@ -489,7 +491,7 @@ object TypedProps { */ @SerialVersionUID(1L) case class TypedProps[T <: AnyRef] protected[TypedProps] ( - interfaces: Seq[Class[_]], + interfaces: immutable.Seq[Class[_]], creator: () ⇒ T, dispatcher: String = TypedProps.defaultDispatcherId, deploy: Deploy = Props.defaultDeploy, diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 36afc8a24c..1b9de36e77 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -420,7 +420,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit case "unbounded" ⇒ UnboundedMailbox() case "bounded" ⇒ new BoundedMailbox(prerequisites.settings, config) case fqcn ⇒ - val args = Seq(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config) + val args = List(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config) prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({ case exception ⇒ throw new IllegalArgumentException( @@ -436,7 +436,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit case null | "" | "fork-join-executor" ⇒ new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites) case "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites) case fqcn ⇒ - val args = Seq( + val args = List( classOf[Config] -> config, classOf[DispatcherPrerequisites] -> prerequisites) prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({ diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index e1ae0ae50b..910a5ceed5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -147,7 +147,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites) case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites) case fqn ⇒ - val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) + val args = List(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({ case exception ⇒ throw new IllegalArgumentException( diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index cb83fbe806..403f7a0dfd 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -10,6 +10,7 @@ import java.util.concurrent.ConcurrentSkipListSet import java.util.Comparator import akka.util.{ Subclassification, SubclassifiedIndex } import scala.collection.immutable.TreeSet +import scala.collection.immutable /** * Represents the base type for EventBuses @@ -167,12 +168,12 @@ trait SubchannelClassification { this: EventBus ⇒ recv foreach (publish(event, _)) } - private def removeFromCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit = + private def removeFromCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit = cache = (cache /: changes) { case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) -- cs) } - private def addToCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit = + private def addToCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit = cache = (cache /: changes) { case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs) } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index c9286cf2c9..dbd561514d 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -9,12 +9,13 @@ import akka.actor._ import akka.{ ConfigurationException, AkkaException } import akka.actor.ActorSystem.Settings import akka.util.{ Timeout, ReentrantGuard } -import scala.concurrent.duration._ import java.util.concurrent.atomic.AtomicInteger -import scala.util.control.NoStackTrace import java.util.concurrent.TimeoutException +import scala.annotation.implicitNotFound +import scala.collection.immutable +import scala.concurrent.duration._ import scala.concurrent.Await -import annotation.implicitNotFound +import scala.util.control.NoStackTrace /** * This trait brings log level handling to the EventStream: it reads the log @@ -448,7 +449,7 @@ object Logging { } // these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type - val AllLogLevels: Seq[LogLevel] = Seq(ErrorLevel, WarningLevel, InfoLevel, DebugLevel) + val AllLogLevels: immutable.Seq[LogLevel] = Vector(ErrorLevel, WarningLevel, InfoLevel, DebugLevel) /** * Obtain LoggingAdapter for the given actor system and source object. This diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index ee2a688345..fda674a02c 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -5,7 +5,8 @@ package akka.japi import language.implicitConversions -import scala.Some + +import scala.collection.immutable import scala.reflect.ClassTag import scala.util.control.NoStackTrace import scala.runtime.AbstractPartialFunction @@ -176,7 +177,7 @@ object Option { object Util { def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz) - def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq + def arrayToSeq[T](arr: Array[T]): immutable.Seq[T] = arr.to[immutable.Seq] - def arrayToSeq(classes: Array[Class[_]]): Seq[Class[_]] = classes.toSeq + def arrayToSeq(classes: Array[Class[_]]): immutable.Seq[Class[_]] = classes.to[immutable.Seq] } diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index 819fea2586..84100f0f21 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -4,7 +4,7 @@ package akka.routing -import scala.collection.immutable.SortedMap +import scala.collection.immutable import scala.reflect.ClassTag import java.util.Arrays @@ -18,7 +18,7 @@ import java.util.Arrays * hash, i.e. make sure it is different for different nodes. * */ -class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtualNodesFactor: Int) { +class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], val virtualNodesFactor: Int) { import ConsistentHash._ @@ -106,7 +106,7 @@ class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtual object ConsistentHash { def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = { - new ConsistentHash(SortedMap.empty[Int, T] ++ + new ConsistentHash(immutable.SortedMap.empty[Int, T] ++ (for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)), virtualNodesFactor) } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 003c9de2b1..1f78fbd3a7 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -4,14 +4,15 @@ package akka.serialization -import akka.AkkaException import com.typesafe.config.Config +import akka.AkkaException import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess } import akka.event.Logging import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer import java.io.NotSerializableException -import util.{ Try, DynamicVariable } +import scala.util.{ Try, DynamicVariable } +import scala.collection.immutable object Serialization { @@ -97,7 +98,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { serializerMap.get(clazz) match { case null ⇒ // bindings are ordered from most specific to least specific - def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean = + def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean = possibilities.size == 1 || (possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) || (possibilities forall (_._2 == possibilities(0)._2)) @@ -122,8 +123,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * loading is performed by the system’s [[akka.actor.DynamicAccess]]. */ def serializerOf(serializerFQN: String): Try[Serializer] = - system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) recoverWith { - case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()) + system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] -> system)) recoverWith { + case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil) } /** @@ -137,21 +138,21 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * bindings is a Seq of tuple representing the mapping from Class to Serializer. * It is primarily ordered by the most specific classes first, and secondly in the configured order. */ - private[akka] val bindings: Seq[ClassSerializer] = - sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v))) + private[akka] val bindings: immutable.Seq[ClassSerializer] = + sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v))).to[immutable.Seq] /** * Sort so that subtypes always precede their supertypes, but without * obeying any order between unrelated subtypes (insert sort). */ - private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] = - (new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒ + private def sort(in: Iterable[ClassSerializer]): immutable.Seq[ClassSerializer] = + ((new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒ buf.indexWhere(_._1 isAssignableFrom ca._1) match { case -1 ⇒ buf append ca case x ⇒ buf insert (x, ca) } buf - } + }).to[immutable.Seq] /** * serializerMap is a Map whose keys is the class that is serializable and values is the serializer diff --git a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala index 565d50e636..d0ee67c1fb 100644 --- a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala +++ b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala @@ -3,6 +3,8 @@ */ package akka.util +import scala.collection.immutable + /** * Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`! */ @@ -74,7 +76,7 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V] import SubclassifiedIndex._ - type Changes = Seq[(K, Set[V])] + type Changes = immutable.Seq[(K, Set[V])] protected var subkeys = Vector.empty[Nonroot[K, V]] @@ -208,5 +210,5 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V] private def mergeChangesByKey(changes: Changes): Changes = (emptyMergeMap[K, V] /: changes) { case (m, (k, s)) ⇒ m.updated(k, m(k) ++ s) - }.toSeq + }.to[immutable.Seq] } diff --git a/akka-camel/src/test/scala/akka/camel/ConcurrentActivationTest.scala b/akka-camel/src/test/scala/akka/camel/ConcurrentActivationTest.scala index a4ad1564c2..ff5524ad6c 100644 --- a/akka-camel/src/test/scala/akka/camel/ConcurrentActivationTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConcurrentActivationTest.scala @@ -1,13 +1,18 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ package akka.camel +import language.postfixOps + import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers +import scala.concurrent.{ Promise, Await, Future } +import scala.collection.immutable import akka.camel.TestSupport.NonSharedCamelSystem import akka.actor.{ ActorRef, Props, Actor } import akka.routing.BroadcastRouter -import concurrent.{ Promise, Await, Future } import scala.concurrent.duration._ -import language.postfixOps import akka.testkit._ import akka.util.Timeout import org.apache.camel.model.RouteDefinition @@ -58,7 +63,7 @@ class ConcurrentActivationTest extends WordSpec with MustMatchers with NonShared activations.size must be(2 * number * number) // must be the size of the activated activated producers and consumers deactivations.size must be(2 * number * number) - def partitionNames(refs: Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer")) + def partitionNames(refs: immutable.Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer")) def assertContainsSameElements(lists: (Seq[_], Seq[_])) { val (a, b) = lists a.intersect(b).size must be(a.size) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 13f93d0482..80fdc69fed 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -75,7 +75,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension { val failureDetector: FailureDetector = { import settings.{ FailureDetectorImplementationClass ⇒ fqcn } system.dynamicAccess.createInstanceFor[FailureDetector]( - fqcn, Seq(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({ + fqcn, List(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({ case e ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString) }).get } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala index 76c01a6381..7ed3699035 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala @@ -557,7 +557,7 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe */ private[cluster] object MetricsCollector { def apply(address: Address, log: LoggingAdapter, dynamicAccess: DynamicAccess): MetricsCollector = - dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty) match { + dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Nil) match { case Success(identity) ⇒ new MetricsCollector(Some(identity), address) case Failure(e) ⇒ log.debug(e.toString) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala index 3efb891a3b..bf1009b472 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUnreachableSpec.scala @@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ import akka.actor.Address +import scala.collection.immutable case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -51,7 +52,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow cluster.down(thirdAddress) enterBarrier("down-third-node") - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress)) clusterView.members.exists(_.address == thirdAddress) must be(false) } @@ -62,7 +63,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow runOn(second, fourth) { enterBarrier("down-third-node") - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress)) } enterBarrier("await-completion") diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala index 5a7308ec92..2a0af15997 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClientDowningNodeThatIsUpSpec.scala @@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ import akka.actor.Address +import scala.collection.immutable case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -49,7 +50,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT markNodeAsUnavailable(thirdAddress) - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress)) clusterView.members.exists(_.address == thirdAddress) must be(false) } @@ -60,7 +61,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT runOn(second, fourth) { enterBarrier("down-third-node") - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress)) } enterBarrier("await-completion") diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala index 134ed4d0d6..279e32ab66 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderDowningNodeThatIsUnreachableSpec.scala @@ -11,6 +11,7 @@ import akka.remote.testkit.MultiNodeSpec import akka.testkit._ import akka.actor._ import scala.concurrent.duration._ +import scala.collection.immutable case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -59,7 +60,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow // --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE --- - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds) } runOn(fourth) { @@ -69,7 +70,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow runOn(second, third) { enterBarrier("down-fourth-node") - awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds) + awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds) } enterBarrier("await-completion-1") @@ -89,7 +90,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow // --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE --- - awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds) + awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30.seconds) } runOn(second) { @@ -99,7 +100,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow runOn(third) { enterBarrier("down-second-node") - awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds) + awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30 seconds) } enterBarrier("await-completion-2") diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala index fc8c4d2619..dfe1553369 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/LeaderElectionSpec.scala @@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ import scala.concurrent.duration._ +import scala.collection.immutable case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val controller = role("controller") @@ -42,7 +43,7 @@ abstract class LeaderElectionSpec(multiNodeConfig: LeaderElectionMultiNodeConfig import multiNodeConfig._ // sorted in the order used by the cluster - lazy val sortedRoles = Seq(first, second, third, fourth).sorted + lazy val sortedRoles = List(first, second, third, fourth).sorted "A cluster of four nodes" must { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala index 12fc8ebbc6..d696f9b62b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MultiNodeClusterSpec.scala @@ -4,21 +4,21 @@ package akka.cluster import language.implicitConversions + +import org.scalatest.Suite +import org.scalatest.exceptions.TestFailedException + import com.typesafe.config.Config import com.typesafe.config.ConfigFactory -import akka.actor.{ Address, ExtendedActorSystem } import akka.remote.testconductor.RoleName import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec } import akka.testkit._ import akka.testkit.TestEvent._ -import scala.concurrent.duration._ -import org.scalatest.Suite -import org.scalatest.exceptions.TestFailedException -import java.util.concurrent.ConcurrentHashMap -import akka.actor.ActorPath -import akka.actor.RootActorPath +import akka.actor.{ ActorSystem, Address } import akka.event.Logging.ErrorLevel -import akka.actor.ActorSystem +import scala.concurrent.duration._ +import scala.collection.immutable +import java.util.concurrent.ConcurrentHashMap object MultiNodeClusterSpec { @@ -158,7 +158,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS * nodes (roles). First node will be started first * and others will join the first. */ - def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq) + def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.to[immutable.Seq]) /** * Initialize the cluster of the specified member @@ -166,11 +166,9 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS * First node will be started first and others will join * the first. */ - def awaitClusterUp(roles: RoleName*): Unit = { - awaitStartCluster(true, roles.toSeq) - } + def awaitClusterUp(roles: RoleName*): Unit = awaitStartCluster(true, roles.to[immutable.Seq]) - private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = { + private def awaitStartCluster(upConvergence: Boolean = true, roles: immutable.Seq[RoleName]): Unit = { runOn(roles.head) { // make sure that the node-to-join is started before other join startClusterNode() @@ -196,16 +194,15 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) ⇒ members(i).address must be(a) } } - def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) { - assertLeaderIn(nodesInCluster) - } + def assertLeader(nodesInCluster: RoleName*): Unit = + if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq]) /** * Assert that the cluster has elected the correct leader * out of all nodes in the cluster. First * member in the cluster ring is expected leader. */ - def assertLeaderIn(nodesInCluster: Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) { + def assertLeaderIn(nodesInCluster: immutable.Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) { nodesInCluster.length must not be (0) val expectedLeader = roleOfLeader(nodesInCluster) val leader = clusterView.leader @@ -221,7 +218,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS */ def awaitUpConvergence( numberOfMembers: Int, - canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address], + canNotBePartOfMemberRing: immutable.Seq[Address] = Nil, timeout: FiniteDuration = 20.seconds): Unit = { within(timeout) { awaitCond(clusterView.members.size == numberOfMembers) @@ -239,7 +236,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS def awaitSeenSameState(addresses: Address*): Unit = awaitCond((addresses.toSet -- clusterView.seenBy).isEmpty) - def roleOfLeader(nodesInCluster: Seq[RoleName] = roles): RoleName = { + def roleOfLeader(nodesInCluster: immutable.Seq[RoleName] = roles): RoleName = { nodesInCluster.length must not be (0) nodesInCluster.sorted.head } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala index 291a59a44f..33ce67ecb5 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SingletonClusterSpec.scala @@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ import scala.concurrent.duration._ +import scala.collection.immutable case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -65,7 +66,7 @@ abstract class SingletonClusterSpec(multiNodeConfig: SingletonClusterMultiNodeCo markNodeAsUnavailable(secondAddress) - awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds) + awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = List(secondAddress), 30.seconds) clusterView.isSingletonCluster must be(true) awaitCond(clusterView.isLeader) } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala index e1b1a4af96..a7a7c6f4ba 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/SplitBrainSpec.scala @@ -9,9 +9,10 @@ import com.typesafe.config.ConfigFactory import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ -import scala.concurrent.duration._ import akka.actor.Address import akka.remote.testconductor.Direction +import scala.concurrent.duration._ +import scala.collection.immutable case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -53,8 +54,8 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig) muteMarkingAsUnreachable() - val side1 = IndexedSeq(first, second) - val side2 = IndexedSeq(third, fourth, fifth) + val side1 = Vector(first, second) + val side2 = Vector(third, fourth, fifth) "A cluster of 5 members" must { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala index db2f9fc930..395d803865 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/UnreachableNodeRejoinsClusterSpec.scala @@ -6,13 +6,14 @@ package akka.cluster import language.postfixOps import org.scalatest.BeforeAndAfter +import com.typesafe.config.ConfigFactory import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeSpec import akka.testkit._ -import com.typesafe.config.ConfigFactory import akka.actor.Address import akka.remote.testconductor.{ RoleName, Direction } import scala.concurrent.duration._ +import scala.collection.immutable case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { val first = role("first") @@ -45,7 +46,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod muteMarkingAsUnreachable() - def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = { + def allBut(role: RoleName, roles: immutable.Seq[RoleName] = roles): immutable.Seq[RoleName] = { roles.filterNot(_ == role) } @@ -125,7 +126,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod } runOn(allBut(victim): _*) { - awaitUpConvergence(roles.size - 1, Seq(victim)) + awaitUpConvergence(roles.size - 1, List(victim)) } endBarrier diff --git a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala index 45476864db..8a9d6eb6fc 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AccrualFailureDetectorSpec.scala @@ -7,7 +7,7 @@ package akka.cluster import akka.actor.Address import akka.testkit._ import akka.testkit.TestEvent._ -import scala.collection.immutable.TreeMap +import scala.collection.immutable import scala.concurrent.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -27,7 +27,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" val conn = Address("akka", "", "localhost", 2552) val conn2 = Address("akka", "", "localhost", 2553) - def fakeTimeGenerator(timeIntervals: Seq[Long]): () ⇒ Long = { + def fakeTimeGenerator(timeIntervals: immutable.Seq[Long]): () ⇒ Long = { var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c)) def timeGenerator(): Long = { val currentTime = times.head @@ -73,7 +73,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec(""" "return realistic phi values" in { val fd = createFailureDetector() - val test = TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3) + val test = immutable.TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3) for ((timeDiff, expectedPhi) ← test) { fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1)) } diff --git a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala index 6589ba0efc..609975db6a 100644 --- a/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala @@ -5,14 +5,16 @@ package akka.cluster import scala.language.postfixOps + +import scala.collection.immutable import scala.concurrent.duration._ import scala.concurrent.Await +import scala.util.{ Success, Try, Failure } import akka.actor._ import akka.testkit._ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers -import util.{ Success, Try, Failure } object MetricsEnabledSpec { val config = """ @@ -207,11 +209,10 @@ trait MetricSpec extends WordSpec with MustMatchers { if (decay > 0) metrics.collect { case m if m.trendable && (!m.initializable) ⇒ m }.foreach(_.average.isDefined must be(true)) } - def collectNodeMetrics(nodes: Set[NodeMetrics]): Seq[Metric] = { - var r: Seq[Metric] = Seq.empty - nodes.foreach(n ⇒ r ++= n.metrics.filter(_.isDefined)) - r - } + def collectNodeMetrics(nodes: Set[NodeMetrics]): immutable.Seq[Metric] = + nodes.foldLeft(Vector[Metric]()) { + case (r, n) ⇒ r ++ n.metrics.filter(_.isDefined) + } } trait AbstractClusterMetricsSpec extends DefaultTimeout { diff --git a/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java b/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java index c78da61fb1..9ea2a332b1 100644 --- a/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java @@ -30,7 +30,7 @@ import static java.util.concurrent.TimeUnit.SECONDS; import akka.japi.Function; import scala.Option; import scala.collection.JavaConverters; -import scala.collection.Seq; +import scala.collection.immutable.Seq; import org.junit.Test; import org.junit.BeforeClass; @@ -220,7 +220,7 @@ public class FaultHandlingTestBase { //#testkit public Seq seq(A... args) { return JavaConverters.collectionAsScalaIterableConverter( - java.util.Arrays.asList(args)).asScala().toSeq(); + java.util.Arrays.asList(args)).asScala().toList(); } //#testkit } diff --git a/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java index c4e7414ce1..73b8c5c639 100644 --- a/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java @@ -11,6 +11,7 @@ import static docs.jrouting.CustomRouterDocTestBase.Message.RepublicanVote; import static org.junit.Assert.assertEquals; import java.util.Arrays; +import java.util.Collections; import java.util.List; import org.junit.After; @@ -69,7 +70,7 @@ public class CustomRouterDocTestBase { //#supervision final SupervisorStrategy strategy = new OneForOneStrategy(5, Duration.create("1 minute"), - new Class[] { Exception.class }); + Collections.>singletonList(Exception.class)); final ActorRef router = system.actorOf(new Props(MyActor.class) .withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy))); //#supervision diff --git a/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala index bcc908990f..cc88416b0e 100644 --- a/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala @@ -8,6 +8,7 @@ import language.postfixOps import akka.testkit.{ AkkaSpec ⇒ MyFavoriteTestFrameWorkPlusAkkaTestKit } //#test-code import akka.actor.Props +import scala.collection.immutable class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { @@ -24,7 +25,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { case object Flush // sent events - case class Batch(obj: Seq[Any]) + case class Batch(obj: immutable.Seq[Any]) //#simple-events //#simple-state // states @@ -34,7 +35,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { sealed trait Data case object Uninitialized extends Data - case class Todo(target: ActorRef, queue: Seq[Any]) extends Data + case class Todo(target: ActorRef, queue: immutable.Seq[Any]) extends Data //#simple-state //#simple-fsm class Buncher extends Actor with FSM[State, Data] { @@ -193,12 +194,12 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { buncher ! SetTarget(testActor) buncher ! Queue(42) buncher ! Queue(43) - expectMsg(Batch(Seq(42, 43))) + expectMsg(Batch(immutable.Seq(42, 43))) buncher ! Queue(44) buncher ! Flush buncher ! Queue(45) - expectMsg(Batch(Seq(44))) - expectMsg(Batch(Seq(45))) + expectMsg(Batch(immutable.Seq(44))) + expectMsg(Batch(immutable.Seq(45))) } "batch not if uninitialized" in { diff --git a/akka-docs/rst/scala/code/docs/testkit/TestKitUsageSpec.scala b/akka-docs/rst/scala/code/docs/testkit/TestKitUsageSpec.scala index 48e6b6664b..8b153c5944 100644 --- a/akka-docs/rst/scala/code/docs/testkit/TestKitUsageSpec.scala +++ b/akka-docs/rst/scala/code/docs/testkit/TestKitUsageSpec.scala @@ -22,6 +22,7 @@ import akka.testkit.DefaultTimeout import akka.testkit.ImplicitSender import akka.testkit.TestKit import scala.concurrent.duration._ +import scala.collection.immutable /** * a Test to show some TestKit examples @@ -38,8 +39,8 @@ class TestKitUsageSpec val filterRef = system.actorOf(Props(new FilteringActor(testActor))) val randomHead = Random.nextInt(6) val randomTail = Random.nextInt(10) - val headList = Seq().padTo(randomHead, "0") - val tailList = Seq().padTo(randomTail, "1") + val headList = immutable.Seq().padTo(randomHead, "0") + val tailList = immutable.Seq().padTo(randomTail, "1") val seqRef = system.actorOf(Props(new SequencingActor(testActor, headList, tailList))) @@ -145,7 +146,7 @@ object TestKitUsageSpec { * like to test that the interesting value is received and that you cant * be bothered with the rest */ - class SequencingActor(next: ActorRef, head: Seq[String], tail: Seq[String]) + class SequencingActor(next: ActorRef, head: immutable.Seq[String], tail: immutable.Seq[String]) extends Actor { def receive = { case msg ⇒ { diff --git a/akka-docs/rst/scala/code/docs/zeromq/ZeromqDocSpec.scala b/akka-docs/rst/scala/code/docs/zeromq/ZeromqDocSpec.scala index 0e3b4df262..28ff2e3d34 100644 --- a/akka-docs/rst/scala/code/docs/zeromq/ZeromqDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/zeromq/ZeromqDocSpec.scala @@ -5,13 +5,13 @@ package docs.zeromq import language.postfixOps -import akka.actor.{ Actor, Props } import scala.concurrent.duration._ +import scala.collection.immutable +import akka.actor.{ Actor, Props } import akka.testkit._ -import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension } +import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind } import java.text.SimpleDateFormat import java.util.Date -import akka.zeromq.{ SocketType, Bind } object ZeromqDocSpec { @@ -52,12 +52,12 @@ object ZeromqDocSpec { val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed, currentHeap.getMax)).get // the first frame is the topic, second is the message - pubSocket ! ZMQMessage(Seq(Frame("health.heap"), Frame(heapPayload))) + pubSocket ! ZMQMessage(immutable.Seq(Frame("health.heap"), Frame(heapPayload))) // use akka SerializationExtension to convert to bytes val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get // the first frame is the topic, second is the message - pubSocket ! ZMQMessage(Seq(Frame("health.load"), Frame(loadPayload))) + pubSocket ! ZMQMessage(immutable.Seq(Frame("health.load"), Frame(loadPayload))) } } //#health @@ -146,7 +146,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") { val payload = Array.empty[Byte] //#pub-topic - pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload))) + pubSocket ! ZMQMessage(Frame("foo.bar"), Frame(payload)) //#pub-topic system.stop(subSocket) diff --git a/akka-kernel/src/main/scala/akka/kernel/Main.scala b/akka-kernel/src/main/scala/akka/kernel/Main.scala index 97ff625ab8..3fe3cac403 100644 --- a/akka-kernel/src/main/scala/akka/kernel/Main.scala +++ b/akka-kernel/src/main/scala/akka/kernel/Main.scala @@ -9,6 +9,7 @@ import java.io.File import java.lang.Boolean.getBoolean import java.net.URLClassLoader import java.util.jar.JarFile +import scala.collection.immutable import scala.collection.JavaConverters._ /** @@ -77,8 +78,8 @@ object Main { Thread.currentThread.setContextClassLoader(classLoader) - val bootClasses: Seq[String] = args.toSeq - val bootables: Seq[Bootable] = bootClasses map { c ⇒ classLoader.loadClass(c).newInstance.asInstanceOf[Bootable] } + val bootClasses: immutable.Seq[String] = args.to[immutable.Seq] + val bootables: immutable.Seq[Bootable] = bootClasses map { c ⇒ classLoader.loadClass(c).newInstance.asInstanceOf[Bootable] } for (bootable ← bootables) { log("Starting up " + bootable.getClass.getName) @@ -122,7 +123,7 @@ object Main { new URLClassLoader(urls, Thread.currentThread.getContextClassLoader) } - private def addShutdownHook(bootables: Seq[Bootable]): Unit = { + private def addShutdownHook(bootables: immutable.Seq[Bootable]): Unit = { Runtime.getRuntime.addShutdownHook(new Thread(new Runnable { def run = { log("") diff --git a/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala index 2728a80894..79d07c65a3 100644 --- a/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala +++ b/akka-osgi-aries/src/test/scala/akka/osgi/aries/blueprint/NamespaceHandlerTest.scala @@ -32,7 +32,7 @@ class SimpleNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT import NamespaceHandlerTest._ - val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + val testBundles = buildTestBundles(List( AKKA_OSGI_BLUEPRINT, bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("simple.xml")))) @@ -62,7 +62,7 @@ class ConfigNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT import NamespaceHandlerTest._ - val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + val testBundles = buildTestBundles(List( AKKA_OSGI_BLUEPRINT, bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("config.xml")))) @@ -94,7 +94,7 @@ class DependencyInjectionNamespaceHandlerTest extends WordSpec with MustMatchers import NamespaceHandlerTest._ - val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + val testBundles = buildTestBundles(List( AKKA_OSGI_BLUEPRINT, bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("injection.xml")))) diff --git a/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala index d582209d76..27455be75e 100644 --- a/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala +++ b/akka-osgi/src/test/scala/akka/osgi/ActorSystemActivatorTest.scala @@ -10,6 +10,7 @@ import akka.actor.ActorSystem import akka.pattern.ask import scala.concurrent.Await import scala.concurrent.duration._ +import scala.collection.immutable import akka.util.Timeout import de.kalpatec.pojosr.framework.launch.BundleDescriptor import test.{ RuntimeNameActorSystemActivator, TestActivators, PingPongActorSystemActivator } @@ -32,7 +33,7 @@ class PingPongActorSystemActivatorTest extends WordSpec with MustMatchers with P import ActorSystemActivatorTest._ - val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq( + val testBundles: immutable.Seq[BundleDescriptor] = buildTestBundles(List( bundle(TEST_BUNDLE_NAME).withActivator(classOf[PingPongActorSystemActivator]))) "PingPongActorSystemActivator" must { @@ -65,7 +66,8 @@ class RuntimeNameActorSystemActivatorTest extends WordSpec with MustMatchers wit import ActorSystemActivatorTest._ - val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator]))) + val testBundles: immutable.Seq[BundleDescriptor] = + buildTestBundles(List(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator]))) "RuntimeNameActorSystemActivator" must { diff --git a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala index e993d04f01..b8c7ea24e5 100644 --- a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala +++ b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala @@ -17,7 +17,7 @@ import java.io._ import org.scalatest.{ BeforeAndAfterAll, Suite } import java.util.{ UUID, Date, ServiceLoader, HashMap } import scala.reflect.ClassTag -import scala.Some +import scala.collection.immutable /** * Trait that provides support for building akka-osgi tests using PojoSR @@ -31,7 +31,7 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll { * All bundles being found on the test classpath are automatically installed and started in the PojoSR runtime. * Implement this to define the extra bundles that should be available for testing. */ - def testBundles: Seq[BundleDescriptor] + def testBundles: immutable.Seq[BundleDescriptor] val bufferedLoadingErrors = new ByteArrayOutputStream() @@ -82,15 +82,11 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll { } } - protected def buildTestBundles(builders: Seq[BundleDescriptorBuilder]): Seq[BundleDescriptor] = builders map (_.build) + protected def buildTestBundles(builders: immutable.Seq[BundleDescriptorBuilder]): immutable.Seq[BundleDescriptor] = + builders map (_.build) - def filterErrors()(block: ⇒ Unit): Unit = { - try { - block - } catch { - case e: Throwable ⇒ System.err.write(bufferedLoadingErrors.toByteArray); throw e - } - } + def filterErrors()(block: ⇒ Unit): Unit = + try block catch { case e: Throwable ⇒ System.err.write(bufferedLoadingErrors.toByteArray); throw e } } object PojoSRTestSupport { diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala index 3f2e739308..26d6f6f243 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala @@ -6,13 +6,13 @@ package akka.remote.testconductor import language.postfixOps import java.net.InetSocketAddress import scala.annotation.tailrec -import scala.collection.immutable.Queue +import scala.collection.immutable +import scala.concurrent.duration._ import org.jboss.netty.buffer.ChannelBuffer import org.jboss.netty.channel.{ SimpleChannelHandler, MessageEvent, Channels, ChannelStateEvent, ChannelHandlerContext, ChannelFutureListener, ChannelFuture } import akka.actor.{ Props, LoggingFSM, Address, ActorSystem, ActorRef, ActorLogging, Actor, FSM } import akka.event.Logging import akka.remote.netty.ChannelAddress -import scala.concurrent.duration._ /** * INTERNAL API. @@ -230,7 +230,7 @@ private[akka] object ThrottleActor { case object Throttle extends State case object Blackhole extends State - case class Data(lastSent: Long, rateMBit: Float, queue: Queue[Send]) + case class Data(lastSent: Long, rateMBit: Float, queue: immutable.Queue[Send]) case class Send(ctx: ChannelHandlerContext, direction: Direction, future: Option[ChannelFuture], msg: AnyRef) case class SetRate(rateMBit: Float) @@ -248,7 +248,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) private val packetSplitThreshold = TestConductor(context.system).Settings.PacketSplitThreshold - startWith(PassThrough, Data(0, -1, Queue())) + startWith(PassThrough, Data(0, -1, immutable.Queue())) when(PassThrough) { case Event(s @ Send(_, _, _, msg), _) ⇒ @@ -258,8 +258,8 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) } when(Throttle) { - case Event(s: Send, data @ Data(_, _, Queue())) ⇒ - stay using sendThrottled(data.copy(lastSent = System.nanoTime, queue = Queue(s))) + case Event(s: Send, data @ Data(_, _, immutable.Queue())) ⇒ + stay using sendThrottled(data.copy(lastSent = System.nanoTime, queue = immutable.Queue(s))) case Event(s: Send, data) ⇒ stay using sendThrottled(data.copy(queue = data.queue.enqueue(s))) case Event(Tick, data) ⇒ @@ -286,7 +286,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) whenUnhandled { case Event(SetRate(rate), d) ⇒ if (rate > 0) { - goto(Throttle) using d.copy(lastSent = System.nanoTime, rateMBit = rate, queue = Queue()) + goto(Throttle) using d.copy(lastSent = System.nanoTime, rateMBit = rate, queue = immutable.Queue()) } else if (rate == 0) { goto(Blackhole) } else { @@ -328,7 +328,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) */ private def schedule(d: Data): (Data, Seq[Send], Option[FiniteDuration]) = { val now = System.nanoTime - @tailrec def rec(d: Data, toSend: Seq[Send]): (Data, Seq[Send], Option[FiniteDuration]) = { + @tailrec def rec(d: Data, toSend: immutable.Seq[Send]): (Data, immutable.Seq[Send], Option[FiniteDuration]) = { if (d.queue.isEmpty) (d, toSend, None) else { val timeForPacket = d.lastSent + (1000 * size(d.queue.head.msg) / d.rateMBit).toLong @@ -344,7 +344,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext) } } } - rec(d, Seq()) + rec(d, Nil) } /** diff --git a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala index 2a3957d146..95bfab1ee5 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testconductor/Player.scala @@ -4,20 +4,20 @@ package akka.remote.testconductor import language.postfixOps + +import java.util.concurrent.TimeoutException import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props, PoisonPill, Status, Address, Scheduler } -import RemoteConnection.getAddrString +import akka.remote.testconductor.RemoteConnection.getAddrString +import scala.collection.immutable +import scala.concurrent.{ ExecutionContext, Await, Future } import scala.concurrent.duration._ +import scala.util.control.NoStackTrace +import scala.reflect.classTag import akka.util.Timeout import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent, WriteCompletionEvent, ExceptionEvent } -import com.typesafe.config.ConfigFactory -import java.util.concurrent.TimeUnit.MILLISECONDS -import java.util.concurrent.TimeoutException import akka.pattern.{ ask, pipe, AskTimeoutException } -import scala.util.control.NoStackTrace import akka.event.{ LoggingAdapter, Logging } import java.net.{ InetSocketAddress, ConnectException } -import scala.reflect.classTag -import concurrent.{ ExecutionContext, Await, Future } /** * The Player is the client component of the @@ -67,15 +67,13 @@ trait Player { this: TestConductorExt ⇒ * Enter the named barriers, one after the other, in the order given. Will * throw an exception in case of timeouts or other errors. */ - def enter(name: String*) { - enter(Settings.BarrierTimeout, name) - } + def enter(name: String*): Unit = enter(Settings.BarrierTimeout, name.to[immutable.Seq]) /** * Enter the named barriers, one after the other, in the order given. Will * throw an exception in case of timeouts or other errors. */ - def enter(timeout: Timeout, name: Seq[String]) { + def enter(timeout: Timeout, name: immutable.Seq[String]) { system.log.debug("entering barriers " + name.mkString("(", ", ", ")")) val stop = Deadline.now + timeout.duration name foreach { b ⇒ diff --git a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala index c40a4e5a7b..e79ab4a1ee 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala @@ -7,16 +7,17 @@ import language.implicitConversions import language.postfixOps import java.net.InetSocketAddress +import java.util.concurrent.TimeoutException import com.typesafe.config.{ ConfigObject, ConfigFactory, Config } +import scala.concurrent.{ Await, Awaitable } +import scala.util.control.NonFatal +import scala.collection.immutable import akka.actor._ import akka.util.Timeout import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName } import akka.remote.RemoteActorRefProvider import akka.testkit._ -import scala.concurrent.{ Await, Awaitable } -import scala.util.control.NonFatal import scala.concurrent.duration._ -import java.util.concurrent.TimeoutException import akka.remote.testconductor.RoleName import akka.remote.testconductor.TestConductorTransport import akka.actor.RootActorPath @@ -30,7 +31,7 @@ abstract class MultiNodeConfig { private var _commonConf: Option[Config] = None private var _nodeConf = Map[RoleName, Config]() private var _roles = Vector[RoleName]() - private var _deployments = Map[RoleName, Seq[String]]() + private var _deployments = Map[RoleName, immutable.Seq[String]]() private var _allDeploy = Vector[String]() private var _testTransport = false @@ -106,9 +107,9 @@ abstract class MultiNodeConfig { configs reduce (_ withFallback _) } - private[testkit] def deployments(node: RoleName): Seq[String] = (_deployments get node getOrElse Nil) ++ _allDeploy + private[testkit] def deployments(node: RoleName): immutable.Seq[String] = (_deployments get node getOrElse Nil) ++ _allDeploy - private[testkit] def roles: Seq[RoleName] = _roles + private[testkit] def roles: immutable.Seq[RoleName] = _roles } @@ -234,7 +235,7 @@ object MultiNodeSpec { * `AskTimeoutException: sending to terminated ref breaks promises`. Using lazy * val is fine. */ -abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: Seq[RoleName], deployments: RoleName ⇒ Seq[String]) +abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: immutable.Seq[RoleName], deployments: RoleName ⇒ Seq[String]) extends TestKit(_system) with MultiNodeSpecCallbacks { import MultiNodeSpec._ @@ -294,7 +295,7 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: /** * All registered roles */ - def roles: Seq[RoleName] = _roles + def roles: immutable.Seq[RoleName] = _roles /** * TO BE DEFINED BY USER: Defines the number of participants required for starting the test. This @@ -335,9 +336,10 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: * Enter the named barriers in the order given. Use the remaining duration from * the innermost enclosing `within` block or the default `BarrierTimeout` */ - def enterBarrier(name: String*) { - testConductor.enter(Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)), name) - } + def enterBarrier(name: String*): Unit = + testConductor.enter( + Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)), + name.to[immutable.Seq]) /** * Query the controller for the transport address of the given node (by role name) and diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 4967978582..48fca0af08 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -70,7 +70,7 @@ class RemoteActorRefProvider( _transport = { val fqn = remoteSettings.RemoteTransport - val args = Seq( + val args = List( classOf[ExtendedActorSystem] -> system, classOf[RemoteActorRefProvider] -> this) 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 a65bac1a66..2a067237a7 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -8,7 +8,9 @@ import java.net.InetSocketAddress import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.concurrent.Executors -import scala.collection.mutable.HashMap +import scala.collection.mutable +import scala.collection.immutable +import scala.util.control.NonFatal import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture } import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.jboss.netty.channel.{ ChannelHandlerContext, Channel, DefaultChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal } @@ -20,7 +22,6 @@ import org.jboss.netty.util.{ DefaultObjectSizeEstimator, HashedWheelTimer } import akka.event.Logging import akka.remote.RemoteProtocol.AkkaRemoteProtocol import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteActorRefProvider, RemoteActorRef, RemoteServerStarted } -import scala.util.control.NonFatal import akka.actor.{ ExtendedActorSystem, Address, ActorRef } import com.google.protobuf.MessageLite @@ -53,7 +54,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider * Construct a DefaultChannelPipeline from a sequence of handlers; to be used * in implementations of ChannelPipelineFactory. */ - def apply(handlers: Seq[ChannelHandler]): DefaultChannelPipeline = + def apply(handlers: immutable.Seq[ChannelHandler]): DefaultChannelPipeline = (new DefaultChannelPipeline /: handlers) { (p, h) ⇒ p.addLast(Logging.simpleName(h.getClass), h); p } /** @@ -69,7 +70,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider * Construct a default protocol stack, excluding the “head” handler (i.e. the one which * actually dispatches the received messages to the local target actors). */ - def defaultStack(withTimeout: Boolean, isClient: Boolean): Seq[ChannelHandler] = + def defaultStack(withTimeout: Boolean, isClient: Boolean): immutable.Seq[ChannelHandler] = (if (settings.EnableSSL) List(NettySSLSupport(settings, NettyRemoteTransport.this.log, isClient)) else Nil) ::: (if (withTimeout) List(timeout) else Nil) ::: msgFormat ::: @@ -138,7 +139,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory = PipelineFactory(Seq(endpoint), withTimeout, isClient) - private val remoteClients = new HashMap[Address, RemoteClient] + private val remoteClients = new mutable.HashMap[Address, RemoteClient] private val clientsLock = new ReentrantReadWriteLock override protected def useUntrustedMode = remoteSettings.UntrustedMode diff --git a/akka-remote/src/main/scala/akka/remote/security/provider/InternetSeedGenerator.scala b/akka-remote/src/main/scala/akka/remote/security/provider/InternetSeedGenerator.scala index f049a4e678..b274c4c0b6 100644 --- a/akka-remote/src/main/scala/akka/remote/security/provider/InternetSeedGenerator.scala +++ b/akka-remote/src/main/scala/akka/remote/security/provider/InternetSeedGenerator.scala @@ -16,6 +16,7 @@ package akka.remote.security.provider import org.uncommons.maths.random.{ SeedGenerator, SeedException, SecureRandomSeedGenerator, RandomDotOrgSeedGenerator, DevRandomSeedGenerator } +import scala.collection.immutable /** * Internal API @@ -33,8 +34,8 @@ object InternetSeedGenerator { /**Singleton instance. */ private final val Instance: InternetSeedGenerator = new InternetSeedGenerator /**Delegate generators. */ - private final val Generators: Seq[SeedGenerator] = - Seq(new RandomDotOrgSeedGenerator, // first try the Internet seed generator + private final val Generators: immutable.Seq[SeedGenerator] = + List(new RandomDotOrgSeedGenerator, // first try the Internet seed generator new SecureRandomSeedGenerator) // this is last because it always works } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 520f2557c7..132e3f5e78 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -138,7 +138,7 @@ object TestActorRef { def apply[T <: Actor](implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName) def apply[T <: Actor](name: String)(implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({ - system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](t.runtimeClass, Seq()).recover({ + system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](t.runtimeClass, Nil).recover({ case exception ⇒ throw ActorInitializationException(null, "Could not instantiate Actor" + "\nMake sure Actor is NOT defined inside a class/trait," + diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index 1fdd339ca5..c46f15a26a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -6,14 +6,15 @@ package akka.testkit import language.existentials import scala.util.matching.Regex +import scala.collection.JavaConverters +import scala.collection.immutable +import scala.concurrent.duration.Duration +import scala.reflect.ClassTag import akka.actor.{ DeadLetter, ActorSystem, Terminated, UnhandledMessage } import akka.dispatch.{ SystemMessage, Terminate } import akka.event.Logging.{ Warning, LogEvent, InitializeLogger, Info, Error, Debug, LoggerInitialized } import akka.event.Logging import java.lang.{ Iterable ⇒ JIterable } -import scala.collection.JavaConverters -import scala.concurrent.duration.Duration -import scala.reflect.ClassTag import akka.actor.NoSerializationVerificationNeeded /** @@ -38,22 +39,22 @@ sealed trait TestEvent */ object TestEvent { object Mute { - def apply(filter: EventFilter, filters: EventFilter*): Mute = new Mute(filter +: filters.toSeq) + def apply(filter: EventFilter, filters: EventFilter*): Mute = new Mute(filter +: filters.to[immutable.Seq]) } - case class Mute(filters: Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded { + case class Mute(filters: immutable.Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded { /** * Java API */ - def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.toSeq) + def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.to[immutable.Seq]) } object UnMute { - def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.toSeq) + def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.to[immutable.Seq]) } - case class UnMute(filters: Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded { + case class UnMute(filters: immutable.Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded { /** * Java API */ - def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.toSeq) + def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.to[immutable.Seq]) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 4f6744b74b..e81acb23a3 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -5,15 +5,15 @@ package akka.testkit import language.postfixOps +import scala.annotation.{ varargs, tailrec } +import scala.collection.immutable +import scala.concurrent.duration._ +import scala.reflect.ClassTag +import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic } +import java.util.concurrent.atomic.AtomicInteger import akka.actor._ import akka.actor.Actor._ -import scala.concurrent.duration._ -import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic } -import atomic.AtomicInteger -import scala.annotation.tailrec import akka.util.{ Timeout, BoxedType } -import scala.annotation.varargs -import scala.reflect.ClassTag object TestActor { type Ignore = Option[PartialFunction[Any, Boolean]] @@ -415,7 +415,7 @@ trait TestKitBase { /** * Same as `expectMsgAllOf(remaining, obj...)`, but correctly treating the timeFactor. */ - def expectMsgAllOf[T](obj: T*): Seq[T] = expectMsgAllOf_internal(remaining, obj: _*) + def expectMsgAllOf[T](obj: T*): immutable.Seq[T] = expectMsgAllOf_internal(remaining, obj: _*) /** * Receive a number of messages from the test actor matching the given @@ -430,19 +430,19 @@ trait TestKitBase { * expectMsgAllOf(1 second, Result1(), Result2()) * */ - def expectMsgAllOf[T](max: FiniteDuration, obj: T*): Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*) + def expectMsgAllOf[T](max: FiniteDuration, obj: T*): immutable.Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*) - private def expectMsgAllOf_internal[T](max: FiniteDuration, obj: T*): Seq[T] = { + private def expectMsgAllOf_internal[T](max: FiniteDuration, obj: T*): immutable.Seq[T] = { val recv = receiveN_internal(obj.size, max) obj foreach (x ⇒ assert(recv exists (x == _), "not found " + x)) recv foreach (x ⇒ assert(obj exists (x == _), "found unexpected " + x)) - recv.asInstanceOf[Seq[T]] + recv.asInstanceOf[immutable.Seq[T]] } /** * Same as `expectMsgAllClassOf(remaining, obj...)`, but correctly treating the timeFactor. */ - def expectMsgAllClassOf[T](obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(remaining, obj: _*) + def expectMsgAllClassOf[T](obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllClassOf(remaining, obj: _*) /** * Receive a number of messages from the test actor matching the given @@ -452,19 +452,19 @@ trait TestKitBase { * Wait time is bounded by the given duration, with an AssertionFailure * being thrown in case of timeout. */ - def expectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*) + def expectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*) - private def internalExpectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = { + private def internalExpectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = { val recv = receiveN_internal(obj.size, max) obj foreach (x ⇒ assert(recv exists (_.getClass eq BoxedType(x)), "not found " + x)) recv foreach (x ⇒ assert(obj exists (c ⇒ BoxedType(c) eq x.getClass), "found non-matching object " + x)) - recv.asInstanceOf[Seq[T]] + recv.asInstanceOf[immutable.Seq[T]] } /** * Same as `expectMsgAllConformingOf(remaining, obj...)`, but correctly treating the timeFactor. */ - def expectMsgAllConformingOf[T](obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(remaining, obj: _*) + def expectMsgAllConformingOf[T](obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllConformingOf(remaining, obj: _*) /** * Receive a number of messages from the test actor matching the given @@ -477,13 +477,13 @@ trait TestKitBase { * Beware that one object may satisfy all given class constraints, which * may be counter-intuitive. */ - def expectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*) + def expectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*) - private def internalExpectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = { + private def internalExpectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = { val recv = receiveN_internal(obj.size, max) obj foreach (x ⇒ assert(recv exists (BoxedType(x) isInstance _), "not found " + x)) recv foreach (x ⇒ assert(obj exists (c ⇒ BoxedType(c) isInstance x), "found non-matching object " + x)) - recv.asInstanceOf[Seq[T]] + recv.asInstanceOf[immutable.Seq[T]] } /** @@ -520,7 +520,7 @@ trait TestKitBase { * assert(series == (1 to 7).toList) * */ - def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = { + def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): immutable.Seq[T] = { val stop = now + remainingOrDilated(max) var msg: Message = NullMessage @@ -553,14 +553,14 @@ trait TestKitBase { * Same as `receiveN(n, remaining)` but correctly taking into account * Duration.timeFactor. */ - def receiveN(n: Int): Seq[AnyRef] = receiveN_internal(n, remaining) + def receiveN(n: Int): immutable.Seq[AnyRef] = receiveN_internal(n, remaining) /** * Receive N messages in a row before the given deadline. */ - def receiveN(n: Int, max: FiniteDuration): Seq[AnyRef] = receiveN_internal(n, max.dilated) + def receiveN(n: Int, max: FiniteDuration): immutable.Seq[AnyRef] = receiveN_internal(n, max.dilated) - private def receiveN_internal(n: Int, max: Duration): Seq[AnyRef] = { + private def receiveN_internal(n: Int, max: Duration): immutable.Seq[AnyRef] = { val stop = max + now for { x ← 1 to n } yield { val timeout = stop - now diff --git a/akka-testkit/src/main/scala/akka/testkit/package.scala b/akka-testkit/src/main/scala/akka/testkit/package.scala index 38d6d853af..ff8926154e 100644 --- a/akka-testkit/src/main/scala/akka/testkit/package.scala +++ b/akka-testkit/src/main/scala/akka/testkit/package.scala @@ -4,14 +4,16 @@ import language.implicitConversions import akka.actor.ActorSystem import scala.concurrent.duration.{ Duration, FiniteDuration } -import java.util.concurrent.TimeUnit.MILLISECONDS import scala.reflect.ClassTag +import scala.collection.immutable +import java.util.concurrent.TimeUnit.MILLISECONDS package object testkit { def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit system: ActorSystem): T = { def now = System.currentTimeMillis - system.eventStream.publish(TestEvent.Mute(eventFilters.toSeq)) + system.eventStream.publish(TestEvent.Mute(eventFilters.to[immutable.Seq])) + try { val result = block @@ -23,7 +25,7 @@ package object testkit { result } finally { - system.eventStream.publish(TestEvent.UnMute(eventFilters.toSeq)) + system.eventStream.publish(TestEvent.UnMute(eventFilters.to[immutable.Seq])) } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 5aecd341e0..27323787aa 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import scala.collection.JavaConverters; -import scala.collection.Seq; +import scala.collection.immutable.Seq; public class UntypedCoordinatedIncrementTest { private static ActorSystem system; @@ -110,6 +110,6 @@ public class UntypedCoordinatedIncrementTest { } public Seq seq(A... args) { - return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toSeq(); + return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toList(); } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 5aae61d9c1..3b841e300b 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -32,7 +32,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import scala.collection.JavaConverters; -import scala.collection.Seq; +import scala.collection.immutable.Seq; public class UntypedTransactorTest { @@ -120,6 +120,6 @@ public class UntypedTransactorTest { public Seq seq(A... args) { return JavaConverters .collectionAsScalaIterableConverter(Arrays.asList(args)).asScala() - .toSeq(); + .toList(); } } diff --git a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala index 1aa9fb555c..6dad1079a3 100644 --- a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala @@ -6,12 +6,13 @@ package akka.transactor import org.scalatest.BeforeAndAfterAll -import akka.actor._ import scala.concurrent.Await import scala.concurrent.duration._ +import scala.concurrent.stm._ +import scala.collection.immutable +import akka.actor._ import akka.util.Timeout import akka.testkit._ -import scala.concurrent.stm._ import akka.pattern.{ AskTimeoutException, ask } object CoordinatedIncrement { @@ -30,7 +31,7 @@ object CoordinatedIncrement { } """ - case class Increment(friends: Seq[ActorRef]) + case class Increment(friends: immutable.Seq[ActorRef]) case object GetCount class Counter(name: String) extends Actor { diff --git a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala index a087ab5b6d..eb75247164 100644 --- a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala @@ -8,21 +8,22 @@ import language.postfixOps import org.scalatest.BeforeAndAfterAll -import akka.actor._ import scala.concurrent.Await import scala.concurrent.duration._ +import scala.concurrent.stm._ +import scala.collection.immutable +import scala.util.Random.{ nextInt ⇒ random } +import scala.util.control.NonFatal +import akka.actor._ import akka.testkit._ import akka.testkit.TestEvent.Mute -import scala.concurrent.stm._ -import scala.util.Random.{ nextInt ⇒ random } import java.util.concurrent.CountDownLatch import akka.pattern.{ AskTimeoutException, ask } import akka.util.Timeout -import scala.util.control.NonFatal object FickleFriends { - case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) - case class Increment(friends: Seq[ActorRef]) + case class FriendlyIncrement(friends: immutable.Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) + case class Increment(friends: immutable.Seq[ActorRef]) case object GetCount /** @@ -120,7 +121,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { "Coordinated fickle friends" should { "eventually succeed to increment all counters by one" in { - val ignoreExceptions = Seq( + val ignoreExceptions = immutable.Seq( EventFilter[ExpectedFailureException](), EventFilter[CoordinatedTransactionException](), EventFilter[AskTimeoutException]()) diff --git a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala index 4fd05a0e1e..0de8a13d97 100644 --- a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala @@ -7,15 +7,16 @@ package akka.transactor import language.postfixOps import akka.actor._ +import scala.collection.immutable import scala.concurrent.Await import scala.concurrent.duration._ +import scala.concurrent.stm._ import akka.util.Timeout import akka.testkit._ -import scala.concurrent.stm._ import akka.pattern.{ AskTimeoutException, ask } object TransactorIncrement { - case class Increment(friends: Seq[ActorRef], latch: TestLatch) + case class Increment(friends: immutable.Seq[ActorRef], latch: TestLatch) case object GetCount class Counter(name: String) extends Transactor { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 20c8b2d723..a9efa56c1e 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -6,13 +6,14 @@ package akka.zeromq import org.zeromq.ZMQ.{ Socket, Poller } import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ +import scala.collection.immutable +import scala.annotation.tailrec import scala.concurrent.{ Promise, Future } import scala.concurrent.duration.Duration -import scala.annotation.tailrec import scala.collection.mutable.ListBuffer +import scala.util.control.NonFatal import akka.event.Logging import java.util.concurrent.TimeUnit -import scala.util.control.NonFatal private[zeromq] object ConcurrentSocketActor { private sealed trait PollMsg @@ -25,7 +26,7 @@ private[zeromq] object ConcurrentSocketActor { private val DefaultContext = Context() } -private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends Actor { +private[zeromq] class ConcurrentSocketActor(params: immutable.Seq[SocketOption]) extends Actor { import ConcurrentSocketActor._ private val noBytes = Array[Byte]() @@ -40,7 +41,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A private val socket: Socket = zmqContext.socket(socketType) private val poller: Poller = zmqContext.poller - private val pendingSends = new ListBuffer[Seq[Frame]] + private val pendingSends = new ListBuffer[immutable.Seq[Frame]] def receive = { case m: PollMsg ⇒ doPoll(m) @@ -151,7 +152,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } } finally notifyListener(Closed) - @tailrec private def flushMessage(i: Seq[Frame]): Boolean = + @tailrec private def flushMessage(i: immutable.Seq[Frame]): Boolean = if (i.isEmpty) true else { @@ -198,7 +199,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A case frames ⇒ notifyListener(deserializer(frames)); doPoll(mode, togo - 1) } - @tailrec private def receiveMessage(mode: PollMsg, currentFrames: Vector[Frame] = Vector.empty): Seq[Frame] = + @tailrec private def receiveMessage(mode: PollMsg, currentFrames: Vector[Frame] = Vector.empty): immutable.Seq[Frame] = if (mode == PollCareful && (poller.poll(0) <= 0)) { if (currentFrames.isEmpty) currentFrames else throw new IllegalStateException("Received partial transmission!") } else { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index f935a9c31d..b70c245327 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -4,9 +4,10 @@ package akka.zeromq import com.google.protobuf.Message -import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor.ActorRef import scala.concurrent.duration._ +import scala.collection.immutable +import org.zeromq.{ ZMQ ⇒ JZMQ } import org.zeromq.ZMQ.{ Poller, Socket } /** @@ -36,7 +37,7 @@ sealed trait SocketConnectOption extends SocketOption { * A base trait for pubsub options for the ZeroMQ socket */ sealed trait PubSubOption extends SocketOption { - def payload: Seq[Byte] + def payload: immutable.Seq[Byte] } /** @@ -79,7 +80,7 @@ class Context(numIoThreads: Int) extends SocketMeta { * A base trait for message deserializers */ trait Deserializer extends SocketOption { - def apply(frames: Seq[Frame]): Any + def apply(frames: immutable.Seq[Frame]): Any } /** @@ -172,12 +173,12 @@ case class Bind(endpoint: String) extends SocketConnectOption * * @param payload the topic to subscribe to */ -case class Subscribe(payload: Seq[Byte]) extends PubSubOption { - def this(topic: String) = this(topic.getBytes("UTF-8")) +case class Subscribe(payload: immutable.Seq[Byte]) extends PubSubOption { + def this(topic: String) = this(topic.getBytes("UTF-8").to[immutable.Seq]) } object Subscribe { def apply(topic: String): Subscribe = new Subscribe(topic) - val all = Subscribe(Seq.empty) + val all = Subscribe("") } /** @@ -189,8 +190,8 @@ object Subscribe { * * @param payload */ -case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption { - def this(topic: String) = this(topic.getBytes("UTF-8")) +case class Unsubscribe(payload: immutable.Seq[Byte]) extends PubSubOption { + def this(topic: String) = this(topic.getBytes("UTF-8").to[immutable.Seq]) } object Unsubscribe { def apply(topic: String): Unsubscribe = new Unsubscribe(topic) @@ -200,17 +201,17 @@ object Unsubscribe { * Send a message over the zeromq socket * @param frames */ -case class Send(frames: Seq[Frame]) extends Request +case class Send(frames: immutable.Seq[Frame]) extends Request /** * A message received over the zeromq socket * @param frames */ -case class ZMQMessage(frames: Seq[Frame]) { +case class ZMQMessage(frames: immutable.Seq[Frame]) { - def this(frame: Frame) = this(Seq(frame)) - def this(frame1: Frame, frame2: Frame) = this(Seq(frame1, frame2)) - def this(frameArray: Array[Frame]) = this(frameArray.toSeq) + def this(frame: Frame) = this(List(frame)) + def this(frame1: Frame, frame2: Frame) = this(List(frame1, frame2)) + def this(frameArray: Array[Frame]) = this(frameArray.to[immutable.Seq]) /** * Convert the bytes in the first frame to a String, using specified charset. @@ -224,8 +225,9 @@ case class ZMQMessage(frames: Seq[Frame]) { def payload(frameIndex: Int): Array[Byte] = frames(frameIndex).payload.toArray } object ZMQMessage { - def apply(bytes: Array[Byte]): ZMQMessage = ZMQMessage(Seq(Frame(bytes))) - def apply(message: Message): ZMQMessage = ZMQMessage(message.toByteArray) + def apply(bytes: Array[Byte]): ZMQMessage = new ZMQMessage(List(Frame(bytes))) + def apply(frames: Frame*): ZMQMessage = new ZMQMessage(frames.to[immutable.Seq]) + def apply(message: Message): ZMQMessage = apply(message.toByteArray) } /** diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index 2d41424e88..d0141bf515 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -3,7 +3,10 @@ */ package akka.zeromq +import scala.collection.immutable + object Frame { + def apply(bytes: Array[Byte]): Frame = new Frame(bytes) def apply(text: String): Frame = new Frame(text) } @@ -11,8 +14,8 @@ object Frame { * A single message frame of a zeromq message * @param payload */ -case class Frame(payload: Seq[Byte]) { - def this(bytes: Array[Byte]) = this(bytes.toSeq) +case class Frame(payload: immutable.Seq[Byte]) { + def this(bytes: Array[Byte]) = this(bytes.to[immutable.Seq]) def this(text: String) = this(text.getBytes("UTF-8")) } @@ -20,5 +23,5 @@ case class Frame(payload: Seq[Byte]) { * Deserializes ZeroMQ messages into an immutable sequence of frames */ class ZMQMessageDeserializer extends Deserializer { - def apply(frames: Seq[Frame]): ZMQMessage = ZMQMessage(frames) + def apply(frames: immutable.Seq[Frame]): ZMQMessage = ZMQMessage(frames) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 85e05e54f3..bc40ea580b 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -7,6 +7,7 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import org.zeromq.ZMQ.Poller import akka.actor._ import akka.pattern.ask +import scala.collection.immutable import scala.concurrent.Await import scala.concurrent.duration.Duration import java.util.concurrent.TimeUnit @@ -66,7 +67,8 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { case s: SocketType.ZMQSocketType ⇒ true case _ ⇒ false }, "A socket type is required") - Props(new ConcurrentSocketActor(socketParameters)).withDispatcher("akka.zeromq.socket-dispatcher") + val params = socketParameters.to[immutable.Seq] + Props(new ConcurrentSocketActor(params)).withDispatcher("akka.zeromq.socket-dispatcher") } /** diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 68123761c5..6feaffd6d6 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -51,7 +51,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec { val msgGenerator = system.scheduler.schedule(100 millis, 10 millis, new Runnable { var number = 0 def run() { - publisher ! ZMQMessage(Seq(Frame(number.toString.getBytes), Frame(Seq()))) + publisher ! ZMQMessage(Frame(number.toString), Frame(Nil)) number += 1 } }) @@ -88,8 +88,8 @@ class ConcurrentSocketActorSpec extends AkkaSpec { try { replierProbe.expectMsg(Connecting) - val request = ZMQMessage(Seq(Frame("Request"))) - val reply = ZMQMessage(Seq(Frame("Reply"))) + val request = ZMQMessage(Frame("Request")) + val reply = ZMQMessage(Frame("Reply")) requester ! request replierProbe.expectMsg(request) @@ -112,7 +112,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec { try { pullerProbe.expectMsg(Connecting) - val message = ZMQMessage(Seq(Frame("Pushed message"))) + val message = ZMQMessage(Frame("Pushed message")) pusher ! message pullerProbe.expectMsg(message) From f5185860ef7dde09041f4e35b6a1efbbcfc7a0ef Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 14 Nov 2012 14:21:56 +0100 Subject: [PATCH 34/62] amend one place in docs referring to blocking system.actorOf --- akka-docs/rst/general/actor-systems.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-docs/rst/general/actor-systems.rst b/akka-docs/rst/general/actor-systems.rst index 99df8bd334..22768a7342 100644 --- a/akka-docs/rst/general/actor-systems.rst +++ b/akka-docs/rst/general/actor-systems.rst @@ -108,8 +108,8 @@ Actor Best Practices #. Top-level actors are the innermost part of your Error Kernel, so create them sparingly and prefer truly hierarchical systems. This has benefits wrt. fault-handling (both considering the granularity of configuration and the - performance) and it also reduces the number of blocking calls made, since - the creation of top-level actors involves synchronous messaging. + performance) and it also reduces the strain on the guardian actor, which is + a single point of contention if over-used. Blocking Needs Careful Management --------------------------------- From b00c47854b5a6645896ba3cdc9e24ce2e7696264 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 7 Nov 2012 16:35:14 +0100 Subject: [PATCH 35/62] Switching over to immutable.Iterable where possible --- .../java/akka/routing/CustomRouteTest.java | 3 +- .../scala/akka/actor/ActorSystemSpec.scala | 11 +- .../scala/akka/routing/CustomRouteSpec.scala | 6 +- .../test/scala/akka/routing/ResizerSpec.scala | 7 +- .../test/scala/akka/routing/RoutingSpec.scala | 21 +-- akka-actor/src/main/java/akka/japi/JAPI.java | 2 +- .../src/main/scala/akka/actor/ActorCell.scala | 6 +- .../src/main/scala/akka/actor/ActorPath.scala | 23 ++- .../scala/akka/actor/ActorRefProvider.scala | 10 +- .../main/scala/akka/actor/ActorSystem.scala | 22 ++- .../src/main/scala/akka/actor/Address.scala | 9 +- .../src/main/scala/akka/actor/Deployer.scala | 5 +- .../main/scala/akka/actor/FaultHandling.scala | 23 +-- .../src/main/scala/akka/actor/Scheduler.scala | 4 +- .../main/scala/akka/actor/TypedActor.scala | 3 +- .../scala/akka/actor/dungeon/Children.scala | 11 +- .../actor/dungeon/ChildrenContainer.scala | 43 +++-- .../akka/actor/dungeon/FaultHandling.scala | 4 +- .../src/main/scala/akka/dispatch/Future.scala | 2 +- .../src/main/scala/akka/event/EventBus.scala | 4 +- .../src/main/scala/akka/japi/JavaAPI.scala | 36 +++- .../routing/ConsistentHashingRouter.scala | 17 +- .../src/main/scala/akka/routing/Routing.scala | 160 +++++++++--------- .../src/main/scala/akka/routing/package.scala | 4 +- .../akka/serialization/Serialization.scala | 30 ++-- .../main/scala/akka/util/Collections.scala | 54 ++++++ .../src/main/scala/akka/util/Convert.scala | 45 ----- .../src/main/scala/akka/util/Index.scala | 2 +- .../src/main/scala/akka/camel/Camel.scala | 9 +- .../scala/akka/cluster/ClusterSettings.scala | 4 +- .../cluster/routing/ClusterRouterConfig.scala | 20 +-- .../docs/actor/FaultHandlingTestBase.java | 4 +- .../jrouting/CustomRouterDocTestBase.java | 8 +- .../project/migration-guide-2.1.x-2.2.x.rst | 26 +++ akka-docs/rst/project/migration-guides.rst | 1 + .../scala/akka/osgi/PojoSRTestSupport.scala | 7 +- .../akka/remote/testkit/MultiNodeSpec.scala | 6 +- .../scala/akka/remote/RemoteDeployer.scala | 9 +- .../scala/akka/remote/netty/Settings.scala | 41 +---- .../remote/routing/RemoteRouterConfig.scala | 22 ++- .../main/java/akka/testkit/JavaTestKit.java | 12 +- .../akka/testkit/TestEventListener.scala | 8 +- .../UntypedCoordinatedIncrementTest.java | 4 +- .../transactor/UntypedTransactorTest.java | 6 +- 44 files changed, 385 insertions(+), 369 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/util/Collections.scala delete mode 100644 akka-actor/src/main/scala/akka/util/Convert.scala create mode 100644 akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst diff --git a/akka-actor-tests/src/test/java/akka/routing/CustomRouteTest.java b/akka-actor-tests/src/test/java/akka/routing/CustomRouteTest.java index d47c49e28d..c0ccd4de26 100644 --- a/akka-actor-tests/src/test/java/akka/routing/CustomRouteTest.java +++ b/akka-actor-tests/src/test/java/akka/routing/CustomRouteTest.java @@ -15,7 +15,8 @@ public class CustomRouteTest { // only to test compilability public void testRoute() { final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1))); - final scala.Function1, scala.collection.Iterable> route = ExtractRoute.apply(ref); + final scala.Function1, + scala.collection.immutable.Iterable> route = ExtractRoute.apply(ref); route.apply(null); } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index bf0128a7e3..038cb2d0e6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -4,14 +4,15 @@ package akka.actor import language.postfixOps + import akka.testkit._ import org.scalatest.junit.JUnitSuite import com.typesafe.config.ConfigFactory import scala.concurrent.Await import scala.concurrent.duration._ -import scala.collection.JavaConverters -import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue } +import java.util.concurrent.{ RejectedExecutionException, ConcurrentLinkedQueue } import akka.util.Timeout +import akka.japi.Util.immutableSeq import scala.concurrent.Future import akka.pattern.ask @@ -102,8 +103,6 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt } "run termination callbacks in order" in { - import scala.collection.JavaConverters._ - val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf) val result = new ConcurrentLinkedQueue[Int] val count = 10 @@ -121,13 +120,11 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt Await.ready(latch, 5 seconds) val expected = (for (i ← 1 to count) yield i).reverse - result.asScala.toSeq must be(expected) + immutableSeq(result) must be(expected) } "awaitTermination after termination callbacks" in { - import scala.collection.JavaConverters._ - val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) @volatile var callbackWasRun = false diff --git a/akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala index 945be2b97d..00bd46f430 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala @@ -19,7 +19,7 @@ class CustomRouteSpec extends AkkaSpec { provider.createRoutees(1) { - case (sender, message: String) ⇒ Seq(Destination(sender, target)) + case (sender, message: String) ⇒ List(Destination(sender, target)) case (sender, message) ⇒ toAll(sender, provider.routees) } } @@ -43,8 +43,8 @@ class CustomRouteSpec extends AkkaSpec { val r = Await.result(router.ask(CurrentRoutees)(1 second). mapTo[RouterRoutees], 1 second) r.routees.size must be(1) - route(testActor -> "hallo") must be(Seq(Destination(testActor, target))) - route(testActor -> 12) must be(Seq(Destination(testActor, r.routees.head))) + route(testActor -> "hallo") must be(List(Destination(testActor, target))) + route(testActor -> 12) must be(List(Destination(testActor, r.routees.head))) //#test-route } diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index ac0de07dcf..9a9dbe24ac 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -10,10 +10,9 @@ import akka.testkit.TestEvent._ import akka.actor.Props import scala.concurrent.Await import scala.concurrent.duration._ +import scala.collection.immutable import akka.actor.ActorRef -import java.util.concurrent.atomic.AtomicInteger import akka.pattern.ask -import java.util.concurrent.TimeoutException import scala.util.Try object ResizerSpec { @@ -61,10 +60,10 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with lowerBound = 2, upperBound = 3) - val c1 = resizer.capacity(IndexedSeq.empty[ActorRef]) + val c1 = resizer.capacity(immutable.IndexedSeq.empty[ActorRef]) c1 must be(2) - val current = IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor])) + val current = immutable.IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor])) val c2 = resizer.capacity(current) c2 must be(0) } diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 6d643005aa..94a650e8c7 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -7,7 +7,7 @@ import language.postfixOps import java.util.concurrent.atomic.AtomicInteger import akka.actor._ -import scala.collection.mutable.LinkedList +import scala.collection.immutable import akka.testkit._ import scala.concurrent.duration._ import scala.concurrent.Await @@ -17,7 +17,7 @@ import akka.pattern.{ ask, pipe } import java.util.concurrent.ConcurrentHashMap import com.typesafe.config.Config import akka.dispatch.Dispatchers -import akka.util.Timeout +import akka.util.Collections.EmptyImmutableSeq object RoutingSpec { @@ -54,11 +54,10 @@ object RoutingSpec { class MyRouter(config: Config) extends RouterConfig { val foo = config.getString("foo") def createRoute(routeeProvider: RouteeProvider): Route = { - val routees = IndexedSeq(routeeProvider.context.actorOf(Props[Echo])) - routeeProvider.registerRoutees(routees) + routeeProvider.registerRoutees(List(routeeProvider.context.actorOf(Props[Echo]))) { - case (sender, message) ⇒ Nil + case (sender, message) ⇒ EmptyImmutableSeq } } def routerDispatcher: String = Dispatchers.DefaultDispatcherId @@ -251,15 +250,15 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with val doneLatch = new TestLatch(connectionCount) //lets create some connections. - var actors = new LinkedList[ActorRef] - var counters = new LinkedList[AtomicInteger] + @volatile var actors = immutable.IndexedSeq[ActorRef]() + @volatile var counters = immutable.IndexedSeq[AtomicInteger]() for (i ← 0 until connectionCount) { counters = counters :+ new AtomicInteger() val actor = system.actorOf(Props(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() - case msg: Int ⇒ counters.get(i).get.addAndGet(msg) + case msg: Int ⇒ counters(i).addAndGet(msg) } })) actors = actors :+ actor @@ -278,10 +277,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with //now wait some and do validations. Await.ready(doneLatch, remaining) - for (i ← 0 until connectionCount) { - val counter = counters.get(i).get - counter.get must be((iterationCount * (i + 1))) - } + for (i ← 0 until connectionCount) + counters(i).get must be((iterationCount * (i + 1))) } "deliver a broadcast message using the !" in { diff --git a/akka-actor/src/main/java/akka/japi/JAPI.java b/akka-actor/src/main/java/akka/japi/JAPI.java index 4808b3e725..4c040220f3 100644 --- a/akka-actor/src/main/java/akka/japi/JAPI.java +++ b/akka-actor/src/main/java/akka/japi/JAPI.java @@ -5,7 +5,7 @@ import scala.collection.Seq; public class JAPI { public static Seq seq(T... ts) { - return Util.arrayToSeq(ts); + return Util.immutableSeq(ts); } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 14d280eb98..75dbd3fefa 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -6,7 +6,7 @@ package akka.actor import java.io.{ ObjectOutputStream, NotSerializableException } import scala.annotation.tailrec -import scala.collection.immutable.TreeSet +import scala.collection.immutable import scala.concurrent.duration.Duration import scala.util.control.NonFatal import akka.actor.dungeon.ChildrenContainer @@ -108,7 +108,7 @@ trait ActorContext extends ActorRefFactory { * val goodLookup = context.actorFor("kid") * }}} */ - def children: Iterable[ActorRef] + def children: immutable.Iterable[ActorRef] /** * Get the child with the given name if it exists. @@ -287,7 +287,7 @@ private[akka] object ActorCell { final val emptyBehaviorStack: List[Actor.Receive] = Nil - final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty + final val emptyActorRefSet: Set[ActorRef] = immutable.TreeSet.empty } //ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit) diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index a20e8220b1..4cb61d2212 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -3,6 +3,8 @@ */ package akka.actor import scala.annotation.tailrec +import scala.collection.immutable +import akka.japi.Util.immutableSeq import java.net.MalformedURLException object ActorPath { @@ -20,6 +22,8 @@ object ActorPath { * http://www.ietf.org/rfc/rfc2396.txt */ val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r + + private[akka] final val emptyActorPath: immutable.Iterable[String] = List("") } /** @@ -68,23 +72,18 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable { /** * ''Java API'': Recursively create a descendant’s path by appending all child names. */ - def descendant(names: java.lang.Iterable[String]): ActorPath = { - import scala.collection.JavaConverters._ - /(names.asScala) - } + def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names)) /** * Sequence of names for this path from root to this. Performance implication: has to allocate a list. */ - def elements: Iterable[String] + def elements: immutable.Iterable[String] /** * ''Java API'': Sequence of names for this path from root to this. Performance implication: has to allocate a list. */ - def getElements: java.lang.Iterable[String] = { - import scala.collection.JavaConverters._ - elements.asJava - } + def getElements: java.lang.Iterable[String] = + scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava /** * Walk up the tree to obtain and return the RootActorPath. @@ -112,7 +111,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act override def /(child: String): ActorPath = new ChildActorPath(this, child) - override val elements: Iterable[String] = List("") + override def elements: immutable.Iterable[String] = ActorPath.emptyActorPath override val toString: String = address + name @@ -134,9 +133,9 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto override def /(child: String): ActorPath = new ChildActorPath(this, child) - override def elements: Iterable[String] = { + override def elements: immutable.Iterable[String] = { @tailrec - def rec(p: ActorPath, acc: List[String]): Iterable[String] = p match { + def rec(p: ActorPath, acc: List[String]): immutable.Iterable[String] = p match { case r: RootActorPath ⇒ acc case _ ⇒ rec(p.parent, p.name :: acc) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index a11da0c150..b0b4c3d939 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -8,8 +8,9 @@ import akka.dispatch._ import akka.routing._ import akka.event._ import akka.util.{ Switch, Helpers } +import akka.japi.Util.immutableSeq +import akka.util.Collections.EmptyImmutableSeq import scala.util.{ Success, Failure } -import scala.util.control.NonFatal import scala.concurrent.{ Future, Promise } import java.util.concurrent.atomic.AtomicLong @@ -271,10 +272,7 @@ trait ActorRefFactory { * * For maximum performance use a collection with efficient head & tail operations. */ - def actorFor(path: java.lang.Iterable[String]): ActorRef = { - import scala.collection.JavaConverters._ - provider.actorFor(lookupRoot, path.asScala) - } + def actorFor(path: java.lang.Iterable[String]): ActorRef = provider.actorFor(lookupRoot, immutableSeq(path)) /** * Construct an [[akka.actor.ActorSelection]] from the given path, which is @@ -480,7 +478,7 @@ class LocalActorRefProvider( def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras private def guardianSupervisorStrategyConfigurator = - dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Nil).get + dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, EmptyImmutableSeq).get /** * Overridable supervision strategy to be used by the “/user” guardian. diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b4309bcb5c..8bada6e0ba 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -6,7 +6,7 @@ package akka.actor import akka.event._ import akka.dispatch._ -import akka.pattern.ask +import akka.japi.Util.immutableSeq import com.typesafe.config.{ Config, ConfigFactory } import scala.annotation.tailrec import scala.collection.immutable @@ -144,7 +144,7 @@ object ActorSystem { final val LogLevel: String = getString("akka.loglevel") final val StdoutLogLevel: String = getString("akka.stdout-loglevel") - final val EventHandlers: immutable.Seq[String] = getStringList("akka.event-handlers").asScala.to[Vector] + final val EventHandlers: immutable.Seq[String] = immutableSeq(getStringList("akka.event-handlers")) final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS)) final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start") @@ -273,8 +273,7 @@ abstract class ActorSystem extends ActorRefFactory { /** * ''Java API'': Recursively create a descendant’s path by appending all child names. */ - def descendant(names: java.lang.Iterable[String]): ActorPath = - /(scala.collection.JavaConverters.iterableAsScalaIterableConverter(names).asScala) + def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names)) /** * Start-up time in milliseconds since the epoch. @@ -674,15 +673,14 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null private def loadExtensions() { - scala.collection.JavaConverters.collectionAsScalaIterableConverter( - settings.config.getStringList("akka.extensions")).asScala foreach { fqcn ⇒ - dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match { - case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()) - case Success(p: ExtensionId[_]) ⇒ registerExtension(p) - case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) - case Failure(problem) ⇒ log.error(problem, "While trying to load extension [{}], skipping...", fqcn) - } + immutableSeq(settings.config.getStringList("akka.extensions")) foreach { fqcn ⇒ + dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match { + case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()) + case Success(p: ExtensionId[_]) ⇒ registerExtension(p) + case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) + case Failure(problem) ⇒ log.error(problem, "While trying to load extension [{}], skipping...", fqcn) } + } } override def toString: String = lookupRoot.path.root.address.toString diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index b8f8a52e45..d98bbcb208 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -120,13 +120,12 @@ object AddressFromURIString { * Given an ActorPath it returns the Address and the path elements if the path is well-formed */ object ActorPathExtractor extends PathUtils { - def unapply(addr: String): Option[(Address, Iterable[String])] = + def unapply(addr: String): Option[(Address, immutable.Iterable[String])] = try { val uri = new URI(addr) - if (uri.getRawPath == null) None - else AddressFromURIString.unapply(uri) match { - case None ⇒ None - case Some(addr) ⇒ Some((addr, split(uri.getRawPath).drop(1))) + uri.getRawPath match { + case null ⇒ None + case path ⇒ AddressFromURIString.unapply(uri).map((_, split(path).drop(1))) } } catch { case _: URISyntaxException ⇒ None diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index e74b54c320..0edb0a0490 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -7,10 +7,11 @@ package akka.actor import scala.concurrent.duration.Duration import com.typesafe.config._ import akka.routing._ +import akka.japi.Util.immutableSeq import java.util.concurrent.{ TimeUnit } import akka.util.WildcardTree import java.util.concurrent.atomic.AtomicReference -import annotation.tailrec +import scala.annotation.tailrec /** * This class represents deployment configuration for a given actor path. It is @@ -141,7 +142,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce val deployment = config.withFallback(default) - val routees = Vector() ++ deployment.getStringList("routees.paths").asScala + val routees = immutableSeq(deployment.getStringList("routees.paths")) val nrOfInstances = deployment.getInt("nr-of-instances") diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 444618df00..7f65c84d02 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -7,9 +7,11 @@ import language.implicitConversions import java.lang.{ Iterable ⇒ JIterable } import java.util.concurrent.TimeUnit +import akka.japi.Util.immutableSeq import scala.collection.mutable.ArrayBuffer import scala.collection.immutable import scala.concurrent.duration.Duration + /** * INTERNAL API */ @@ -171,7 +173,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Implicit conversion from `Seq` of Throwables to a `Decider`. * This maps the given Throwables to restarts, otherwise escalates. */ - implicit def seqThrowable2Decider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeImmutableDecider(trapExit) + implicit def seqThrowable2Decider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit) type Decider = PartialFunction[Throwable, Directive] type JDecider = akka.japi.Function[Throwable, Directive] @@ -181,22 +183,15 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeImmutableDecider(trapExit) + def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = { + case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate + } + /** * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = - makeImmutableDecider(scala.collection.JavaConverters.iterableAsScalaIterableConverter(trapExit).asScala) - - private[this] def makeImmutableDecider(trapExit: Iterable[Class[_]]): Decider = { - val traps = trapExit match { // This is the sad, awkward, truth - case s: immutable.Seq[_] ⇒ s.asInstanceOf[immutable.Seq[Class[_]]] - case other ⇒ other.to[immutable.Seq] - } - - { case x ⇒ if (traps exists (_ isInstance x)) Restart else Escalate } - } + def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(immutableSeq(trapExit)) /** * Decider builder for Iterables of cause-directive pairs, e.g. a map obtained @@ -228,7 +223,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { case x ⇒ buf insert (x, ca) } buf - }.to[immutable.Seq] + }.to[immutable.IndexedSeq] private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] = if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 8e11baada0..2f8310a071 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -203,8 +203,8 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter) } override def close(): Unit = { - import scala.collection.JavaConverters._ - hashedWheelTimer.stop().asScala foreach execDirectly + val i = hashedWheelTimer.stop().iterator() + while (i.hasNext) execDirectly(i.next()) } } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index f7b0e853ef..cc12ed07a2 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -13,6 +13,7 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.concurrent.{ Await, Future } import akka.japi.{ Creator, Option ⇒ JOption } +import akka.japi.Util.{ immutableSeq, immutableSingletonSeq } import akka.util.Timeout import akka.util.Reflect.instantiator import akka.serialization.{ JavaSerializer, SerializationExtension } @@ -442,7 +443,7 @@ object TypedProps { * or a sequence containing only itself, if itself is an interface. */ def extractInterfaces(clazz: Class[_]): immutable.Seq[Class[_]] = - if (clazz.isInterface) List[Class[_]](clazz) else clazz.getInterfaces.to[List] + if (clazz.isInterface) immutableSingletonSeq(clazz) else immutableSeq(clazz.getInterfaces) /** * Uses the supplied class as the factory for the TypedActor implementation, diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala index d59b0b71da..74e06f3034 100644 --- a/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/Children.scala @@ -5,14 +5,12 @@ package akka.actor.dungeon import scala.annotation.tailrec -import scala.collection.JavaConverters.asJavaIterableConverter import scala.util.control.NonFatal +import scala.collection.immutable import akka.actor._ -import akka.actor.ActorCell import akka.actor.ActorPath.ElementRegex import akka.serialization.SerializationExtension import akka.util.{ Unsafe, Helpers } -import akka.actor.ChildNameReserved private[akka] trait Children { this: ActorCell ⇒ @@ -24,8 +22,9 @@ private[akka] trait Children { this: ActorCell ⇒ def childrenRefs: ChildrenContainer = Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer] - final def children: Iterable[ActorRef] = childrenRefs.children - final def getChildren(): java.lang.Iterable[ActorRef] = children.asJava + final def children: immutable.Iterable[ActorRef] = childrenRefs.children + final def getChildren(): java.lang.Iterable[ActorRef] = + scala.collection.JavaConverters.asJavaIterableConverter(children).asJava final def child(name: String): Option[ActorRef] = Option(getChild(name)) final def getChild(name: String): ActorRef = childrenRefs.getByName(name) match { @@ -141,7 +140,7 @@ private[akka] trait Children { this: ActorCell ⇒ protected def getChildByRef(ref: ActorRef): Option[ChildRestartStats] = childrenRefs.getByRef(ref) - protected def getAllChildStats: Iterable[ChildRestartStats] = childrenRefs.stats + protected def getAllChildStats: immutable.Iterable[ChildRestartStats] = childrenRefs.stats protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = { childrenRefs match { diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala b/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala index eeb28cf018..1fccbf8078 100644 --- a/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/ChildrenContainer.scala @@ -4,10 +4,11 @@ package akka.actor.dungeon -import scala.collection.immutable.TreeMap +import scala.collection.immutable import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef } import akka.dispatch.SystemMessage +import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable } /** * INTERNAL API @@ -20,8 +21,8 @@ private[akka] trait ChildrenContainer { def getByName(name: String): Option[ChildStats] def getByRef(actor: ActorRef): Option[ChildRestartStats] - def children: Iterable[ActorRef] - def stats: Iterable[ChildRestartStats] + def children: immutable.Iterable[ActorRef] + def stats: immutable.Iterable[ChildRestartStats] def shallDie(actor: ActorRef): ChildrenContainer @@ -49,6 +50,18 @@ private[akka] object ChildrenContainer { case class Creation() extends SuspendReason with WaitingForChildren case object Termination extends SuspendReason + class ChildRestartsIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] { + override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats] + override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats] + override final def valuesIterator = stats.valuesIterator + } + + class ChildrenIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] { + override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child + override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats] + override final def valuesIterator = stats.valuesIterator + } + trait WaitingForChildren { private var todo: SystemMessage = null def enqueue(message: SystemMessage) = { message.next = todo; todo = message } @@ -56,13 +69,13 @@ private[akka] object ChildrenContainer { } trait EmptyChildrenContainer extends ChildrenContainer { - val emptyStats = TreeMap.empty[String, ChildStats] + val emptyStats = immutable.TreeMap.empty[String, ChildStats] override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, stats)) override def remove(child: ActorRef): ChildrenContainer = this override def getByName(name: String): Option[ChildRestartStats] = None override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None - override def children: Iterable[ActorRef] = Nil - override def stats: Iterable[ChildRestartStats] = Nil + override def children: immutable.Iterable[ActorRef] = EmptyImmutableSeq + override def stats: immutable.Iterable[ChildRestartStats] = EmptyImmutableSeq override def shallDie(actor: ActorRef): ChildrenContainer = this override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved)) override def unreserve(name: String): ChildrenContainer = this @@ -95,7 +108,7 @@ private[akka] object ChildrenContainer { * calling context.stop(child) and processing the ChildTerminated() system * message). */ - class NormalChildrenContainer(val c: TreeMap[String, ChildStats]) extends ChildrenContainer { + class NormalChildrenContainer(val c: immutable.TreeMap[String, ChildStats]) extends ChildrenContainer { override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(c.updated(name, stats)) @@ -108,9 +121,11 @@ private[akka] object ChildrenContainer { case _ ⇒ None } - override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child } + override def children: immutable.Iterable[ActorRef] = + if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c) - override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c } + override def stats: immutable.Iterable[ChildRestartStats] = + if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c) override def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest) @@ -130,7 +145,7 @@ private[akka] object ChildrenContainer { } object NormalChildrenContainer { - def apply(c: TreeMap[String, ChildStats]): ChildrenContainer = + def apply(c: immutable.TreeMap[String, ChildStats]): ChildrenContainer = if (c.isEmpty) EmptyChildrenContainer else new NormalChildrenContainer(c) } @@ -145,7 +160,7 @@ private[akka] object ChildrenContainer { * type of container, depending on whether or not children are left and whether or not * the reason was “Terminating”. */ - case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason) + case class TerminatingChildrenContainer(c: immutable.TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason) extends ChildrenContainer { override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats)) @@ -166,9 +181,11 @@ private[akka] object ChildrenContainer { case _ ⇒ None } - override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child } + override def children: immutable.Iterable[ActorRef] = + if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c) - override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c } + override def stats: immutable.Iterable[ChildRestartStats] = + if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c) override def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor) diff --git a/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala index 5ea0772f61..ac4f5b5c36 100644 --- a/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/dungeon/FaultHandling.scala @@ -10,7 +10,7 @@ import akka.dispatch._ import akka.event.Logging.{ Warning, Error, Debug } import scala.util.control.NonFatal import akka.event.Logging -import scala.Some +import scala.collection.immutable import akka.dispatch.ChildTerminated import akka.actor.PreRestartException import akka.actor.Failed @@ -160,7 +160,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ } } - final def handleInvokeFailure(childrenNotToSuspend: Iterable[ActorRef], t: Throwable, message: String): Unit = { + final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable, message: String): Unit = { publish(Error(t, self.path.toString, clazz(actor), message)) // prevent any further messages to be processed until the actor has been restarted if (!isFailed) try { diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 59c0afe5aa..a7c964b750 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -95,7 +95,7 @@ object Futures { */ def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = { implicit val ec = executor - Future.find[T](futures.asScala)(predicate.apply(_))(executor).map(JOption.fromScalaOption(_)) + Future.find[T](futures.asScala)(predicate.apply(_))(executor) map JOption.fromScalaOption } /** diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index 403f7a0dfd..6e3e25e42c 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -266,9 +266,9 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒ } } - protected final def dissociate(monitored: ActorRef): Iterable[ActorRef] = { + protected final def dissociate(monitored: ActorRef): immutable.Iterable[ActorRef] = { @tailrec - def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = { + def dissociateAsMonitored(monitored: ActorRef): immutable.Iterable[ActorRef] = { val current = mappings get monitored current match { case null ⇒ empty diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index fda674a02c..87bb338b0f 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -10,6 +10,7 @@ import scala.collection.immutable import scala.reflect.ClassTag import scala.util.control.NoStackTrace import scala.runtime.AbstractPartialFunction +import akka.util.Collections.EmptyImmutableSeq import java.util.Collections.{ emptyList, singletonList } /** @@ -175,9 +176,40 @@ object Option { * This class hold common utilities for Java */ object Util { + + /** + * Returns a ClassTag describing the provided Class. + * + * Java API + */ def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz) - def arrayToSeq[T](arr: Array[T]): immutable.Seq[T] = arr.to[immutable.Seq] + /** + * Returns an immutable.Seq representing the provided array of Classes, + * an overloading of the generic immutableSeq in Util, to accommodate for erasure. + * + * Java API + */ + def immutableSeq(arr: Array[Class[_]]): immutable.Seq[Class[_]] = immutableSeq[Class[_]](arr) - def arrayToSeq(classes: Array[Class[_]]): immutable.Seq[Class[_]] = classes.to[immutable.Seq] + /** + * + */ + def immutableSeq[T](arr: Array[T]): immutable.Seq[T] = if ((arr ne null) && arr.length > 0) Vector(arr: _*) else Nil + + def immutableSeq[T](iterable: java.lang.Iterable[T]): immutable.Seq[T] = + iterable match { + case imm: immutable.Seq[_] ⇒ imm.asInstanceOf[immutable.Seq[T]] + case other ⇒ + val i = other.iterator() + if (i.hasNext) { + val builder = new immutable.VectorBuilder[T] + + do { builder += i.next() } while (i.hasNext) + + builder.result() + } else EmptyImmutableSeq + } + + def immutableSingletonSeq[T](value: T): immutable.Seq[T] = value :: Nil } diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index ca9027c8e8..0214c6736e 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -3,11 +3,11 @@ */ package akka.routing -import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import scala.collection.immutable +import akka.japi.Util.immutableSeq import scala.util.control.NonFatal import akka.actor.ActorRef import akka.actor.SupervisorStrategy -import akka.actor.Props import akka.dispatch.Dispatchers import akka.event.Logging import akka.serialization.SerializationExtension @@ -19,16 +19,13 @@ object ConsistentHashingRouter { /** * Creates a new ConsistentHashingRouter, routing to the specified routees */ - def apply(routees: Iterable[ActorRef]): ConsistentHashingRouter = + def apply(routees: immutable.Iterable[ActorRef]): ConsistentHashingRouter = new ConsistentHashingRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala) - } + def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees)) /** * If you don't define the `hashMapping` when @@ -146,7 +143,7 @@ object ConsistentHashingRouter { */ @SerialVersionUID(1L) case class ConsistentHashingRouter( - nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, + nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, val virtualNodesFactor: Int = 0, @@ -165,7 +162,7 @@ case class ConsistentHashingRouter( * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -227,7 +224,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] def virtualNodesFactor: Int diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 9ccf43fb58..8c3f059a40 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -5,18 +5,20 @@ package akka.routing import language.implicitConversions import language.postfixOps -import akka.actor._ -import scala.concurrent.duration._ -import akka.ConfigurationException -import akka.pattern.pipe -import com.typesafe.config.Config + +import scala.collection.immutable import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import scala.concurrent.duration._ +import akka.actor._ +import akka.ConfigurationException +import akka.dispatch.Dispatchers +import akka.pattern.pipe +import akka.japi.Util.immutableSeq +import com.typesafe.config.Config import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit import scala.concurrent.forkjoin.ThreadLocalRandom -import akka.dispatch.Dispatchers import scala.annotation.tailrec -import concurrent.ExecutionContext /** * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to @@ -50,7 +52,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo private val resizeCounter = new AtomicLong @volatile - private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute + private var _routees: immutable.IndexedSeq[ActorRef] = immutable.IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute def routees = _routees @volatile @@ -75,14 +77,11 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo * end of construction */ - def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { - case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil - case CurrentRoutees ⇒ - sender ! RouterRoutees(_routees) - Nil - case _ ⇒ - if (route.isDefinedAt(sender, message)) route(sender, message) - else Nil + def applyRoute(sender: ActorRef, message: Any): immutable.Iterable[Destination] = message match { + case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil + case CurrentRoutees ⇒ sender ! RouterRoutees(_routees); Nil + case msg if route.isDefinedAt(sender, msg) ⇒ route(sender, message) + case _ ⇒ Nil } /** @@ -91,7 +90,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo * Not thread safe, but intended to be called from protected points, such as * `RouterConfig.createRoute` and `Resizer.resize` */ - private[akka] def addRoutees(newRoutees: Iterable[ActorRef]): Unit = { + private[akka] def addRoutees(newRoutees: immutable.Iterable[ActorRef]): Unit = { _routees = _routees ++ newRoutees // subscribe to Terminated messages for all route destinations, to be handled by Router actor newRoutees foreach watch @@ -103,7 +102,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo * Not thread safe, but intended to be called from protected points, such as * `Resizer.resize` */ - private[akka] def removeRoutees(abandonedRoutees: Iterable[ActorRef]): Unit = { + private[akka] def removeRoutees(abandonedRoutees: immutable.Iterable[ActorRef]): Unit = { _routees = abandonedRoutees.foldLeft(_routees) { (xs, x) ⇒ unwatch(x); xs.filterNot(_ == x) } } @@ -195,7 +194,7 @@ trait RouterConfig { */ def withFallback(other: RouterConfig): RouterConfig = this - protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] = + protected def toAll(sender: ActorRef, routees: immutable.Iterable[ActorRef]): immutable.Iterable[Destination] = routees.map(Destination(sender, _)) /** @@ -207,7 +206,7 @@ trait RouterConfig { /** * Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early. */ - def verifyConfig(): Unit = {} + def verifyConfig(): Unit = () } @@ -226,7 +225,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * Not thread safe, but intended to be called from protected points, such as * `RouterConfig.createRoute` and `Resizer.resize`. */ - def registerRoutees(routees: Iterable[ActorRef]): Unit = routedCell.addRoutees(routees) + def registerRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.addRoutees(routees) /** * Adds the routees to the router. @@ -235,7 +234,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * `RouterConfig.createRoute` and `Resizer.resize`. * Java API. */ - def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(routees.asScala) + def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(immutableSeq(routees)) /** * Removes routees from the router. This method doesn't stop the routees. @@ -243,7 +242,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * Not thread safe, but intended to be called from protected points, such as * `Resizer.resize`. */ - def unregisterRoutees(routees: Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees) + def unregisterRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees) /** * Removes routees from the router. This method doesn't stop the routees. @@ -252,28 +251,25 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * `Resizer.resize`. * JAVA API */ - def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(routees.asScala) + def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(immutableSeq(routees)) /** * Looks up routes with specified paths and registers them. */ - def registerRouteesFor(paths: Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_))) + def registerRouteesFor(paths: immutable.Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_))) /** * Looks up routes with specified paths and registers them. * JAVA API */ - def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(paths.asScala) + def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(immutableSeq(paths)) /** * Creates new routees from specified `Props` and registers them. */ - def createRoutees(nrOfInstances: Int): Unit = { - if (nrOfInstances <= 0) throw new IllegalArgumentException( - "Must specify nrOfInstances or routees for [%s]" format context.self.path.toString) - else - registerRoutees(IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps))) - } + def createRoutees(nrOfInstances: Int): Unit = + if (nrOfInstances <= 0) throw new IllegalArgumentException("Must specify nrOfInstances or routees for [%s]" format context.self.path.toString) + else registerRoutees(immutable.IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps))) /** * Remove specified number of routees by unregister them @@ -296,7 +292,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi * Give concurrent messages a chance to be placed in mailbox before * sending PoisonPill. */ - protected def delayedStop(scheduler: Scheduler, abandon: Iterable[ActorRef], stopDelay: FiniteDuration): Unit = { + protected def delayedStop(scheduler: Scheduler, abandon: immutable.Iterable[ActorRef], stopDelay: FiniteDuration): Unit = { if (abandon.nonEmpty) { if (stopDelay <= Duration.Zero) { abandon foreach (_ ! PoisonPill) @@ -314,7 +310,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi /** * All routees of the router */ - def routees: IndexedSeq[ActorRef] = routedCell.routees + def routees: immutable.IndexedSeq[ActorRef] = routedCell.routees /** * All routees of the router @@ -335,7 +331,7 @@ abstract class CustomRouterConfig extends RouterConfig { val customRoute = createCustomRoute(routeeProvider) { - case (sender, message) ⇒ customRoute.destinationsFor(sender, message).asScala + case (sender, message) ⇒ customRoute.destinationsFor(sender, message) } } @@ -344,7 +340,13 @@ abstract class CustomRouterConfig extends RouterConfig { } trait CustomRoute { - def destinationsFor(sender: ActorRef, message: Any): java.lang.Iterable[Destination] + /** + * use akka.japi.Util.immutableSeq to convert a java.lang.Iterable to the return type needed for destinationsFor, + * or if you just want to return a single Destination, use akka.japi.Util.immutableSingletonSeq + * + * Java API + */ + def destinationsFor(sender: ActorRef, message: Any): immutable.Seq[Destination] } /** @@ -366,7 +368,7 @@ trait Router extends Actor { if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProvider)) finally ab.set(false) case Terminated(child) ⇒ - ref.removeRoutees(IndexedSeq(child)) + ref.removeRoutees(child :: Nil) if (ref.routees.isEmpty) context.stop(self) }: Receive) orElse routerReceive @@ -426,7 +428,7 @@ case object CurrentRoutees extends CurrentRoutees { * Message used to carry information about what routees the router is currently using. */ @SerialVersionUID(1L) -case class RouterRoutees(routees: Iterable[ActorRef]) +case class RouterRoutees(routees: immutable.Iterable[ActorRef]) /** * For every message sent to a router, its route determines a set of destinations, @@ -494,16 +496,14 @@ object RoundRobinRouter { /** * Creates a new RoundRobinRouter, routing to the specified routees */ - def apply(routees: Iterable[ActorRef]): RoundRobinRouter = + def apply(routees: immutable.Iterable[ActorRef]): RoundRobinRouter = new RoundRobinRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala) - } + def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = + apply(immutableSeq(routees)) } /** * A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort. @@ -547,7 +547,7 @@ object RoundRobinRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ @SerialVersionUID(1L) -case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, +case class RoundRobinRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with RoundRobinLike { @@ -564,7 +564,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -602,7 +602,7 @@ trait RoundRobinLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { @@ -622,7 +622,7 @@ trait RoundRobinLike { this: RouterConfig ⇒ case (sender, message) ⇒ message match { case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) - case msg ⇒ List(Destination(sender, getNext())) + case msg ⇒ Destination(sender, getNext()) :: Nil } } } @@ -632,15 +632,13 @@ object RandomRouter { /** * Creates a new RandomRouter, routing to the specified routees */ - def apply(routees: Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString)) + def apply(routees: immutable.Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala) - } + def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = + apply(immutableSeq(routees)) } /** * A Router that randomly selects one of the target connections to send a message to. @@ -684,7 +682,7 @@ object RandomRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ @SerialVersionUID(1L) -case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, +case class RandomRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with RandomLike { @@ -701,7 +699,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -738,7 +736,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, trait RandomLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { @@ -756,7 +754,7 @@ trait RandomLike { this: RouterConfig ⇒ case (sender, message) ⇒ message match { case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) - case msg ⇒ List(Destination(sender, getNext())) + case msg ⇒ Destination(sender, getNext()) :: Nil } } } @@ -766,16 +764,14 @@ object SmallestMailboxRouter { /** * Creates a new SmallestMailboxRouter, routing to the specified routees */ - def apply(routees: Iterable[ActorRef]): SmallestMailboxRouter = + def apply(routees: immutable.Iterable[ActorRef]): SmallestMailboxRouter = new SmallestMailboxRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala) - } + def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter = + apply(immutableSeq(routees)) } /** * A Router that tries to send to the non-suspended routee with fewest messages in mailbox. @@ -828,7 +824,7 @@ object SmallestMailboxRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ @SerialVersionUID(1L) -case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, +case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with SmallestMailboxLike { @@ -845,7 +841,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -882,7 +878,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin trait SmallestMailboxLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] /** * Returns true if the actor is currently processing a message. @@ -954,7 +950,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ // 4. An ActorRef with unknown mailbox size that isn't processing anything // 5. An ActorRef with a known mailbox size // 6. An ActorRef without any messages - @tailrec def getNext(targets: IndexedSeq[ActorRef] = routeeProvider.routees, + @tailrec def getNext(targets: immutable.IndexedSeq[ActorRef] = routeeProvider.routees, proposedTarget: ActorRef = routeeProvider.context.system.deadLetters, currentScore: Long = Long.MaxValue, at: Int = 0, @@ -985,7 +981,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒ case (sender, message) ⇒ message match { case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees) - case msg ⇒ List(Destination(sender, getNext())) + case msg ⇒ Destination(sender, getNext()) :: Nil } } } @@ -995,15 +991,13 @@ object BroadcastRouter { /** * Creates a new BroadcastRouter, routing to the specified routees */ - def apply(routees: Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString)) + def apply(routees: immutable.Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString)) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala) - } + def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = + apply(immutableSeq(routees)) } /** * A Router that uses broadcasts a message to all its connections. @@ -1047,7 +1041,7 @@ object BroadcastRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ @SerialVersionUID(1L) -case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None, +case class BroadcastRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) extends RouterConfig with BroadcastLike { @@ -1064,7 +1058,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -1102,7 +1096,7 @@ trait BroadcastLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] def createRoute(routeeProvider: RouteeProvider): Route = { if (resizer.isEmpty) { @@ -1120,16 +1114,14 @@ object ScatterGatherFirstCompletedRouter { /** * Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration */ - def apply(routees: Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = + def apply(routees: immutable.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within) /** * Java API to create router with the supplied 'routees' actors. */ - def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = { - import scala.collection.JavaConverters._ - apply(routees.asScala, within) - } + def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = + apply(immutableSeq(routees), within) } /** * Simple router that broadcasts the message to all routees, and replies with the first response. @@ -1175,7 +1167,7 @@ object ScatterGatherFirstCompletedRouter { * using `actorFor` in [[akka.actor.ActorRefProvider]] */ @SerialVersionUID(1L) -case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: FiniteDuration, +case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, within: FiniteDuration, override val resizer: Option[Resizer] = None, val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) @@ -1196,7 +1188,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = routeePaths.asScala, within = w) + def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = immutableSeq(routeePaths), within = w) /** * Constructor that sets the resizer to be used. @@ -1234,7 +1226,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ def nrOfInstances: Int - def routees: Iterable[String] + def routees: immutable.Iterable[String] def within: FiniteDuration @@ -1394,7 +1386,7 @@ case class DefaultResizer( * @param routees The current actor in the resizer * @return the number of routees by which the resizer should be adjusted (positive, negative or zero) */ - def capacity(routees: IndexedSeq[ActorRef]): Int = { + def capacity(routees: immutable.IndexedSeq[ActorRef]): Int = { val currentSize = routees.size val press = pressure(routees) val delta = filter(press, currentSize) @@ -1422,7 +1414,7 @@ case class DefaultResizer( * @param routees the current resizer of routees * @return number of busy routees, between 0 and routees.size */ - def pressure(routees: IndexedSeq[ActorRef]): Int = { + def pressure(routees: immutable.IndexedSeq[ActorRef]): Int = { routees count { case a: ActorRefWithCell ⇒ a.underlying match { diff --git a/akka-actor/src/main/scala/akka/routing/package.scala b/akka-actor/src/main/scala/akka/routing/package.scala index 0b40793861..76dc2f3104 100644 --- a/akka-actor/src/main/scala/akka/routing/package.scala +++ b/akka-actor/src/main/scala/akka/routing/package.scala @@ -4,10 +4,12 @@ package akka +import scala.collection.immutable + package object routing { /** * Routing logic, partial function from (sender, message) to a * set of destinations. */ - type Route = PartialFunction[(akka.actor.ActorRef, Any), Iterable[Destination]] + type Route = PartialFunction[(akka.actor.ActorRef, Any), immutable.Iterable[Destination]] } diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 1f78fbd3a7..071195c11e 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -5,8 +5,7 @@ package akka.serialization import com.typesafe.config.Config -import akka.AkkaException -import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess } +import akka.actor.{ Extension, ExtendedActorSystem, Address } import akka.event.Logging import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer @@ -28,17 +27,13 @@ object Serialization { val currentTransportAddress = new DynamicVariable[Address](null) class Settings(val config: Config) { + val Serializers: Map[String, String] = configToMap("akka.actor.serializers") + val SerializationBindings: Map[String, String] = configToMap("akka.actor.serialization-bindings") - import scala.collection.JavaConverters._ - import config._ - - val Serializers: Map[String, String] = configToMap(getConfig("akka.actor.serializers")) - - val SerializationBindings: Map[String, String] = configToMap(getConfig("akka.actor.serialization-bindings")) - - private def configToMap(cfg: Config): Map[String, String] = - cfg.root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) } - + private final def configToMap(path: String): Map[String, String] = { + import scala.collection.JavaConverters._ + config.getConfig(path).root.unwrapped.asScala.mapValues(_.toString).toMap + } } } @@ -63,16 +58,16 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * using the optional type hint to the Serializer and the optional ClassLoader ot load it into. * Returns either the resulting object or an Exception if one was thrown. */ - def deserialize(bytes: Array[Byte], - serializerId: Int, - clazz: Option[Class[_]]): Try[AnyRef] = Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz)) + def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]]): Try[AnyRef] = + Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz)) /** * Deserializes the given array of bytes using the specified type to look up what Serializer should be used. * You can specify an optional ClassLoader to load the object into. * Returns either the resulting object or an Exception if one was thrown. */ - def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] = Try(serializerFor(clazz).fromBinary(bytes, Some(clazz))) + def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] = + Try(serializerFor(clazz).fromBinary(bytes, Some(clazz))) /** * Returns the Serializer configured for the given object, returns the NullSerializer if it's null. @@ -96,8 +91,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { */ def serializerFor(clazz: Class[_]): Serializer = serializerMap.get(clazz) match { - case null ⇒ - // bindings are ordered from most specific to least specific + case null ⇒ // bindings are ordered from most specific to least specific def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean = possibilities.size == 1 || (possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) || diff --git a/akka-actor/src/main/scala/akka/util/Collections.scala b/akka-actor/src/main/scala/akka/util/Collections.scala new file mode 100644 index 0000000000..0ccbcd408c --- /dev/null +++ b/akka-actor/src/main/scala/akka/util/Collections.scala @@ -0,0 +1,54 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ + +package akka.util + +import scala.collection.immutable +import scala.annotation.tailrec + +/** + * INTERNAL API + */ +private[akka] object Collections { + + case object EmptyImmutableSeq extends immutable.Seq[Nothing] { + override final def iterator = Iterator.empty + override final def apply(idx: Int): Nothing = throw new java.lang.IndexOutOfBoundsException(idx.toString) + override final def length: Int = 0 + } + + abstract class PartialImmutableValuesIterable[From, To] extends immutable.Iterable[To] { + def isDefinedAt(from: From): Boolean + def apply(from: From): To + def valuesIterator: Iterator[From] + final def iterator: Iterator[To] = { + val superIterator = valuesIterator + new Iterator[To] { + private[this] var _next: To = _ + private[this] var _hasNext = false + + @tailrec override final def hasNext: Boolean = + if (!_hasNext && superIterator.hasNext) { // If we need and are able to look for the next value + val potentiallyNext = superIterator.next() + if (isDefinedAt(potentiallyNext)) { + _next = apply(potentiallyNext) + _hasNext = true + true + } else hasNext //Attempt to find the next + } else _hasNext // Return if we found one + + override final def next(): To = if (hasNext) { + val ret = _next + _next = null.asInstanceOf[To] // Mark as consumed (nice to the GC, don't leak the last returned value) + _hasNext = false // Mark as consumed (we need to look for the next value) + ret + } else throw new java.util.NoSuchElementException("next") + } + } + + override lazy val size: Int = iterator.size + override def foreach[C](f: To ⇒ C) = iterator foreach f + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/util/Convert.scala b/akka-actor/src/main/scala/akka/util/Convert.scala deleted file mode 100644 index 3fead7aef7..0000000000 --- a/akka-actor/src/main/scala/akka/util/Convert.scala +++ /dev/null @@ -1,45 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka.util -//FIXME DOCS! -object Convert { - - def intToBytes(value: Int): Array[Byte] = { - val bytes = Array.fill[Byte](4)(0) - bytes(0) = (value >>> 24).asInstanceOf[Byte] - bytes(1) = (value >>> 16).asInstanceOf[Byte] - bytes(2) = (value >>> 8).asInstanceOf[Byte] - bytes(3) = value.asInstanceOf[Byte] - bytes - } - - def bytesToInt(bytes: Array[Byte], offset: Int): Int = { - (0 until 4).foldLeft(0)((value, index) ⇒ value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8))) - } - - def longToBytes(value: Long): Array[Byte] = { - val writeBuffer = Array.fill[Byte](8)(0) - writeBuffer(0) = (value >>> 56).asInstanceOf[Byte] - writeBuffer(1) = (value >>> 48).asInstanceOf[Byte] - writeBuffer(2) = (value >>> 40).asInstanceOf[Byte] - writeBuffer(3) = (value >>> 32).asInstanceOf[Byte] - writeBuffer(4) = (value >>> 24).asInstanceOf[Byte] - writeBuffer(5) = (value >>> 16).asInstanceOf[Byte] - writeBuffer(6) = (value >>> 8).asInstanceOf[Byte] - writeBuffer(7) = (value >>> 0).asInstanceOf[Byte] - writeBuffer - } - - def bytesToLong(buf: Array[Byte]): Long = { - ((buf(0) & 0xFFL) << 56) | - ((buf(1) & 0xFFL) << 48) | - ((buf(2) & 0xFFL) << 40) | - ((buf(3) & 0xFFL) << 32) | - ((buf(4) & 0xFFL) << 24) | - ((buf(5) & 0xFFL) << 16) | - ((buf(6) & 0xFFL) << 8) | - ((buf(7) & 0xFFL) << 0) - } -} diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index ccaab901c3..83d8a40885 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -75,7 +75,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] = container get key match { case null ⇒ None - case set ⇒ set.iterator.asScala.find(f) + case set ⇒ set.iterator.asScala find f } /** diff --git a/akka-camel/src/main/scala/akka/camel/Camel.scala b/akka-camel/src/main/scala/akka/camel/Camel.scala index afb2d3c3d8..c72193becb 100644 --- a/akka-camel/src/main/scala/akka/camel/Camel.scala +++ b/akka-camel/src/main/scala/akka/camel/Camel.scala @@ -4,16 +4,15 @@ package akka.camel -import internal._ +import akka.camel.internal._ import akka.actor._ +import akka.ConfigurationException import org.apache.camel.ProducerTemplate import org.apache.camel.impl.DefaultCamelContext import org.apache.camel.model.RouteDefinition import com.typesafe.config.Config -import akka.ConfigurationException -import scala.concurrent.duration.Duration +import scala.concurrent.duration.{ Duration, FiniteDuration } import java.util.concurrent.TimeUnit._ -import scala.concurrent.duration.FiniteDuration /** * Camel trait encapsulates the underlying camel machinery. @@ -88,8 +87,8 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess final val StreamingCache: Boolean = config.getBoolean("akka.camel.streamingCache") final val Conversions: (String, RouteDefinition) ⇒ RouteDefinition = { - import scala.collection.JavaConverters.asScalaSetConverter val specifiedConversions = { + import scala.collection.JavaConverters.asScalaSetConverter val section = config.getConfig("akka.camel.conversions") section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey))) } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala index b8fa31fbc3..88c408e70c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterSettings.scala @@ -45,9 +45,7 @@ class ClusterSettings(val config: Config, val systemName: String) { require(n > 0, "failure-detector.monitored-by-nr-of-members must be > 0"); n } - final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { - case AddressFromURIString(addr) ⇒ addr - }.toIndexedSeq + final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { case AddressFromURIString(addr) ⇒ addr }.toIndexedSeq final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS) final val PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS) final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS) diff --git a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala index ef99ced2d8..ee6cd88a7d 100644 --- a/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala +++ b/akka-cluster/src/main/scala/akka/cluster/routing/ClusterRouterConfig.scala @@ -5,16 +5,13 @@ package akka.cluster.routing import java.lang.IllegalStateException import java.util.concurrent.atomic.AtomicInteger -import scala.collection.immutable.SortedSet +import scala.collection.immutable import com.typesafe.config.ConfigFactory import akka.ConfigurationException -import akka.actor.Actor import akka.actor.ActorContext import akka.actor.ActorRef -import akka.actor.ActorSystemImpl import akka.actor.Address import akka.actor.Deploy -import akka.actor.InternalActorRef import akka.actor.Props import akka.actor.SupervisorStrategy import akka.cluster.Cluster @@ -51,7 +48,7 @@ final case class ClusterRouterConfig(local: RouterConfig, settings: ClusterRoute // Intercept ClusterDomainEvent and route them to the ClusterRouterActor ({ - case (sender, message: ClusterDomainEvent) ⇒ Seq(Destination(sender, routeeProvider.context.self)) + case (sender, message: ClusterDomainEvent) ⇒ List(Destination(sender, routeeProvider.context.self)) }: Route) orElse localRoute } @@ -156,7 +153,7 @@ private[akka] class ClusterRouteeProvider( // need this counter as instance variable since Resizer may call createRoutees several times private val childNameCounter = new AtomicInteger - override def registerRouteesFor(paths: Iterable[String]): Unit = + override def registerRouteesFor(paths: immutable.Iterable[String]): Unit = throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]" format context.self.path.toString) @@ -183,7 +180,7 @@ private[akka] class ClusterRouteeProvider( context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false) } // must register each one, since registered routees are used in selectDeploymentTarget - registerRoutees(Some(ref)) + registerRoutees(List(ref)) // recursion until all created doCreateRoutees() @@ -222,27 +219,26 @@ private[akka] class ClusterRouteeProvider( case a ⇒ a } - private[routing] def availableNodes: SortedSet[Address] = { + private[routing] def availableNodes: immutable.SortedSet[Address] = { import Member.addressOrdering val currentNodes = nodes if (currentNodes.isEmpty && settings.allowLocalRoutees) //use my own node, cluster information not updated yet - SortedSet(cluster.selfAddress) + immutable.SortedSet(cluster.selfAddress) else currentNodes } @volatile - private[routing] var nodes: SortedSet[Address] = { + private[routing] var nodes: immutable.SortedSet[Address] = { import Member.addressOrdering cluster.readView.members.collect { case m if isAvailable(m) ⇒ m.address } } - private[routing] def isAvailable(m: Member): Boolean = { + private[routing] def isAvailable(m: Member): Boolean = m.status == MemberStatus.Up && (settings.allowLocalRoutees || m.address != cluster.selfAddress) - } } diff --git a/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java b/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java index 9ea2a332b1..4494bb0c51 100644 --- a/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/rst/java/code/docs/actor/FaultHandlingTestBase.java @@ -27,6 +27,7 @@ import akka.testkit.ErrorFilter; import akka.testkit.EventFilter; import akka.testkit.TestEvent; import static java.util.concurrent.TimeUnit.SECONDS; +import static akka.japi.Util.immutableSeq; import akka.japi.Function; import scala.Option; import scala.collection.JavaConverters; @@ -219,8 +220,7 @@ public class FaultHandlingTestBase { //#testkit public Seq seq(A... args) { - return JavaConverters.collectionAsScalaIterableConverter( - java.util.Arrays.asList(args)).asScala().toList(); + return immutableSeq(args); } //#testkit } diff --git a/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java b/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java index 73b8c5c639..239a3c318d 100644 --- a/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java +++ b/akka-docs/rst/java/code/docs/jrouting/CustomRouterDocTestBase.java @@ -180,16 +180,14 @@ public class CustomRouterDocTestBase { //#crRoutingLogic return new CustomRoute() { @Override - public Iterable destinationsFor(ActorRef sender, Object msg) { + public scala.collection.immutable.Seq destinationsFor(ActorRef sender, Object msg) { switch ((Message) msg) { case DemocratVote: case DemocratCountResult: - return Arrays.asList( - new Destination[] { new Destination(sender, democratActor) }); + return akka.japi.Util.immutableSingletonSeq(new Destination(sender, democratActor)); case RepublicanVote: case RepublicanCountResult: - return Arrays.asList( - new Destination[] { new Destination(sender, republicanActor) }); + return akka.japi.Util.immutableSingletonSeq(new Destination(sender, republicanActor)); default: throw new IllegalArgumentException("Unknown message: " + msg); } diff --git a/akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst b/akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst new file mode 100644 index 0000000000..80bdccd803 --- /dev/null +++ b/akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst @@ -0,0 +1,26 @@ +.. _migration-2.2: + +################################ + Migration Guide 2.1.x to 2.2.x +################################ + +The 2.2 release contains several structural changes that require some +simple, mechanical source-level changes in client code. + +When migrating from 1.3.x to 2.1.x you should first follow the instructions for +migrating `1.3.x to 2.0.x `_ and then :ref:`2.0.x to 2.1.x `. + +Immutable everywhere +==================== + +Akka has in 2.2 been refactored to require ``scala.collection.immutable`` data structures as much as possible, +this leads to fewer bugs and more opportunity for sharing data safely. + +==================================== ==================================== +Search Replace with +==================================== ==================================== +``akka.japi.Util.arrayToSeq`` ``akka.japi.Util.immutableSeq`` +==================================== ==================================== + +If you need to convert from Java to ``scala.collection.immutable.Seq`` or ``scala.collection.immutable.Iterable`` you should use ``akka.japi.Util.immutableSeq(…)``, +and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.]`` method. \ No newline at end of file diff --git a/akka-docs/rst/project/migration-guides.rst b/akka-docs/rst/project/migration-guides.rst index 79e2f7b8cc..5f464f3a08 100644 --- a/akka-docs/rst/project/migration-guides.rst +++ b/akka-docs/rst/project/migration-guides.rst @@ -8,3 +8,4 @@ Migration Guides migration-guide-1.3.x-2.0.x migration-guide-2.0.x-2.1.x + migration-guide-2.1.x-2.2.x diff --git a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala index b8c7ea24e5..1c70d03d7b 100644 --- a/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala +++ b/akka-osgi/src/test/scala/akka/osgi/PojoSRTestSupport.scala @@ -6,7 +6,6 @@ package akka.osgi import de.kalpatec.pojosr.framework.launch.{ BundleDescriptor, PojoServiceRegistryFactory, ClasspathScanner } import scala.collection.JavaConversions.seqAsJavaList -import scala.collection.JavaConversions.collectionAsScalaIterable import org.apache.commons.io.IOUtils.copy import org.osgi.framework._ @@ -138,12 +137,12 @@ class BundleDescriptorBuilder(name: String) { } def extractHeaders(file: File): HashMap[String, String] = { + import scala.collection.JavaConverters.iterableAsScalaIterableConverter val headers = new HashMap[String, String]() - val jis = new JarInputStream(new FileInputStream(file)) try { - for (entry ← jis.getManifest().getMainAttributes().entrySet()) - headers.put(entry.getKey().toString(), entry.getValue().toString()) + for (entry ← jis.getManifest.getMainAttributes.entrySet.asScala) + headers.put(entry.getKey.toString, entry.getValue.toString) } finally jis.close() headers diff --git a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala index e79ab4a1ee..8a076dd683 100644 --- a/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala +++ b/akka-remote-tests/src/main/scala/akka/remote/testkit/MultiNodeSpec.scala @@ -402,10 +402,8 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: } import scala.collection.JavaConverters._ ConfigFactory.parseString(deployString).root.asScala foreach { - case (key, value: ConfigObject) ⇒ - deployer.parseConfig(key, value.toConfig) foreach deployer.deploy - case (key, x) ⇒ - throw new IllegalArgumentException("key " + key + " must map to deployment section, not simple value " + x) + case (key, value: ConfigObject) ⇒ deployer.parseConfig(key, value.toConfig) foreach deployer.deploy + case (key, x) ⇒ throw new IllegalArgumentException(s"key $key must map to deployment section, not simple value $x") } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index fbc9c7b913..60c77fb4cc 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -6,8 +6,9 @@ package akka.remote import akka.actor._ import akka.routing._ import akka.remote.routing._ -import com.typesafe.config._ import akka.ConfigurationException +import akka.japi.Util.immutableSeq +import com.typesafe.config._ @SerialVersionUID(1L) case class RemoteScope(node: Address) extends Scope { @@ -22,9 +23,9 @@ private[akka] class RemoteDeployer(_settings: ActorSystem.Settings, _pm: Dynamic case d @ Some(deploy) ⇒ deploy.config.getString("remote") match { case AddressFromURIString(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.toIndexedSeq map (AddressFromURIString(_)) + case str if !str.isEmpty ⇒ throw new ConfigurationException("unparseable remote node name " + str) + case _ ⇒ + val nodes = immutableSeq(deploy.config.getStringList("target.nodes")).map(AddressFromURIString(_)) if (nodes.isEmpty || deploy.routerConfig == NoRouter) d else Some(deploy.copy(routerConfig = RemoteRouterConfig(deploy.routerConfig, nodes))) } 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 7c1ca16540..5852f7a3ca 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/Settings.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/Settings.scala @@ -8,7 +8,7 @@ import scala.concurrent.duration.Duration import java.util.concurrent.TimeUnit._ import java.net.InetAddress import akka.ConfigurationException -import scala.collection.JavaConverters.iterableAsScalaIterableConverter +import akka.japi.Util.immutableSeq import scala.concurrent.duration.FiniteDuration import akka.dispatch.ThreadPoolConfig @@ -89,42 +89,19 @@ private[akka] class NettySettings(config: Config, val systemName: String) { case sz ⇒ sz } - val SSLKeyStore = getString("ssl.key-store") match { - case "" ⇒ None - case keyStore ⇒ Some(keyStore) - } + val SSLKeyStore = Option(getString("ssl.key-store")).filter(_.length > 0) + val SSLTrustStore = Option(getString("ssl.trust-store")).filter(_.length > 0) + val SSLKeyStorePassword = Option(getString("ssl.key-store-password")).filter(_.length > 0) - val SSLTrustStore = getString("ssl.trust-store") match { - case "" ⇒ None - case trustStore ⇒ Some(trustStore) - } + val SSLTrustStorePassword = Option(getString("ssl.trust-store-password")).filter(_.length > 0) - val SSLKeyStorePassword = getString("ssl.key-store-password") match { - case "" ⇒ None - case password ⇒ Some(password) - } + val SSLEnabledAlgorithms = immutableSeq(getStringList("ssl.enabled-algorithms")).to[Set] - val SSLTrustStorePassword = getString("ssl.trust-store-password") match { - case "" ⇒ None - case password ⇒ Some(password) - } + val SSLProtocol = Option(getString("ssl.protocol")).filter(_.length > 0) - val SSLEnabledAlgorithms = iterableAsScalaIterableConverter(getStringList("ssl.enabled-algorithms")).asScala.toSet[String] + val SSLRandomSource = Option(getString("ssl.sha1prng-random-source")).filter(_.length > 0) - val SSLProtocol = getString("ssl.protocol") match { - case "" ⇒ None - case protocol ⇒ Some(protocol) - } - - val SSLRandomSource = getString("ssl.sha1prng-random-source") match { - case "" ⇒ None - case path ⇒ Some(path) - } - - val SSLRandomNumberGenerator = getString("ssl.random-number-generator") match { - case "" ⇒ None - case rng ⇒ Some(rng) - } + val SSLRandomNumberGenerator = Option(getString("ssl.random-number-generator")).filter(_.length > 0) val EnableSSL = { val enableSSL = getBoolean("ssl.enable") diff --git a/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala index 8a4e3bce7c..369d8b0c7b 100644 --- a/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/remote/routing/RemoteRouterConfig.scala @@ -6,19 +6,17 @@ package akka.remote.routing import akka.routing.{ Route, Router, RouterConfig, RouteeProvider, Resizer } import com.typesafe.config.ConfigFactory import akka.actor.ActorContext -import akka.actor.ActorRef import akka.actor.Deploy -import akka.actor.InternalActorRef import akka.actor.Props -import akka.ConfigurationException -import akka.remote.RemoteScope -import akka.actor.AddressFromURIString import akka.actor.SupervisorStrategy import akka.actor.Address -import scala.collection.JavaConverters._ +import akka.actor.ActorCell +import akka.ConfigurationException +import akka.remote.RemoteScope +import akka.japi.Util.immutableSeq +import scala.collection.immutable import java.util.concurrent.atomic.AtomicInteger import java.lang.IllegalStateException -import akka.actor.ActorCell /** * [[akka.routing.RouterConfig]] implementation for remote deployment on defined @@ -29,7 +27,7 @@ import akka.actor.ActorCell @SerialVersionUID(1L) final case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address]) extends RouterConfig { - def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, nodes.asScala) + def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, immutableSeq(nodes)) def this(local: RouterConfig, nodes: Array[Address]) = this(local, nodes: Iterable[Address]) override def createRouteeProvider(context: ActorContext, routeeProps: Props) = @@ -64,20 +62,20 @@ final case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address final class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _routeeProps: Props, _resizer: Option[Resizer]) extends RouteeProvider(_context, _routeeProps, _resizer) { - if (nodes.isEmpty) throw new ConfigurationException("Must specify list of remote target.nodes for [%s]" - format context.self.path.toString) + if (nodes.isEmpty) + throw new ConfigurationException("Must specify list of remote target.nodes for [%s]" format context.self.path.toString) // need this iterator as instance variable since Resizer may call createRoutees several times private val nodeAddressIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator // need this counter as instance variable since Resizer may call createRoutees several times private val childNameCounter = new AtomicInteger - override def registerRouteesFor(paths: Iterable[String]): Unit = + override def registerRouteesFor(paths: immutable.Iterable[String]): Unit = throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]" format context.self.path.toString) override def createRoutees(nrOfInstances: Int): Unit = { - val refs = IndexedSeq.fill(nrOfInstances) { + val refs = immutable.IndexedSeq.fill(nrOfInstances) { val name = "c" + childNameCounter.incrementAndGet val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig, scope = RemoteScope(nodeAddressIter.next)) diff --git a/akka-testkit/src/main/java/akka/testkit/JavaTestKit.java b/akka-testkit/src/main/java/akka/testkit/JavaTestKit.java index 8fbb628d7c..835fd1939a 100644 --- a/akka-testkit/src/main/java/akka/testkit/JavaTestKit.java +++ b/akka-testkit/src/main/java/akka/testkit/JavaTestKit.java @@ -184,31 +184,31 @@ public class JavaTestKit { } public Object expectMsgAnyOf(Object... msgs) { - return p.expectMsgAnyOf(Util.arrayToSeq(msgs)); + return p.expectMsgAnyOf(Util.immutableSeq(msgs)); } public Object expectMsgAnyOf(FiniteDuration max, Object... msgs) { - return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs)); + return p.expectMsgAnyOf(max, Util.immutableSeq(msgs)); } public Object[] expectMsgAllOf(Object... msgs) { - return (Object[]) p.expectMsgAllOf(Util.arrayToSeq(msgs)).toArray( + return (Object[]) p.expectMsgAllOf(Util.immutableSeq(msgs)).toArray( Util.classTag(Object.class)); } public Object[] expectMsgAllOf(FiniteDuration max, Object... msgs) { - return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray( + return (Object[]) p.expectMsgAllOf(max, Util.immutableSeq(msgs)).toArray( Util.classTag(Object.class)); } @SuppressWarnings("unchecked") public T expectMsgAnyClassOf(Class... classes) { - final Object result = p.expectMsgAnyClassOf(Util.arrayToSeq(classes)); + final Object result = p.expectMsgAnyClassOf(Util.immutableSeq(classes)); return (T) result; } public Object expectMsgAnyClassOf(FiniteDuration max, Class... classes) { - return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes)); + return p.expectMsgAnyClassOf(max, Util.immutableSeq(classes)); } public void expectNoMsg() { diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index c46f15a26a..dfcd7b9dd2 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -6,7 +6,6 @@ package akka.testkit import language.existentials import scala.util.matching.Regex -import scala.collection.JavaConverters import scala.collection.immutable import scala.concurrent.duration.Duration import scala.reflect.ClassTag @@ -14,8 +13,9 @@ import akka.actor.{ DeadLetter, ActorSystem, Terminated, UnhandledMessage } import akka.dispatch.{ SystemMessage, Terminate } import akka.event.Logging.{ Warning, LogEvent, InitializeLogger, Info, Error, Debug, LoggerInitialized } import akka.event.Logging -import java.lang.{ Iterable ⇒ JIterable } import akka.actor.NoSerializationVerificationNeeded +import akka.japi.Util.immutableSeq +import java.lang.{ Iterable ⇒ JIterable } /** * Implementation helpers of the EventFilter facilities: send `Mute` @@ -45,7 +45,7 @@ object TestEvent { /** * Java API */ - def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.to[immutable.Seq]) + def this(filters: JIterable[EventFilter]) = this(immutableSeq(filters)) } object UnMute { def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.to[immutable.Seq]) @@ -54,7 +54,7 @@ object TestEvent { /** * Java API */ - def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.to[immutable.Seq]) + def this(filters: JIterable[EventFilter]) = this(immutableSeq(filters)) } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 27323787aa..f73a659c46 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -25,7 +25,7 @@ import akka.testkit.ErrorFilter; import akka.testkit.TestEvent; import akka.util.Timeout; -import java.util.Arrays; +import static akka.japi.Util.immutableSeq; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -110,6 +110,6 @@ public class UntypedCoordinatedIncrementTest { } public Seq seq(A... args) { - return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toList(); + return immutableSeq(args); } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 3b841e300b..ade645dfd8 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -25,7 +25,7 @@ import akka.testkit.ErrorFilter; import akka.testkit.TestEvent; import akka.util.Timeout; -import java.util.Arrays; +import static akka.japi.Util.immutableSeq; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -118,8 +118,6 @@ public class UntypedTransactorTest { } public Seq seq(A... args) { - return JavaConverters - .collectionAsScalaIterableConverter(Arrays.asList(args)).asScala() - .toList(); + return immutableSeq(args); } } From 69505855fdf87c13c4786ab118b4a5ae359363c9 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 14 Nov 2012 16:19:54 +0100 Subject: [PATCH 36/62] make HashedWheelTimer tolerate wrap-arounds, see #2686 --- .../test/scala/akka/util/DurationSpec.scala | 23 ++++++++++++++++++- .../akka/util/internal/HashedWheelTimer.java | 22 +++++++++++++----- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala index d5c020be49..ca285274aa 100644 --- a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala @@ -8,10 +8,31 @@ import language.postfixOps import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import scala.concurrent.duration._ +import scala.concurrent.Await import java.util.concurrent.TimeUnit._ +import akka.testkit.AkkaSpec +import akka.testkit.TestLatch +import java.util.concurrent.TimeoutException +import akka.testkit.LongRunningTest -class DurationSpec extends WordSpec with MustMatchers { +class DurationSpec extends AkkaSpec { + + "A HashedWheelTimer" must { + + "not mess up long timeouts" taggedAs LongRunningTest in { + val longish = Long.MaxValue.nanos + val barrier = TestLatch() + import system.dispatcher + val job = system.scheduler.scheduleOnce(longish)(barrier.countDown()) + intercept[TimeoutException] { + // this used to fire after 46 seconds due to wrap-around + Await.ready(barrier, 90 seconds) + } + job.cancel() + } + + } "Duration" must { diff --git a/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java index fc50854ab7..e95ff9ad95 100644 --- a/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java +++ b/akka-actor/src/main/java/akka/util/internal/HashedWheelTimer.java @@ -263,8 +263,11 @@ public class HashedWheelTimer implements Timer { void scheduleTimeout(HashedWheelTimeout timeout, long delay) { // Prepare the required parameters to schedule the timeout object. - final long relativeIndex = Math.max(1, (delay + tickDuration - 1) / tickDuration); // If relative index < 1 then it should be 1 - + long relativeIndex = (delay + tickDuration - 1) / tickDuration; + // if the previous line had an overflow going on, then we’ll just schedule this timeout + // one tick early; that shouldn’t matter since we’re talking 270 years here + if (relativeIndex < 0) relativeIndex = delay / tickDuration; + if (relativeIndex == 0) relativeIndex = 1; final long remainingRounds = relativeIndex / wheel.length; // Add the timeout to the wheel. @@ -304,7 +307,7 @@ public class HashedWheelTimer implements Timer { while (!shutdown()) { final long deadline = waitForNextTick(); - if (deadline > 0) + if (deadline > Long.MIN_VALUE) notifyExpiredTimeouts(fetchExpiredTimeouts(deadline)); } } @@ -332,7 +335,7 @@ public class HashedWheelTimer implements Timer { HashedWheelTimeout timeout = i.next(); if (timeout.remainingRounds <= 0) { i.remove(); - if (timeout.deadline <= deadline) { + if (timeout.deadline - deadline <= 0) { expiredTimeouts.add(timeout); } else { // Handle the case where the timeout is put into a wrong @@ -368,6 +371,12 @@ public class HashedWheelTimer implements Timer { expiredTimeouts.clear(); } + /** + * calculate goal nanoTime from startTime and current tick number, + * then wait until that goal has been reached. + * + * @return Long.MIN_VALUE if received a shutdown request, current time otherwise (with Long.MIN_VALUE changed by +1) + */ private long waitForNextTick() { long deadline = startTime + tickDuration * tick; @@ -378,7 +387,8 @@ public class HashedWheelTimer implements Timer { if (sleepTimeMs <= 0) { tick += 1; - return currentTime; + if (currentTime == Long.MIN_VALUE) return -Long.MAX_VALUE; + else return currentTime; } // Check if we run on windows, as if thats the case we will need @@ -394,7 +404,7 @@ public class HashedWheelTimer implements Timer { Thread.sleep(sleepTimeMs); } catch (InterruptedException e) { if (shutdown()) { - return -1; + return Long.MIN_VALUE; } } } From b1d96c87943486f0a29ad283a8a385b8efc5abc0 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 15 Nov 2012 00:46:13 +0100 Subject: [PATCH 37/62] Preventing the TimerBasedThrottlerSpec printing to the console --- .../scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala index be077489ee..cf13f42a24 100644 --- a/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala +++ b/akka-contrib/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala @@ -36,7 +36,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp } "A throttler" must { - + def println(a: Any) = () "must pass the ScalaDoc class documentation example program" in { //#demo-code // A simple actor that prints whatever it receives From 5569cd676ad4942510098feb49fa0ec862bf081c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Wed, 7 Nov 2012 11:51:59 +0100 Subject: [PATCH 38/62] Release script instructions added. --- project/scripts/release | 85 +++++++++++++++++++++++++++++++++++++++-- 1 file changed, 81 insertions(+), 4 deletions(-) diff --git a/project/scripts/release b/project/scripts/release index a803277959..c660aaeeda 100755 --- a/project/scripts/release +++ b/project/scripts/release @@ -2,11 +2,88 @@ # # Release script for Akka. # -# To run this script you need access to repo.akka.io and github.com/akka/akka. +# Prerequisites and Installation Instructions # -# To always use the akkarepo user with repo.akka.io add the following to .ssh/config: -# Host repo.akka.io -# User akkarepo +# 1) You must be able to sign the artifacts with PGP +# +# 1.1) If you don't have PGP and a PGP key +# On OS X from othe command line: +# shell> brew install gnupg +# shell> gpg --gen-key +# +# Default values for the key type and 2048 bits is OK. +# Make sure to use the email address that you will use later to register +# with Sonatype. +# +# 1.2) Install the sbt-pgp plugin from http://www.scala-sbt.org/sbt-pgp/ +# The plugin will try to use the default key stored in ~/.gnupg/pubring.gpg +# and ~/.gnupg/secring.gpg. +# +# 1.3) Check that signing works +# From inside sbt do the following +# sbt> publish-local +# It should should ask you for your pass phrase, and create .asc files for +# all artifacts +# +# 1.4) Publish your key to a server that Sonatype use +# From the command line: +# shell> gpg --keyserver hkp://pool.sks-keyservers.net/ --send-keys +# To find out your key id do this from the command line: +# shell> gpg --list-keys +# pub 2048/ ... +# +# 2) You must have publishing rights to oss.sonatype.org +# +# 2.1) Register with oss.sonatype.org by only following the instructions under +# sign up here https://docs.sonatype.org/display/Repository/Sonatype+OSS+Maven+Repository+Usage+Guide +# Use the same email address as you used for the pgp key. +# +# 2.2) Ask Jonas who is the original creator of this ticket https://issues.sonatype.org/browse/OSSRH-3097 +# to add a comment that says that your username (not your full name) should +# have publish access to that project. There is manual administration of +# the ticket at Sonatype, so it could take a little while. +# +# 2.3) Add your credentials to sbt by adding a global.sbt file in your sbt home +# directory containing the following. +# credentials += Credentials("Sonatype Nexus Repository Manager", +# "oss.sonatype.org", +# "", +# "") +# +# 3) You must have publishing rights to scalasbt.artifactoryonline.com +# +# 3.1) Politely ask the Q-branch to create a user for you +# +# 3.2) Add your credentials to sbt by adding this to your global.sbt file +# credentials += Credentials("Artifactory Realm", +# "scalasbt.artifactoryonline.com", +# "", +# "") +# The encrypted password is available in your profile here +# http://scalasbt.artifactoryonline.com/scalasbt/webapp/profile.html +# +# 4) You must have access to repo.akka.io +# +# 4.1) Ask someone in the team for login information for the akkarepo user. +# +# 4.2) Install your public ssh key to avoid typing in your password. +# From the command line: +# shell> cat ~/.ssh/id_rsa.pub | ssh akkarepo@repo.akka.io "cat >> ~/.ssh/authorized_keys" +# +# 5) Have access to github.com/akka/akka. This should be a given. +# +# Now you should be all set to run the script +# +# Run the script in two stages. +# First a dry run: +# shell> project/scripts/release --dry-run +# And if all goes well a real run: +# shell> project/scripts/release +# +# The sbt plugin is published directly to scalasbt.artifactoryonline.com, but the +# artifacts published to oss.sonatype.org needs to be released by following the +# instructions under release here +# https://docs.sonatype.org/display/Repository/Sonatype+OSS+Maven+Repository+Usage+Guide # defaults declare -r default_server="akkarepo@repo.akka.io" From c33bd9fb9e719b0fd01c9cc3fc00361ed7dbc742 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Wed, 7 Nov 2012 09:44:02 +0100 Subject: [PATCH 39/62] Building with 2.10.0-RC2 and all dependencies that are published. --- .../cluster/ClusterMetricsCollector.scala | 20 +++++++++---------- project/AkkaBuild.scala | 12 ++++++----- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala index 7ed3699035..a3abd94316 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala @@ -9,8 +9,8 @@ import scala.concurrent.duration._ import scala.collection.immutable.{ SortedSet, Map } import scala.concurrent.forkjoin.ThreadLocalRandom import scala.util.{ Try, Success, Failure } -import scala.math.ScalaNumericConversions -import scala.runtime.{ RichLong, RichDouble, RichInt } +import scala.math.ScalaNumericAnyConversions +import runtime.{ ScalaNumberProxy, RichLong, RichDouble, RichInt } import akka.actor._ import akka.event.LoggingAdapter @@ -238,7 +238,7 @@ private[cluster] case class MetricsGossipEnvelope(from: Address, gossip: Metrics * * @param startTime the time of initial sampling for this data stream */ -private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions, startTime: Long, timestamp: Long) +private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericAnyConversions, startTime: Long, timestamp: Long) extends ClusterMessage with MetricNumericConverter { /** @@ -249,13 +249,13 @@ private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions /** * Calculates the exponentially weighted moving average for a given monitored data set. - * The datam can be too large to fit into an int or long, thus we use ScalaNumericConversions, + * The datam can be too large to fit into an int or long, thus we use ScalaNumericAnyConversions, * and defer to BigInt or BigDecimal. * * @param xn the new data point * @return an new [[akka.cluster.DataStream]] with the updated yn and timestamp */ - def :+(xn: ScalaNumericConversions): DataStream = convert(xn) fold ( + def :+(xn: ScalaNumericAnyConversions): DataStream = convert(xn) fold ( nl ⇒ copy(ewma = BigInt(α * nl + 1 - α * ewma.longValue()), timestamp = newTimestamp), nd ⇒ copy(ewma = BigDecimal(α * nd + 1 - α * ewma.doubleValue()), timestamp = newTimestamp)) @@ -273,7 +273,7 @@ private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions */ private[cluster] object DataStream { - def apply(decay: Int, data: ScalaNumericConversions): Option[DataStream] = if (decay > 0) + def apply(decay: Int, data: ScalaNumericAnyConversions): Option[DataStream] = if (decay > 0) Some(DataStream(decay, data, newTimestamp, newTimestamp)) else None } @@ -288,7 +288,7 @@ private[cluster] object DataStream { * @param average the data stream of the metric value, for trending over time. Metrics that are already * averages (e.g. system load average) or finite (e.g. as total cores), are not trended. */ -private[cluster] case class Metric(name: String, value: Option[ScalaNumericConversions], average: Option[DataStream]) +private[cluster] case class Metric(name: String, value: Option[ScalaNumericAnyConversions], average: Option[DataStream]) extends ClusterMessage with MetricNumericConverter { /** @@ -352,7 +352,7 @@ private[cluster] object Metric extends MetricNumericConverter { * or defined for the OS (JMX). If undefined we set the value option to None and do not modify * the latest sampled metric to avoid skewing the statistical trend. */ - def apply(name: String, value: Option[ScalaNumericConversions]): Metric = value match { + def apply(name: String, value: Option[ScalaNumericAnyConversions]): Metric = value match { case Some(v) if defined(v) ⇒ Metric(name, value, None) case _ ⇒ Metric(name, None, None) } @@ -409,13 +409,13 @@ private[cluster] trait MetricNumericConverter { *
  • JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned
  • *
  • SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)
*/ - def defined(value: ScalaNumericConversions): Boolean = + def defined(value: ScalaNumericAnyConversions): Boolean = convert(value) fold (a ⇒ value.underlying != -1, b ⇒ !(b.isNaN || b.isInfinite)) /** * May involve rounding or truncation. */ - def convert(from: ScalaNumericConversions): Either[Long, Double] = from match { + def convert(from: ScalaNumericAnyConversions): Either[Long, Double] = from match { case n: BigInt ⇒ Left(n.longValue()) case n: BigDecimal ⇒ Right(n.doubleValue()) case n: RichInt ⇒ Left(n.abs) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 8c1a4a6335..2b4f1f93f0 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -30,7 +30,8 @@ object AkkaBuild extends Build { organization := "com.typesafe.akka", version := "2.2-SNAPSHOT", // FIXME: use 2.10.0 for final - scalaVersion := System.getProperty("akka.scalaVersion", "2.10.0-RC1") + scalaVersion := System.getProperty("akka.scalaVersion", "2.10.0-RC2"), + scalaBinaryVersion <<= scalaVersion.identity // FIXME remove for 2.10.0 final ) lazy val akka = Project( @@ -221,7 +222,6 @@ object AkkaBuild extends Build { base = file("akka-zeromq"), dependencies = Seq(actor, testkit % "test;test->test"), settings = defaultSettings ++ OSGi.zeroMQ ++ Seq( - resolvers += "sonatype" at "http://oss.sonatype.org/content/repositories/releases", libraryDependencies ++= Dependencies.zeroMQ, previousArtifact := akkaPreviousArtifact("akka-zeromq") ) @@ -406,7 +406,9 @@ object AkkaBuild extends Build { shellPrompt := { s => Project.extract(s).currentProject.id + " > " } ) - lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings + lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings ++ Seq( + resolvers += "Sonatype Release" at "https://oss.sonatype.org/content/repositories/releases" // FIXME remove after staging RC2 + ) lazy val parentSettings = baseSettings ++ Seq( publishArtifact in Compile := false @@ -664,7 +666,7 @@ object Dependencies { val scalaStm = "org.scala-stm" % "scala-stm" % "0.6" cross CrossVersion.full // Modified BSD (Scala) val slf4jApi = "org.slf4j" % "slf4j-api" % "1.7.2" // MIT - val zeroMQClient = "org.zeromq" % "zeromq-scala-binding_2.10" % "0.0.6-RC1" // ApacheV2 + val zeroMQClient = "org.zeromq" % "zeromq-scala-binding" % "0.0.6" cross CrossVersion.full // ApacheV2 val uncommonsMath = "org.uncommons.maths" % "uncommons-maths" % "1.2.2a" exclude("jfree", "jcommon") exclude("jfree", "jfreechart") // ApacheV2 val ariesBlueprint = "org.apache.aries.blueprint" % "org.apache.aries.blueprint" % "0.3.2" // ApacheV2 val osgiCore = "org.osgi" % "org.osgi.core" % "4.2.0" // ApacheV2 @@ -681,7 +683,7 @@ object Dependencies { val junit = "junit" % "junit" % "4.10" % "test" // Common Public License 1.0 val logback = "ch.qos.logback" % "logback-classic" % "1.0.7" % "test" // EPL 1.0 / LGPL 2.1 val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" // MIT - val scalatest = "org.scalatest" % "scalatest" % "1.8" % "test" cross CrossVersion.full // ApacheV2 + val scalatest = "org.scalatest" % "scalatest" % "1.8-B2" % "test" cross CrossVersion.full // ApacheV2 val scalacheck = "org.scalacheck" % "scalacheck" % "1.10.0" % "test" cross CrossVersion.full // New BSD val ariesProxy = "org.apache.aries.proxy" % "org.apache.aries.proxy.impl" % "0.3" % "test" // ApacheV2 val pojosr = "com.googlecode.pojosr" % "de.kalpatec.pojosr.framework" % "0.1.4" % "test" // ApacheV2 From c2bf33afbf8d501c93c5c58daebd5eee793472b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Thu, 8 Nov 2012 09:00:31 +0100 Subject: [PATCH 40/62] Update doc with correct version of ScalaTest --- akka-docs/rst/dev/multi-jvm-testing.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-docs/rst/dev/multi-jvm-testing.rst b/akka-docs/rst/dev/multi-jvm-testing.rst index 92a4e9c6b1..bee19cbe71 100644 --- a/akka-docs/rst/dev/multi-jvm-testing.rst +++ b/akka-docs/rst/dev/multi-jvm-testing.rst @@ -72,7 +72,7 @@ Here is an example Build.scala file that uses the MultiJvm plugin: "test" cross CrossVersion.full, "com.typesafe.akka" %% "akka-remote-tests-experimental" % "@version@" % "test" cross CrossVersion.full, - "org.scalatest" %% "scalatest" % "1.8" % "test" cross CrossVersion.full, + "org.scalatest" %% "scalatest" % "1.8-B2" % "test" cross CrossVersion.full, "junit" % "junit" % "4.5" % "test" ) } From 21c55a8387abc6739cc964667212d64e1b5b4a26 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Bj=C3=B6rn=20Antonsson?= Date: Fri, 9 Nov 2012 08:00:30 +0100 Subject: [PATCH 41/62] Cleaning up after 2.1.0-RC2 staging --- project/AkkaBuild.scala | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 2b4f1f93f0..ef67a662be 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -30,8 +30,7 @@ object AkkaBuild extends Build { organization := "com.typesafe.akka", version := "2.2-SNAPSHOT", // FIXME: use 2.10.0 for final - scalaVersion := System.getProperty("akka.scalaVersion", "2.10.0-RC2"), - scalaBinaryVersion <<= scalaVersion.identity // FIXME remove for 2.10.0 final + scalaVersion := System.getProperty("akka.scalaVersion", "2.10.0-RC2") ) lazy val akka = Project( @@ -406,9 +405,7 @@ object AkkaBuild extends Build { shellPrompt := { s => Project.extract(s).currentProject.id + " > " } ) - lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings ++ Seq( - resolvers += "Sonatype Release" at "https://oss.sonatype.org/content/repositories/releases" // FIXME remove after staging RC2 - ) + lazy val baseSettings = Defaults.defaultSettings ++ Publish.settings lazy val parentSettings = baseSettings ++ Seq( publishArtifact in Compile := false From 524bb65e0fe6354383b13d5819cea24476e2f1a0 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 15 Nov 2012 08:28:35 +0100 Subject: [PATCH 42/62] Increase within timeouts in cluster sample tests, see #2699 --- .../sample/cluster/stats/StatsSampleSingleMasterSpec.scala | 2 +- .../scala/sample/cluster/stats/StatsSampleSpec.scala | 6 +++--- .../sample/cluster/stats/japi/StatsSampleJapiSpec.scala | 6 +++--- .../stats/japi/StatsSampleSingleMasterJapiSpec.scala | 2 +- .../cluster/transformation/TransformationSampleSpec.scala | 2 +- .../transformation/japi/TransformationSampleJapiSpec.scala | 2 +- 6 files changed, 10 insertions(+), 10 deletions(-) diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala index ad678d377f..7acd2ff202 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala @@ -67,7 +67,7 @@ abstract class StatsSampleSingleMasterSpec extends MultiNodeSpec(StatsSampleSing override def afterAll() = multiNodeSpecAfterAll() "The stats sample with single master" must { - "illustrate how to startup cluster" in within(10 seconds) { + "illustrate how to startup cluster" in within(15 seconds) { Cluster(system).subscribe(testActor, classOf[MemberUp]) expectMsgClass(classOf[CurrentClusterState]) diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala index 64c2e2994d..b9eeef3159 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala @@ -76,7 +76,7 @@ abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpecConfig) "The stats sample" must { //#startup-cluster - "illustrate how to startup cluster" in within(10 seconds) { + "illustrate how to startup cluster" in within(15 seconds) { Cluster(system).subscribe(testActor, classOf[MemberUp]) expectMsgClass(classOf[CurrentClusterState]) @@ -106,7 +106,7 @@ abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpecConfig) //#test-statsService - "show usage of the statsService from one node" in within(5 seconds) { + "show usage of the statsService from one node" in within(15 seconds) { runOn(second) { val service = system.actorFor(node(third) / "user" / "statsService") service ! StatsJob("this is the text that will be analyzed") @@ -120,7 +120,7 @@ abstract class StatsSampleSpec extends MultiNodeSpec(StatsSampleSpecConfig) } //#test-statsService - "show usage of the statsService from all nodes" in within(5 seconds) { + "show usage of the statsService from all nodes" in within(15 seconds) { val service = system.actorFor(node(third) / "user" / "statsService") service ! StatsJob("this is the text that will be analyzed") val meanWordLength = expectMsgPF() { diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala index 0712305b4b..1f4624674b 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleJapiSpec.scala @@ -65,7 +65,7 @@ abstract class StatsSampleJapiSpec extends MultiNodeSpec(StatsSampleJapiSpecConf "The japi stats sample" must { - "illustrate how to startup cluster" in within(10 seconds) { + "illustrate how to startup cluster" in within(15 seconds) { Cluster(system).subscribe(testActor, classOf[MemberUp]) expectMsgClass(classOf[CurrentClusterState]) @@ -89,7 +89,7 @@ abstract class StatsSampleJapiSpec extends MultiNodeSpec(StatsSampleJapiSpecConf } - "show usage of the statsService from one node" in within(5 seconds) { + "show usage of the statsService from one node" in within(15 seconds) { runOn(second) { val service = system.actorFor(node(third) / "user" / "statsService") service ! new StatsJob("this is the text that will be analyzed") @@ -103,7 +103,7 @@ abstract class StatsSampleJapiSpec extends MultiNodeSpec(StatsSampleJapiSpecConf } //#test-statsService - "show usage of the statsService from all nodes" in within(5 seconds) { + "show usage of the statsService from all nodes" in within(15 seconds) { val service = system.actorFor(node(third) / "user" / "statsService") service ! new StatsJob("this is the text that will be analyzed") val meanWordLength = expectMsgPF() { diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala index 7a5275a7f8..9299e007c4 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/japi/StatsSampleSingleMasterJapiSpec.scala @@ -66,7 +66,7 @@ abstract class StatsSampleSingleMasterJapiSpec extends MultiNodeSpec(StatsSample override def afterAll() = multiNodeSpecAfterAll() "The japi stats sample with single master" must { - "illustrate how to startup cluster" in within(10 seconds) { + "illustrate how to startup cluster" in within(15 seconds) { Cluster(system).subscribe(testActor, classOf[MemberUp]) expectMsgClass(classOf[CurrentClusterState]) diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala index 131997730f..8129bad77e 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/TransformationSampleSpec.scala @@ -52,7 +52,7 @@ abstract class TransformationSampleSpec extends MultiNodeSpec(TransformationSamp override def afterAll() = multiNodeSpecAfterAll() "The transformation sample" must { - "illustrate how to start first frontend" in { + "illustrate how to start first frontend" in within(15 seconds) { runOn(frontend1) { // this will only run on the 'first' node Cluster(system) join node(frontend1).address diff --git a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala index d7d328f81d..a37458a129 100644 --- a/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala +++ b/akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/transformation/japi/TransformationSampleJapiSpec.scala @@ -53,7 +53,7 @@ abstract class TransformationSampleJapiSpec extends MultiNodeSpec(Transformation override def afterAll() = multiNodeSpecAfterAll() "The japi transformation sample" must { - "illustrate how to start first frontend" in { + "illustrate how to start first frontend" in within(15 seconds) { runOn(frontend1) { // this will only run on the 'first' node Cluster(system) join node(frontend1).address From 4c440c91af4b562b3e624b315acd0a89fbc5f066 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:09:26 +0100 Subject: [PATCH 43/62] Silencing unhandled message in IOActor --- .../src/test/scala/akka/actor/IOActor.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 441f1515be..5cd9075e38 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,7 +14,7 @@ import akka.dispatch.MessageDispatcher import akka.pattern.ask import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress } import scala.util.Failure -import annotation.tailrec +import scala.annotation.tailrec object IOActorSpec { @@ -55,6 +55,8 @@ object IOActorSpec { def receive = { + case _: IO.Connected ⇒ //don't care + case bytes: ByteString ⇒ val source = sender socket write bytes @@ -65,9 +67,9 @@ object IOActorSpec { case IO.Closed(`socket`, cause) ⇒ state(cause) - throw cause match { - case IO.Error(e) ⇒ e - case _ ⇒ new RuntimeException("Socket closed") + cause match { + case IO.Error(e) ⇒ throw e + case _ ⇒ throw new RuntimeException("Socket closed") } } @@ -154,6 +156,8 @@ object IOActorSpec { case IO.Read(socket, bytes) ⇒ state(socket)(IO Chunk bytes) + case _: IO.Connected ⇒ //don't care + case IO.Closed(socket, cause) ⇒ state -= socket @@ -181,6 +185,8 @@ object IOActorSpec { readResult map (source !) } + case _: IO.Connected ⇒ //don't care + case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) @@ -276,7 +282,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { } "an IO Actor" must { - implicit val ec = system.dispatcher + import system.dispatcher "run echo server" in { filterException[java.net.ConnectException] { val addressPromise = Promise[SocketAddress]() From cc6e94021cd02211de6b34341cd190490cc3bf5c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:10:39 +0100 Subject: [PATCH 44/62] Fixing SupervisorHierarchySpec after discovered bug in patmat --- .../scala/akka/actor/SupervisorHierarchySpec.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index a34d220cc5..eb30bb182b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -194,7 +194,7 @@ object SupervisorHierarchySpec { case x ⇒ (x, x) } override val supervisorStrategy = OneForOneStrategy()(unwrap andThen { - case _: Failure if pongsToGo > 0 ⇒ + case (_: Failure, _) if pongsToGo > 0 ⇒ log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this)) Resume case (f: Failure, orig) ⇒ @@ -391,10 +391,10 @@ object SupervisorHierarchySpec { // don’t escalate from this one! override val supervisorStrategy = OneForOneStrategy() { - case f: Failure ⇒ f.directive - case OriginalRestartException(f: Failure) ⇒ f.directive - case ActorInitializationException(f: Failure) ⇒ f.directive - case _ ⇒ Stop + case f: Failure ⇒ f.directive + case OriginalRestartException(f: Failure) ⇒ f.directive + case ActorInitializationException(_, _, f: Failure) ⇒ f.directive + case _ ⇒ Stop } var children = Vector.empty[ActorRef] From 37643ca84eae82a1aef8e70a7c59b5711316d1a9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:11:02 +0100 Subject: [PATCH 45/62] Adding ConfigSpec test for unstarted-push-timeout --- akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 1aa168b924..9a43631894 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -45,6 +45,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10) settings.DefaultVirtualNodesFactor must be(10) + + getMilliseconds("akka.actor.unstarted-push-timeout") must be(10.seconds.toMillis) + settings.UnstartedPushTimeout.duration must be(10.seconds) } { From 6a8bed784bd737c2b1c186bc47594f6e2e805f8d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:12:32 +0100 Subject: [PATCH 46/62] Making RoutingSpec a bit more elaborate, to put pressure on the implementation --- .../test/scala/akka/routing/RoutingSpec.scala | 48 +++++++++++-------- 1 file changed, 27 insertions(+), 21 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 94a650e8c7..6dcb564190 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -5,7 +5,6 @@ package akka.routing import language.postfixOps -import java.util.concurrent.atomic.AtomicInteger import akka.actor._ import scala.collection.immutable import akka.testkit._ @@ -18,6 +17,8 @@ import java.util.concurrent.ConcurrentHashMap import com.typesafe.config.Config import akka.dispatch.Dispatchers import akka.util.Collections.EmptyImmutableSeq +import akka.util.Timeout +import java.util.concurrent.atomic.AtomicInteger object RoutingSpec { @@ -100,33 +101,36 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } "be able to send their routees" in { - class TheActor extends Actor { - val routee1 = context.actorOf(Props[TestActor], "routee1") - val routee2 = context.actorOf(Props[TestActor], "routee2") - val routee3 = context.actorOf(Props[TestActor], "routee3") - val router = context.actorOf(Props[TestActor].withRouter( - ScatterGatherFirstCompletedRouter( - routees = List(routee1, routee2, routee3), - within = 5 seconds))) + val actor = system.actorOf(Props(new Actor { def receive = { - case "doIt" ⇒ router ! CurrentRoutees - case routees: RouterRoutees ⇒ testActor forward routees + case (id: String, names: immutable.Iterable[_], actors: Int) ⇒ + val routerProps = Props[TestActor].withRouter( + ScatterGatherFirstCompletedRouter( + routees = names collect { case name: String ⇒ context.actorOf(Props(new TestActor), name) }, + within = 5 seconds)) + + 1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) } } - } + })) - val theActor = system.actorOf(Props(new TheActor), "theActor") - theActor ! "doIt" - val routees = expectMsgPF() { - case RouterRoutees(routees) ⇒ routees.toSet - } + val actors = 15 + val names = 1 to 20 map { "routee" + _ } toList - routees.map(_.path.name) must be(Set("routee1", "routee2", "routee3")) + actor ! (("test", names, actors)) + + 1 to actors foreach { _ ⇒ + val routees = expectMsgType[RouterRoutees].routees + routees.map(_.path.name) must be === names + } + expectNoMsg(500.millis) + actor ! PoisonPill } "use configured nr-of-instances when FromConfig" in { val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1") - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) watch(router) system.stop(router) expectMsgType[Terminated] @@ -134,7 +138,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with "use configured nr-of-instances when router is specified" in { val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2") - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) system.stop(router) } @@ -149,7 +154,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3") Await.ready(latch, remaining) - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) system.stop(router) } From 8067ffda3d35f0f35f00a32212dc6d161e2d5eaa Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:18:44 +0100 Subject: [PATCH 47/62] Clarifying docs in ActorCell --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 75dbd3fefa..9cf2b5b3df 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -259,12 +259,12 @@ private[akka] trait Cell { */ def isLocal: Boolean /** - * If the actor isLocal, returns whether messages are currently queued, + * If the actor isLocal, returns whether "user messages" are currently queued, * “false” otherwise. */ def hasMessages: Boolean /** - * If the actor isLocal, returns the number of messages currently queued, + * If the actor isLocal, returns the number of "user messages" currently queued, * which may be a costly operation, 0 otherwise. */ def numberOfMessages: Int From ae11fe4b538e3f296b9ac7c3d3e17973e5fc9ed2 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:20:10 +0100 Subject: [PATCH 48/62] Adding return type on foreachChild --- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 00b06dc594..50df88c171 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -530,7 +530,7 @@ private[akka] class VirtualPathContainer( def hasChildren: Boolean = !children.isEmpty - def foreachChild(f: ActorRef ⇒ Unit) = { + def foreachChild(f: ActorRef ⇒ Unit): Unit = { val iter = children.values.iterator while (iter.hasNext) f(iter.next) } From 36bfdaddcc36ba73953283ce8573b6e1ec76767c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 15 Nov 2012 10:42:14 +0100 Subject: [PATCH 49/62] Rewriting PriorityDispatcherSpec not to rely on suspension --- .../dispatch/PriorityDispatcherSpec.scala | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index 1e71627e69..4dad37c1be 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -6,10 +6,9 @@ import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import com.typesafe.config.Config -import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor } +import akka.actor.{ Props, ActorSystem, Actor } import akka.pattern.ask import akka.testkit.{ DefaultTimeout, AkkaSpec } -import scala.concurrent.Await import scala.concurrent.duration._ object PriorityDispatcherSpec { @@ -50,24 +49,23 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit } def testOrdering(dispatcherKey: String) { + val msgs = (1 to 100) toList val actor = system.actorOf(Props(new Actor { - var acc: List[Int] = Nil + + val acc = scala.collection.mutable.ListBuffer[Int]() + + scala.util.Random.shuffle(msgs) foreach { m ⇒ self ! m } + + self.tell('Result, testActor) def receive = { - case i: Int ⇒ acc = i :: acc - case 'Result ⇒ sender ! acc + case i: Int ⇒ acc += i + case 'Result ⇒ sender ! acc.toList } - }).withDispatcher(dispatcherKey)).asInstanceOf[InternalActorRef] + }).withDispatcher(dispatcherKey)) - actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages - - val msgs = (1 to 100).toList - for (m ← msgs) actor ! m - - actor.resume(causedByFailure = null) //Signal the actor to start treating it's message backlog - - Await.result(actor.?('Result).mapTo[List[Int]], timeout.duration) must be === msgs.reverse + expectMsgType[List[_]] must be === msgs } } From da78a7c1678d280070c1fb57abd92017fdc491a0 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 14 Nov 2012 17:15:58 +0100 Subject: [PATCH 50/62] #2694 - Fixing DataflowDocSpec examples --- .../src/main/scala/akka/dataflow/package.scala | 6 +++--- .../scala/code/docs/dataflow/DataflowDocSpec.scala | 13 ++++++++----- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/akka-dataflow/src/main/scala/akka/dataflow/package.scala b/akka-dataflow/src/main/scala/akka/dataflow/package.scala index 9f4e6a0da2..31248958d1 100644 --- a/akka-dataflow/src/main/scala/akka/dataflow/package.scala +++ b/akka-dataflow/src/main/scala/akka/dataflow/package.scala @@ -46,7 +46,7 @@ package object dataflow { implicit class DataflowPromise[T](val promise: Promise[T]) extends AnyVal { /** - * Completes the Promise with the speicifed value or throws an exception if already + * Completes the Promise with the specified value or throws an exception if already * completed. See Promise.success(value) for semantics. * * @param value The value which denotes the successful value of the Promise @@ -59,7 +59,7 @@ package object dataflow { /** * Completes this Promise with the value of the specified Future when/if it completes. * - * @param other The Future whose value will be transfered to this Promise upon completion + * @param other The Future whose value will be transferred to this Promise upon completion * @param ec An ExecutionContext which will be used to execute callbacks registered in this method * @return A Future representing the result of this operation */ @@ -75,7 +75,7 @@ package object dataflow { /** * Completes this Promise with the value of the specified Promise when/if it completes. * - * @param other The Promise whose value will be transfered to this Promise upon completion + * @param other The Promise whose value will be transferred to this Promise upon completion * @param ec An ExecutionContext which will be used to execute callbacks registered in this method * @return A Future representing the result of this operation */ diff --git a/akka-docs/rst/scala/code/docs/dataflow/DataflowDocSpec.scala b/akka-docs/rst/scala/code/docs/dataflow/DataflowDocSpec.scala index a32ee6fcae..345d23b4ac 100644 --- a/akka-docs/rst/scala/code/docs/dataflow/DataflowDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/dataflow/DataflowDocSpec.scala @@ -44,21 +44,23 @@ class DataflowDocSpec extends WordSpec with MustMatchers { } "demonstrate the use of dataflow variables" in { - def println[T](any: Try[T]): Unit = any.get must be === 20 + val result = Promise[Int]() + def println(any: Try[Int]): Unit = result.complete(any) //#dataflow-variable-a + val v1, v2 = Promise[Int]() flow { - val v1, v2 = Promise[Int]() - // v1 will become the value of v2 + 10 when v2 gets a value v1 << v2() + 10 - v2 << flow { 5 } // As you can see, no blocking! v1() + v2() } onComplete println + flow { v2 << 5 } // As you can see, no blocking above! //#dataflow-variable-a + Await.result(result.future, 10.seconds) must be === 20 } "demonstrate the difference between for and flow" in { - def println[T](any: Try[T]): Unit = any.get must be === 2 + val result = Promise[Int]() + def println(any: Try[Int]): Unit = result.tryComplete(any) //#for-vs-flow val f1, f2 = Future { 1 } @@ -68,6 +70,7 @@ class DataflowDocSpec extends WordSpec with MustMatchers { usingFor onComplete println usingFlow onComplete println //#for-vs-flow + Await.result(result.future, 10.seconds) must be === 2 } } From 9205c1eef7c9ae79787d8731814b47753e10995c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 15 Nov 2012 15:47:55 +0100 Subject: [PATCH 51/62] #2710 - requiring sender and recipient of DeadLetters to be non-null --- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 12 +++++++++--- .../main/scala/akka/actor/RepointableActorRef.scala | 7 ++++--- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 00b06dc594..6bf3b3bf7c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -419,7 +419,10 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { * to the ActorSystem's EventStream */ @SerialVersionUID(1L) -case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) +case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) { + require(sender ne null, "DeadLetter sender may not be null") + require(recipient ne null, "DeadLetter recipient may not be null") +} private[akka] object DeadLetterActorRef { @SerialVersionUID(1L) @@ -446,8 +449,11 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider, override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message) override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match { - case d: DeadLetter ⇒ specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend! - case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this)) + case d: DeadLetter ⇒ + specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend! + case _ if !specialHandle(message) ⇒ + eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this)) + case _ ⇒ } protected def specialHandle(msg: Any): Boolean = msg match { diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala index b644ba6445..8ecb1cbb72 100644 --- a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala @@ -189,15 +189,16 @@ private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: Rep def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer def getChildByName(name: String): Option[ChildRestartStats] = None def tell(message: Any, sender: ActorRef): Unit = { + val useSender = if (sender eq Actor.noSender) system.deadLetters else sender if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) { try { - if (self.underlying eq this) queue enqueue Envelope(message, sender, system) - else self.underlying.tell(message, sender) + if (self.underlying eq this) queue enqueue Envelope(message, useSender, system) + else self.underlying.tell(message, useSender) } finally { lock.unlock() } } else { - system.deadLetters ! DeadLetter(message, sender, self) + system.deadLetters ! DeadLetter(message, useSender, self) } } def sendSystemMessage(msg: SystemMessage): Unit = { From 8194ddd13c4992ed732773c971aafa0b2b9d5df8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 15 Nov 2012 02:31:40 +0100 Subject: [PATCH 52/62] #2703 - unprivatizing NoLogging and adding a Java API for obtaining the reference --- .../test/java/akka/japi/JavaAPITestBase.java | 8 +++++ .../src/main/scala/akka/event/Logging.scala | 30 ++++++++++++------- .../code/docs/event/LoggingDocTestBase.java | 1 - 3 files changed, 28 insertions(+), 11 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/japi/JavaAPITestBase.java b/akka-actor-tests/src/test/java/akka/japi/JavaAPITestBase.java index c0361530da..b3a092b1f9 100644 --- a/akka-actor-tests/src/test/java/akka/japi/JavaAPITestBase.java +++ b/akka-actor-tests/src/test/java/akka/japi/JavaAPITestBase.java @@ -1,5 +1,7 @@ package akka.japi; +import akka.event.LoggingAdapter; +import akka.event.NoLogging; import org.junit.Test; import static org.junit.Assert.*; @@ -46,4 +48,10 @@ public class JavaAPITestBase { public void shouldBeSingleton() { assertSame(Option.none(), Option.none()); } + + @Test + public void mustBeAbleToGetNoLogging() { + LoggingAdapter a = NoLogging.getInstance(); + assertNotNull(a); + } } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index dbd561514d..14ba99bcaa 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -878,15 +878,25 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message)) } -private[akka] object NoLogging extends LoggingAdapter { - def isErrorEnabled = false - def isWarningEnabled = false - def isInfoEnabled = false - def isDebugEnabled = false +/** + * NoLogging is a LoggingAdapter that does absolutely nothing – no logging at all. + */ +object NoLogging extends LoggingAdapter { - protected def notifyError(message: String): Unit = () - protected def notifyError(cause: Throwable, message: String): Unit = () - protected def notifyWarning(message: String): Unit = () - protected def notifyInfo(message: String): Unit = () - protected def notifyDebug(message: String): Unit = () + /** + * Java API to return the reference to NoLogging + * @return The NoLogging instance + */ + def getInstance = this + + final override def isErrorEnabled = false + final override def isWarningEnabled = false + final override def isInfoEnabled = false + final override def isDebugEnabled = false + + final protected override def notifyError(message: String): Unit = () + final protected override def notifyError(cause: Throwable, message: String): Unit = () + final protected override def notifyWarning(message: String): Unit = () + final protected override def notifyInfo(message: String): Unit = () + final protected override def notifyDebug(message: String): Unit = () } diff --git a/akka-docs/rst/java/code/docs/event/LoggingDocTestBase.java b/akka-docs/rst/java/code/docs/event/LoggingDocTestBase.java index 54847c4f66..3e3fa46844 100644 --- a/akka-docs/rst/java/code/docs/event/LoggingDocTestBase.java +++ b/akka-docs/rst/java/code/docs/event/LoggingDocTestBase.java @@ -119,5 +119,4 @@ public class LoggingDocTestBase { } } //#deadletter-actor - } From 0e37586d17d48d612f2782ac46b5c2208f6bff70 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:09:26 +0100 Subject: [PATCH 53/62] Silencing unhandled message in IOActor --- .../src/test/scala/akka/actor/IOActor.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 441f1515be..5cd9075e38 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,7 +14,7 @@ import akka.dispatch.MessageDispatcher import akka.pattern.ask import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress } import scala.util.Failure -import annotation.tailrec +import scala.annotation.tailrec object IOActorSpec { @@ -55,6 +55,8 @@ object IOActorSpec { def receive = { + case _: IO.Connected ⇒ //don't care + case bytes: ByteString ⇒ val source = sender socket write bytes @@ -65,9 +67,9 @@ object IOActorSpec { case IO.Closed(`socket`, cause) ⇒ state(cause) - throw cause match { - case IO.Error(e) ⇒ e - case _ ⇒ new RuntimeException("Socket closed") + cause match { + case IO.Error(e) ⇒ throw e + case _ ⇒ throw new RuntimeException("Socket closed") } } @@ -154,6 +156,8 @@ object IOActorSpec { case IO.Read(socket, bytes) ⇒ state(socket)(IO Chunk bytes) + case _: IO.Connected ⇒ //don't care + case IO.Closed(socket, cause) ⇒ state -= socket @@ -181,6 +185,8 @@ object IOActorSpec { readResult map (source !) } + case _: IO.Connected ⇒ //don't care + case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) @@ -276,7 +282,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { } "an IO Actor" must { - implicit val ec = system.dispatcher + import system.dispatcher "run echo server" in { filterException[java.net.ConnectException] { val addressPromise = Promise[SocketAddress]() From 75ccbef0f75948017ca6dcb1c3b8e45d1e3262bc Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:10:39 +0100 Subject: [PATCH 54/62] Fixing SupervisorHierarchySpec after discovered bug in patmat --- .../scala/akka/actor/SupervisorHierarchySpec.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index a34d220cc5..eb30bb182b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -194,7 +194,7 @@ object SupervisorHierarchySpec { case x ⇒ (x, x) } override val supervisorStrategy = OneForOneStrategy()(unwrap andThen { - case _: Failure if pongsToGo > 0 ⇒ + case (_: Failure, _) if pongsToGo > 0 ⇒ log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this)) Resume case (f: Failure, orig) ⇒ @@ -391,10 +391,10 @@ object SupervisorHierarchySpec { // don’t escalate from this one! override val supervisorStrategy = OneForOneStrategy() { - case f: Failure ⇒ f.directive - case OriginalRestartException(f: Failure) ⇒ f.directive - case ActorInitializationException(f: Failure) ⇒ f.directive - case _ ⇒ Stop + case f: Failure ⇒ f.directive + case OriginalRestartException(f: Failure) ⇒ f.directive + case ActorInitializationException(_, _, f: Failure) ⇒ f.directive + case _ ⇒ Stop } var children = Vector.empty[ActorRef] From a3f9e12d9f0a3f875601b195227328471d8d9f9c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:11:02 +0100 Subject: [PATCH 55/62] Adding ConfigSpec test for unstarted-push-timeout --- akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 1aa168b924..9a43631894 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -45,6 +45,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10) settings.DefaultVirtualNodesFactor must be(10) + + getMilliseconds("akka.actor.unstarted-push-timeout") must be(10.seconds.toMillis) + settings.UnstartedPushTimeout.duration must be(10.seconds) } { From 672c5971fc7e64c95c60e58edb25c0f060f4d1f3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:12:32 +0100 Subject: [PATCH 56/62] Making RoutingSpec a bit more elaborate, to put pressure on the implementation --- .../test/scala/akka/routing/RoutingSpec.scala | 48 +++++++++++-------- 1 file changed, 27 insertions(+), 21 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 94a650e8c7..6dcb564190 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -5,7 +5,6 @@ package akka.routing import language.postfixOps -import java.util.concurrent.atomic.AtomicInteger import akka.actor._ import scala.collection.immutable import akka.testkit._ @@ -18,6 +17,8 @@ import java.util.concurrent.ConcurrentHashMap import com.typesafe.config.Config import akka.dispatch.Dispatchers import akka.util.Collections.EmptyImmutableSeq +import akka.util.Timeout +import java.util.concurrent.atomic.AtomicInteger object RoutingSpec { @@ -100,33 +101,36 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } "be able to send their routees" in { - class TheActor extends Actor { - val routee1 = context.actorOf(Props[TestActor], "routee1") - val routee2 = context.actorOf(Props[TestActor], "routee2") - val routee3 = context.actorOf(Props[TestActor], "routee3") - val router = context.actorOf(Props[TestActor].withRouter( - ScatterGatherFirstCompletedRouter( - routees = List(routee1, routee2, routee3), - within = 5 seconds))) + val actor = system.actorOf(Props(new Actor { def receive = { - case "doIt" ⇒ router ! CurrentRoutees - case routees: RouterRoutees ⇒ testActor forward routees + case (id: String, names: immutable.Iterable[_], actors: Int) ⇒ + val routerProps = Props[TestActor].withRouter( + ScatterGatherFirstCompletedRouter( + routees = names collect { case name: String ⇒ context.actorOf(Props(new TestActor), name) }, + within = 5 seconds)) + + 1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) } } - } + })) - val theActor = system.actorOf(Props(new TheActor), "theActor") - theActor ! "doIt" - val routees = expectMsgPF() { - case RouterRoutees(routees) ⇒ routees.toSet - } + val actors = 15 + val names = 1 to 20 map { "routee" + _ } toList - routees.map(_.path.name) must be(Set("routee1", "routee2", "routee3")) + actor ! (("test", names, actors)) + + 1 to actors foreach { _ ⇒ + val routees = expectMsgType[RouterRoutees].routees + routees.map(_.path.name) must be === names + } + expectNoMsg(500.millis) + actor ! PoisonPill } "use configured nr-of-instances when FromConfig" in { val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1") - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) watch(router) system.stop(router) expectMsgType[Terminated] @@ -134,7 +138,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with "use configured nr-of-instances when router is specified" in { val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2") - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) system.stop(router) } @@ -149,7 +154,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3") Await.ready(latch, remaining) - Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3) + router ! CurrentRoutees + expectMsgType[RouterRoutees].routees.size must be(3) system.stop(router) } From b47ed349d0d73a9889f7bae52b18ef7de56865f2 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:18:44 +0100 Subject: [PATCH 57/62] Clarifying docs in ActorCell --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 75dbd3fefa..9cf2b5b3df 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -259,12 +259,12 @@ private[akka] trait Cell { */ def isLocal: Boolean /** - * If the actor isLocal, returns whether messages are currently queued, + * If the actor isLocal, returns whether "user messages" are currently queued, * “false” otherwise. */ def hasMessages: Boolean /** - * If the actor isLocal, returns the number of messages currently queued, + * If the actor isLocal, returns the number of "user messages" currently queued, * which may be a costly operation, 0 otherwise. */ def numberOfMessages: Int From ef84352289fdbdb267039690282d3e045a246af3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 14:20:10 +0100 Subject: [PATCH 58/62] Adding return type on foreachChild --- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 6bf3b3bf7c..a6685ae549 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -536,7 +536,7 @@ private[akka] class VirtualPathContainer( def hasChildren: Boolean = !children.isEmpty - def foreachChild(f: ActorRef ⇒ Unit) = { + def foreachChild(f: ActorRef ⇒ Unit): Unit = { val iter = children.values.iterator while (iter.hasNext) f(iter.next) } From b5bcdb081fe5628f5db904563d98bb2a0aeaf8d1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 16 Nov 2012 17:30:27 +0100 Subject: [PATCH 59/62] Minor RoutingSpec refactoring --- .../src/test/scala/akka/routing/RoutingSpec.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 6dcb564190..9d7522f950 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -101,13 +101,13 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with } "be able to send their routees" in { - + case class TestRun(id: String, names: immutable.Iterable[String], actors: Int) val actor = system.actorOf(Props(new Actor { def receive = { - case (id: String, names: immutable.Iterable[_], actors: Int) ⇒ + case TestRun(id, names, actors) ⇒ val routerProps = Props[TestActor].withRouter( ScatterGatherFirstCompletedRouter( - routees = names collect { case name: String ⇒ context.actorOf(Props(new TestActor), name) }, + routees = names map { context.actorOf(Props(new TestActor), _) }, within = 5 seconds)) 1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) } @@ -117,14 +117,13 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with val actors = 15 val names = 1 to 20 map { "routee" + _ } toList - actor ! (("test", names, actors)) + actor ! TestRun("test", names, actors) 1 to actors foreach { _ ⇒ val routees = expectMsgType[RouterRoutees].routees routees.map(_.path.name) must be === names } expectNoMsg(500.millis) - actor ! PoisonPill } "use configured nr-of-instances when FromConfig" in { From 739583e384e3d97c8f04d99e70be25cd53b92547 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Thu, 15 Nov 2012 21:28:40 -0500 Subject: [PATCH 60/62] Fix bug where ActorSystems created via OsgiActorSystemFactory(ctx).createActorSystem fail to load akka.event classes unless bundle imports akka.event package (cherry picked from commit 3070343a5de78cd7b363e93a09d56565b1e6df9b) --- akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala index 608b80403b..859d671ed3 100644 --- a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala +++ b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala @@ -17,7 +17,7 @@ class OsgiActorSystemFactory(val context: BundleContext) { /* * Classloader that delegates to the bundle for which the factory is creating an ActorSystem */ - private val classloader = BundleDelegatingClassLoader(context) + private val classloader = new BundleDelegatingClassLoader(context.getBundle, Some(classOf[ActorSystem].getClassLoader)) /** * Creates the [[akka.actor.ActorSystem]], using the name specified From 44c317832a5dabafdeb0e55c45a3cdbce1daaa0e Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 16 Nov 2012 12:55:27 -0500 Subject: [PATCH 61/62] Allow fallback classloader to be explicitly specified (cherry picked from commit f51a2894150e0f5cc6923b8e3aa42d7ebdd6663d) --- .../src/main/scala/akka/osgi/OsgiActorSystemFactory.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala index 859d671ed3..3f02936bfd 100644 --- a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala +++ b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala @@ -12,12 +12,12 @@ import org.osgi.framework.BundleContext * Factory class to create ActorSystem implementations in an OSGi environment. This mainly involves dealing with * bundle classloaders appropriately to ensure that configuration files and classes get loaded properly */ -class OsgiActorSystemFactory(val context: BundleContext) { +class OsgiActorSystemFactory(val context: BundleContext, val fallbackClassLoader: Option[ClassLoader]) { /* * Classloader that delegates to the bundle for which the factory is creating an ActorSystem */ - private val classloader = new BundleDelegatingClassLoader(context.getBundle, Some(classOf[ActorSystem].getClassLoader)) + private val classloader = new BundleDelegatingClassLoader(context.getBundle, fallbackClassLoader) /** * Creates the [[akka.actor.ActorSystem]], using the name specified @@ -52,5 +52,5 @@ object OsgiActorSystemFactory { /* * Create an [[OsgiActorSystemFactory]] instance to set up Akka in an OSGi environment */ - def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context) + def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context, Some(classOf[ActorSystem].getClassLoader)) } From 608d04c0501dcbd075e62ba2fe2351133cf46430 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 16 Nov 2012 13:41:36 -0500 Subject: [PATCH 62/62] Fix build breakage in akka-osgi-aries due to last commit (cherry picked from commit 68b60745f2d7b30d9ccd57536cd9427b04303b84) --- .../aries/blueprint/BlueprintActorSystemFactory.scala | 4 +++- .../main/scala/akka/osgi/OsgiActorSystemFactory.scala | 9 +++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala index 30720a230c..ce759a4fa8 100644 --- a/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala +++ b/akka-osgi-aries/src/main/scala/akka/osgi/aries/blueprint/BlueprintActorSystemFactory.scala @@ -15,7 +15,9 @@ import com.typesafe.config.{ Config, ConfigFactory } * If you're looking for a way to set up Akka using Blueprint without the namespace handler, you should use * [[akka.osgi.OsgiActorSystemFactory]] instead. */ -class BlueprintActorSystemFactory(context: BundleContext, name: String) extends OsgiActorSystemFactory(context) { +class BlueprintActorSystemFactory(context: BundleContext, name: String, fallbackClassLoader: Option[ClassLoader]) extends OsgiActorSystemFactory(context, fallbackClassLoader) { + + def this(context: BundleContext, name: String) = this(context, name, Some(OsgiActorSystemFactory.akkaActorClassLoader)) var config: Option[String] = None diff --git a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala index 3f02936bfd..447719ef39 100644 --- a/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala +++ b/akka-osgi/src/main/scala/akka/osgi/OsgiActorSystemFactory.scala @@ -37,7 +37,7 @@ class OsgiActorSystemFactory(val context: BundleContext, val fallbackClassLoader * loaded from the akka-actor bundle. */ def actorSystemConfig(context: BundleContext): Config = - ConfigFactory.load(classloader).withFallback(ConfigFactory.defaultReference(classOf[ActorSystem].getClassLoader)) + ConfigFactory.load(classloader).withFallback(ConfigFactory.defaultReference(OsgiActorSystemFactory.akkaActorClassLoader)) /** * Determine the name for the [[akka.actor.ActorSystem]] @@ -49,8 +49,13 @@ class OsgiActorSystemFactory(val context: BundleContext, val fallbackClassLoader } object OsgiActorSystemFactory { + /** + * Class loader of akka-actor bundle. + */ + def akkaActorClassLoader = classOf[ActorSystem].getClassLoader + /* * Create an [[OsgiActorSystemFactory]] instance to set up Akka in an OSGi environment */ - def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context, Some(classOf[ActorSystem].getClassLoader)) + def apply(context: BundleContext): OsgiActorSystemFactory = new OsgiActorSystemFactory(context, Some(akkaActorClassLoader)) }