Merge branch 'master' into wip-2053d-actorbased-remote-drewhk

This commit is contained in:
Endre Sándor Varga 2012-11-21 11:44:39 +01:00
commit 0f0c5cb17a
160 changed files with 2071 additions and 1089 deletions

View file

@ -1,5 +1,7 @@
package akka.japi; package akka.japi;
import akka.event.LoggingAdapter;
import akka.event.NoLogging;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
@ -46,4 +48,10 @@ public class JavaAPITestBase {
public void shouldBeSingleton() { public void shouldBeSingleton() {
assertSame(Option.none(), Option.none()); assertSame(Option.none(), Option.none());
} }
@Test
public void mustBeAbleToGetNoLogging() {
LoggingAdapter a = NoLogging.getInstance();
assertNotNull(a);
}
} }

View file

@ -15,7 +15,8 @@ public class CustomRouteTest {
// only to test compilability // only to test compilability
public void testRoute() { public void testRoute() {
final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1))); final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1)));
final scala.Function1<scala.Tuple2<ActorRef, Object>, scala.collection.Iterable<Destination>> route = ExtractRoute.apply(ref); final scala.Function1<scala.Tuple2<ActorRef, Object>,
scala.collection.immutable.Iterable<Destination>> route = ExtractRoute.apply(ref);
route.apply(null); route.apply(null);
} }

View file

@ -103,6 +103,32 @@ class ActorDSLSpec extends AkkaSpec {
i.receive() must be("hi") 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 { "support setup/teardown" in {
//#simple-start-stop //#simple-start-stop
val a = actor(new Act { val a = actor(new Act {
@ -188,7 +214,7 @@ class ActorDSLSpec extends AkkaSpec {
become { become {
case 1 stash() case 1 stash()
case 2 case 2
testActor ! 2; unstashAll(); become { testActor ! 2; unstashAll(); becomeStacked {
case 1 testActor ! 1; unbecome() case 1 testActor ! 1; unbecome()
} }
} }

View file

@ -4,14 +4,15 @@
package akka.actor package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.JavaConverters import java.util.concurrent.{ RejectedExecutionException, ConcurrentLinkedQueue }
import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue }
import akka.util.Timeout import akka.util.Timeout
import akka.japi.Util.immutableSeq
import scala.concurrent.Future import scala.concurrent.Future
import akka.pattern.ask import akka.pattern.ask
@ -102,8 +103,6 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
} }
"run termination callbacks in order" in { "run termination callbacks in order" in {
import scala.collection.JavaConverters._
val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf) val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf)
val result = new ConcurrentLinkedQueue[Int] val result = new ConcurrentLinkedQueue[Int]
val count = 10 val count = 10
@ -121,13 +120,11 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
Await.ready(latch, 5 seconds) Await.ready(latch, 5 seconds)
val expected = (for (i 1 to count) yield i).reverse 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 { "awaitTermination after termination callbacks" in {
import scala.collection.JavaConverters._
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
@volatile @volatile
var callbackWasRun = false var callbackWasRun = false

View file

@ -14,7 +14,7 @@ import akka.dispatch.MessageDispatcher
import akka.pattern.ask import akka.pattern.ask
import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress } import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress }
import scala.util.Failure import scala.util.Failure
import annotation.tailrec import scala.annotation.tailrec
object IOActorSpec { object IOActorSpec {
@ -55,6 +55,8 @@ object IOActorSpec {
def receive = { def receive = {
case _: IO.Connected //don't care
case bytes: ByteString case bytes: ByteString
val source = sender val source = sender
socket write bytes socket write bytes
@ -65,9 +67,9 @@ object IOActorSpec {
case IO.Closed(`socket`, cause) case IO.Closed(`socket`, cause)
state(cause) state(cause)
throw cause match { cause match {
case IO.Error(e) e case IO.Error(e) throw e
case _ new RuntimeException("Socket closed") case _ throw new RuntimeException("Socket closed")
} }
} }
@ -154,6 +156,8 @@ object IOActorSpec {
case IO.Read(socket, bytes) case IO.Read(socket, bytes)
state(socket)(IO Chunk bytes) state(socket)(IO Chunk bytes)
case _: IO.Connected //don't care
case IO.Closed(socket, cause) case IO.Closed(socket, cause)
state -= socket state -= socket
@ -181,6 +185,8 @@ object IOActorSpec {
readResult map (source !) readResult map (source !)
} }
case _: IO.Connected //don't care
case IO.Read(`socket`, bytes) case IO.Read(`socket`, bytes)
state(IO Chunk bytes) state(IO Chunk bytes)
@ -276,7 +282,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout {
} }
"an IO Actor" must { "an IO Actor" must {
implicit val ec = system.dispatcher import system.dispatcher
"run echo server" in { "run echo server" in {
filterException[java.net.ConnectException] { filterException[java.net.ConnectException] {
val addressPromise = Promise[SocketAddress]() val addressPromise = Promise[SocketAddress]()

View file

@ -43,9 +43,10 @@ class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.confi
val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552" val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552"
val a = system.actorOf(Props(new Actor { val a = system.actorOf(Props(new Actor {
val child = context.actorOf(Props.empty, name = childName) val child = context.actorOf(Props.empty, name = childName)
assert(childName == child.path.name)
def receive = { 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) a.tell("lookup", testActor)

View file

@ -6,24 +6,22 @@ package akka.actor
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import java.net.URLEncoder import java.net.URLEncoder
import scala.collection.immutable
class RelativeActorPathSpec extends WordSpec with MustMatchers { class RelativeActorPathSpec extends WordSpec with MustMatchers {
def elements(path: String): Seq[String] = path match { def elements(path: String): immutable.Seq[String] = RelativeActorPath.unapply(path).getOrElse(Nil)
case RelativeActorPath(elem) elem.toSeq
case _ Nil
}
"RelativeActorPath" must { "RelativeActorPath" must {
"match single name" in { "match single name" in {
elements("foo") must be(Seq("foo")) elements("foo") must be(List("foo"))
} }
"match path separated names" in { "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 { "match url encoded name" in {
val name = URLEncoder.encode("akka://ClusterSystem@127.0.0.1:2552", "UTF-8") 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))
} }
} }
} }

View file

@ -194,7 +194,7 @@ object SupervisorHierarchySpec {
case x (x, x) case x (x, x)
} }
override val supervisorStrategy = OneForOneStrategy()(unwrap andThen { override val supervisorStrategy = OneForOneStrategy()(unwrap andThen {
case _: Failure if pongsToGo > 0 case (_: Failure, _) if pongsToGo > 0
log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this)) log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this))
Resume Resume
case (f: Failure, orig) case (f: Failure, orig)
@ -391,10 +391,10 @@ object SupervisorHierarchySpec {
// dont escalate from this one! // dont escalate from this one!
override val supervisorStrategy = OneForOneStrategy() { override val supervisorStrategy = OneForOneStrategy() {
case f: Failure f.directive case f: Failure f.directive
case OriginalRestartException(f: Failure) f.directive case OriginalRestartException(f: Failure) f.directive
case ActorInitializationException(f: Failure) f.directive case ActorInitializationException(_, _, f: Failure) f.directive
case _ Stop case _ Stop
} }
var children = Vector.empty[ActorRef] var children = Vector.empty[ActorRef]

View file

@ -5,18 +5,19 @@ package akka.actor
import language.postfixOps import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } 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.{ Await, Future, Promise }
import scala.concurrent.duration._ import scala.concurrent.duration._
import java.util.concurrent.atomic.AtomicReference
import annotation.tailrec
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec } import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
import akka.util.Timeout
import akka.japi.{ Option JOption } import akka.japi.{ Option JOption }
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.{ Dispatchers } import akka.dispatch.Dispatchers
import akka.pattern.ask import akka.pattern.ask
import akka.serialization.JavaSerializer import akka.serialization.JavaSerializer
import akka.actor.TypedActor._ import akka.actor.TypedActor._
import java.util.concurrent.atomic.AtomicReference
import java.lang.IllegalStateException import java.lang.IllegalStateException
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } 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 def hasNext = items != Nil

View file

@ -45,6 +45,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10) getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10)
settings.DefaultVirtualNodesFactor 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)
} }
{ {

View file

@ -6,10 +6,9 @@ import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner import org.scalatest.junit.JUnitRunner
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor } import akka.actor.{ Props, ActorSystem, Actor }
import akka.pattern.ask import akka.pattern.ask
import akka.testkit.{ DefaultTimeout, AkkaSpec } import akka.testkit.{ DefaultTimeout, AkkaSpec }
import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
object PriorityDispatcherSpec { object PriorityDispatcherSpec {
@ -50,24 +49,23 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
} }
def testOrdering(dispatcherKey: String) { def testOrdering(dispatcherKey: String) {
val msgs = (1 to 100) toList
val actor = system.actorOf(Props(new Actor { 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 = { def receive = {
case i: Int acc = i :: acc case i: Int acc += i
case 'Result sender ! acc 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 expectMsgType[List[_]] must be === msgs
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
} }
} }

View file

@ -12,17 +12,18 @@ import java.io.PrintWriter
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import scala.collection.mutable.{ Map MutableMap } import scala.collection.mutable.{ Map MutableMap }
import scala.collection.immutable
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
trait BenchResultRepository { trait BenchResultRepository {
def add(stats: Stats) 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 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 def isBaseline(stats: Stats): Boolean
@ -38,9 +39,9 @@ object BenchResultRepository {
} }
class FileBenchResultRepository extends 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 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 def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir")
private val serDir = resultDir + "/ser" private val serDir = resultDir + "/ser"
private def serDirExists: Boolean = new File(serDir).exists private def serDirExists: Boolean = new File(serDir).exists
@ -51,13 +52,13 @@ class FileBenchResultRepository extends BenchResultRepository {
case class Key(name: String, load: Int) case class Key(name: String, load: Int)
def add(stats: Stats): Unit = synchronized { 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 statsByName(stats.name) = values :+ stats
save(stats) save(stats)
} }
def get(name: String): Seq[Stats] = synchronized { def get(name: String): immutable.Seq[Stats] = synchronized {
statsByName.getOrElse(name, IndexedSeq.empty) statsByName.getOrElse(name, Vector.empty)
} }
def get(name: String, load: Int): Option[Stats] = synchronized { 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) 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 key = Key(name, load)
val historical = historicalStats.getOrElse(key, IndexedSeq.empty) val historical = historicalStats.getOrElse(key, Vector.empty)
val baseline = baselineStats.get(key) val baseline = baselineStats.get(key)
val current = get(name, load) 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) limited.sortBy(_.timestamp)
} }
@ -94,7 +95,7 @@ class FileBenchResultRepository extends BenchResultRepository {
} }
val historical = load(historicalFiles) val historical = load(historicalFiles)
for (h historical) { 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 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 = val result =
for (f files) yield { for (f files) yield {
var in: ObjectInputStream = null var in: ObjectInputStream = null
@ -132,11 +133,11 @@ class FileBenchResultRepository extends BenchResultRepository {
case e: Throwable case e: Throwable
None None
} finally { } 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() loadFiles()

View file

@ -3,7 +3,7 @@ package akka.performance.workbench
import java.io.UnsupportedEncodingException import java.io.UnsupportedEncodingException
import java.net.URLEncoder 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/ * 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. * 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) "" if (statsByTimestamp.isEmpty) ""
else { else {
val loads = statsByTimestamp.values.head.map(_.load) val loads = statsByTimestamp.values.head.map(_.load)
@ -46,7 +46,7 @@ object GoogleChartBuilder {
//sb.append("&") //sb.append("&")
// legend // legend
val legendStats = statsByTimestamp.values.map(_.head).toSeq val legendStats = statsByTimestamp.values.toVector.map(_.head)
appendLegend(legendStats, sb, legend) appendLegend(legendStats, sb, legend)
sb.append("&") sb.append("&")
// bar spacing // bar spacing
@ -60,10 +60,7 @@ object GoogleChartBuilder {
val loadStr = loads.mkString(",") val loadStr = loads.mkString(",")
sb.append("chd=t:") sb.append("chd=t:")
val maxValue = allStats.map(_.tps).max val maxValue = allStats.map(_.tps).max
val tpsSeries: Iterable[String] = val tpsSeries: Iterable[String] = for (statsSeq statsByTimestamp.values) yield statsSeq.map(_.tps).mkString(",")
for (statsSeq statsByTimestamp.values) yield {
statsSeq.map(_.tps).mkString(",")
}
sb.append(tpsSeries.mkString("|")) sb.append(tpsSeries.mkString("|"))
// y range // y range
@ -83,7 +80,7 @@ object GoogleChartBuilder {
/** /**
* Builds a bar chart for all percentiles and the mean in the statistics. * 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) "" if (statistics.isEmpty) ""
else { else {
val current = statistics.last 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:|") sb.append("chxl=1:|")
val s = percentiles.keys.toList.map(_ + "%").mkString("|") val s = percentiles.keys.toList.map(_ + "%").mkString("|")
sb.append(s) 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(_)) val legends = statistics.map(legend(_))
sb.append("chdl=") sb.append("chdl=")
val s = legends.map(urlEncode(_)).mkString("|") val s = legends.map(urlEncode(_)).mkString("|")
@ -166,7 +163,7 @@ object GoogleChartBuilder {
sb.append(s) 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 = val percentileSeries =
for { for {
percentiles allPercentiles percentiles allPercentiles
@ -181,7 +178,7 @@ object GoogleChartBuilder {
sb.append(series.mkString("|")) 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(_)) val series = values.map(formatDouble(_))
sb.append(series.mkString("|")) 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) "" if (statistics.isEmpty) ""
else { else {
val sb = new StringBuilder val sb = new StringBuilder

View file

@ -5,7 +5,7 @@ import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.Logging import akka.event.Logging
import scala.collection.immutable.TreeMap import scala.collection.immutable
class Report( class Report(
system: ActorSystem, system: ActorSystem,
@ -19,7 +19,7 @@ class Report(
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm") val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss") val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss")
def html(statistics: Seq[Stats]) { def html(statistics: immutable.Seq[Stats]) {
val current = statistics.last val current = statistics.last
val sb = new StringBuilder val sb = new StringBuilder
@ -80,13 +80,13 @@ class Report(
chartUrl chartUrl
} }
def comparePercentilesAndMeanChart(stats: Stats): Seq[String] = { def comparePercentilesAndMeanChart(stats: Stats): immutable.Seq[String] = {
for { for {
compareName compareResultWith.toSeq compareName compareResultWith.to[immutable.Seq]
compareStats resultRepository.get(compareName, stats.load) compareStats resultRepository.get(compareName, stats.load)
} yield { } yield {
val chartTitle = stats.name + " vs. " + compareName + ", " + stats.load + " clients" + ", Percentiles and Mean (microseconds)" 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 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) { if (statistics.isEmpty) {
None None
} else { } else {
val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp) 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 { (for (ts histTimestamps) yield {
val seq = val seq =
for (stats statistics) yield { 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) val cell = withHistorical.find(_.timestamp == ts)
cell.getOrElse(Stats(stats.name, stats.load, ts)) cell.getOrElse(Stats(stats.name, stats.load, ts))
} }
@ -131,7 +131,7 @@ class Report(
chartUrl chartUrl
} }
def formatResultsTable(statsSeq: Seq[Stats]): String = { def formatResultsTable(statsSeq: immutable.Seq[Stats]): String = {
val name = statsSeq.head.name val name = statsSeq.head.name

View file

@ -19,7 +19,7 @@ class CustomRouteSpec extends AkkaSpec {
provider.createRoutees(1) 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) case (sender, message) toAll(sender, provider.routees)
} }
} }
@ -43,8 +43,8 @@ class CustomRouteSpec extends AkkaSpec {
val r = Await.result(router.ask(CurrentRoutees)(1 second). val r = Await.result(router.ask(CurrentRoutees)(1 second).
mapTo[RouterRoutees], 1 second) mapTo[RouterRoutees], 1 second)
r.routees.size must be(1) r.routees.size must be(1)
route(testActor -> "hallo") must be(Seq(Destination(testActor, target))) route(testActor -> "hallo") must be(List(Destination(testActor, target)))
route(testActor -> 12) must be(Seq(Destination(testActor, r.routees.head))) route(testActor -> 12) must be(List(Destination(testActor, r.routees.head)))
//#test-route //#test-route
} }

View file

@ -10,10 +10,9 @@ import akka.testkit.TestEvent._
import akka.actor.Props import akka.actor.Props
import scala.concurrent.Await import scala.concurrent.Await
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
import akka.actor.ActorRef import akka.actor.ActorRef
import java.util.concurrent.atomic.AtomicInteger
import akka.pattern.ask import akka.pattern.ask
import java.util.concurrent.TimeoutException
import scala.util.Try import scala.util.Try
object ResizerSpec { object ResizerSpec {
@ -61,10 +60,10 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
lowerBound = 2, lowerBound = 2,
upperBound = 3) upperBound = 3)
val c1 = resizer.capacity(IndexedSeq.empty[ActorRef]) val c1 = resizer.capacity(immutable.IndexedSeq.empty[ActorRef])
c1 must be(2) 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) val c2 = resizer.capacity(current)
c2 must be(0) c2 must be(0)
} }

View file

@ -5,9 +5,8 @@ package akka.routing
import language.postfixOps import language.postfixOps
import java.util.concurrent.atomic.AtomicInteger
import akka.actor._ import akka.actor._
import scala.collection.mutable.LinkedList import scala.collection.immutable
import akka.testkit._ import akka.testkit._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Await import scala.concurrent.Await
@ -17,7 +16,9 @@ import akka.pattern.{ ask, pipe }
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.util.Collections.EmptyImmutableSeq
import akka.util.Timeout import akka.util.Timeout
import java.util.concurrent.atomic.AtomicInteger
object RoutingSpec { object RoutingSpec {
@ -54,11 +55,10 @@ object RoutingSpec {
class MyRouter(config: Config) extends RouterConfig { class MyRouter(config: Config) extends RouterConfig {
val foo = config.getString("foo") val foo = config.getString("foo")
def createRoute(routeeProvider: RouteeProvider): Route = { def createRoute(routeeProvider: RouteeProvider): Route = {
val routees = IndexedSeq(routeeProvider.context.actorOf(Props[Echo])) routeeProvider.registerRoutees(List(routeeProvider.context.actorOf(Props[Echo])))
routeeProvider.registerRoutees(routees)
{ {
case (sender, message) Nil case (sender, message) EmptyImmutableSeq
} }
} }
def routerDispatcher: String = Dispatchers.DefaultDispatcherId def routerDispatcher: String = Dispatchers.DefaultDispatcherId
@ -101,33 +101,34 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
} }
"be able to send their routees" in { "be able to send their routees" in {
class TheActor extends Actor { case class TestRun(id: String, names: immutable.Iterable[String], actors: Int)
val routee1 = context.actorOf(Props[TestActor], "routee1") val actor = system.actorOf(Props(new Actor {
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)))
def receive = { def receive = {
case "doIt" router ! CurrentRoutees case TestRun(id, names, actors)
case routees: RouterRoutees testActor forward routees val routerProps = Props[TestActor].withRouter(
ScatterGatherFirstCompletedRouter(
routees = names map { context.actorOf(Props(new TestActor), _) },
within = 5 seconds))
1 to actors foreach { i context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) }
} }
} }))
val theActor = system.actorOf(Props(new TheActor), "theActor") val actors = 15
theActor ! "doIt" val names = 1 to 20 map { "routee" + _ } toList
val routees = expectMsgPF() {
case RouterRoutees(routees) routees.toSet
}
routees.map(_.path.name) must be(Set("routee1", "routee2", "routee3")) actor ! TestRun("test", names, actors)
1 to actors foreach { _
val routees = expectMsgType[RouterRoutees].routees
routees.map(_.path.name) must be === names
}
expectNoMsg(500.millis)
} }
"use configured nr-of-instances when FromConfig" in { "use configured nr-of-instances when FromConfig" in {
val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1") 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) watch(router)
system.stop(router) system.stop(router)
expectMsgType[Terminated] expectMsgType[Terminated]
@ -135,7 +136,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
"use configured nr-of-instances when router is specified" in { "use configured nr-of-instances when router is specified" in {
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2") 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) system.stop(router)
} }
@ -150,7 +152,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
} }
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3") val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3")
Await.ready(latch, remaining) 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) system.stop(router)
} }
@ -251,15 +254,15 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
val doneLatch = new TestLatch(connectionCount) val doneLatch = new TestLatch(connectionCount)
//lets create some connections. //lets create some connections.
var actors = new LinkedList[ActorRef] @volatile var actors = immutable.IndexedSeq[ActorRef]()
var counters = new LinkedList[AtomicInteger] @volatile var counters = immutable.IndexedSeq[AtomicInteger]()
for (i 0 until connectionCount) { for (i 0 until connectionCount) {
counters = counters :+ new AtomicInteger() counters = counters :+ new AtomicInteger()
val actor = system.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
def receive = { def receive = {
case "end" doneLatch.countDown() case "end" doneLatch.countDown()
case msg: Int counters.get(i).get.addAndGet(msg) case msg: Int counters(i).addAndGet(msg)
} }
})) }))
actors = actors :+ actor actors = actors :+ actor
@ -278,10 +281,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
//now wait some and do validations. //now wait some and do validations.
Await.ready(doneLatch, remaining) Await.ready(doneLatch, remaining)
for (i 0 until connectionCount) { for (i 0 until connectionCount)
val counter = counters.get(i).get counters(i).get must be((iterationCount * (i + 1)))
counter.get must be((iterationCount * (i + 1)))
}
} }
"deliver a broadcast message using the !" in { "deliver a broadcast message using the !" in {

View file

@ -8,10 +8,31 @@ import language.postfixOps
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Await
import java.util.concurrent.TimeUnit._ 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 { "Duration" must {

View file

@ -5,7 +5,7 @@ import scala.collection.Seq;
public class JAPI { public class JAPI {
public static <T> Seq<T> seq(T... ts) { public static <T> Seq<T> seq(T... ts) {
return Util.arrayToSeq(ts); return Util.immutableSeq(ts);
} }
} }

View file

@ -263,8 +263,11 @@ public class HashedWheelTimer implements Timer {
void scheduleTimeout(HashedWheelTimeout timeout, long delay) { void scheduleTimeout(HashedWheelTimeout timeout, long delay) {
// Prepare the required parameters to schedule the timeout object. // 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 well just schedule this timeout
// one tick early; that shouldnt matter since were talking 270 years here
if (relativeIndex < 0) relativeIndex = delay / tickDuration;
if (relativeIndex == 0) relativeIndex = 1;
final long remainingRounds = relativeIndex / wheel.length; final long remainingRounds = relativeIndex / wheel.length;
// Add the timeout to the wheel. // Add the timeout to the wheel.
@ -304,7 +307,7 @@ public class HashedWheelTimer implements Timer {
while (!shutdown()) { while (!shutdown()) {
final long deadline = waitForNextTick(); final long deadline = waitForNextTick();
if (deadline > 0) if (deadline > Long.MIN_VALUE)
notifyExpiredTimeouts(fetchExpiredTimeouts(deadline)); notifyExpiredTimeouts(fetchExpiredTimeouts(deadline));
} }
} }
@ -332,7 +335,7 @@ public class HashedWheelTimer implements Timer {
HashedWheelTimeout timeout = i.next(); HashedWheelTimeout timeout = i.next();
if (timeout.remainingRounds <= 0) { if (timeout.remainingRounds <= 0) {
i.remove(); i.remove();
if (timeout.deadline <= deadline) { if (timeout.deadline - deadline <= 0) {
expiredTimeouts.add(timeout); expiredTimeouts.add(timeout);
} else { } else {
// Handle the case where the timeout is put into a wrong // Handle the case where the timeout is put into a wrong
@ -368,6 +371,12 @@ public class HashedWheelTimer implements Timer {
expiredTimeouts.clear(); 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() { private long waitForNextTick() {
long deadline = startTime + tickDuration * tick; long deadline = startTime + tickDuration * tick;
@ -378,7 +387,8 @@ public class HashedWheelTimer implements Timer {
if (sleepTimeMs <= 0) { if (sleepTimeMs <= 0) {
tick += 1; 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 // 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); Thread.sleep(sleepTimeMs);
} catch (InterruptedException e) { } catch (InterruptedException e) {
if (shutdown()) { if (shutdown()) {
return -1; return Long.MIN_VALUE;
} }
} }
} }

View file

@ -6,7 +6,7 @@ package akka.actor
import java.io.{ ObjectOutputStream, NotSerializableException } import java.io.{ ObjectOutputStream, NotSerializableException }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.TreeSet import scala.collection.immutable
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.dungeon.ChildrenContainer import akka.actor.dungeon.ChildrenContainer
@ -76,8 +76,14 @@ trait ActorContext extends ActorRefFactory {
/** /**
* Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler. * Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler.
* Puts the behavior on top of the hotswap stack. * This method acts upon the behavior stack as follows:
* If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack *
* - 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 def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit
@ -102,7 +108,7 @@ trait ActorContext extends ActorRefFactory {
* val goodLookup = context.actorFor("kid") * val goodLookup = context.actorFor("kid")
* }}} * }}}
*/ */
def children: Iterable[ActorRef] def children: immutable.Iterable[ActorRef]
/** /**
* Get the child with the given name if it exists. * Get the child with the given name if it exists.
@ -167,14 +173,20 @@ trait UntypedActorContext extends ActorContext {
/** /**
* Changes the Actor's behavior to become the new 'Procedure' handler. * 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 def become(behavior: Procedure[Any]): Unit
/** /**
* Changes the Actor's behavior to become the new 'Procedure' handler. * Changes the Actor's behavior to become the new 'Procedure' handler.
* Puts the behavior on top of the hotswap stack. * This method acts upon the behavior stack as follows:
* If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack *
* - 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 def become(behavior: Procedure[Any], discardOld: Boolean): Unit
@ -247,12 +259,12 @@ private[akka] trait Cell {
*/ */
def isLocal: Boolean 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. * false otherwise.
*/ */
def hasMessages: Boolean 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. * which may be a costly operation, 0 otherwise.
*/ */
def numberOfMessages: Int def numberOfMessages: Int
@ -275,7 +287,7 @@ private[akka] object ActorCell {
final val emptyBehaviorStack: List[Actor.Receive] = Nil 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) //ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit)

View file

@ -3,6 +3,8 @@
*/ */
package akka.actor package akka.actor
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable
import akka.japi.Util.immutableSeq
import java.net.MalformedURLException import java.net.MalformedURLException
object ActorPath { object ActorPath {
@ -20,6 +22,8 @@ object ActorPath {
* http://www.ietf.org/rfc/rfc2396.txt * http://www.ietf.org/rfc/rfc2396.txt
*/ */
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r 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 descendants path by appending all child names. * ''Java API'': Recursively create a descendants path by appending all child names.
*/ */
def descendant(names: java.lang.Iterable[String]): ActorPath = { def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
import scala.collection.JavaConverters._
/(names.asScala)
}
/** /**
* Sequence of names for this path from root to this. Performance implication: has to allocate a list. * 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. * ''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] = { def getElements: java.lang.Iterable[String] =
import scala.collection.JavaConverters._ scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava
elements.asJava
}
/** /**
* Walk up the tree to obtain and return the RootActorPath. * 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 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 override val toString: String = address + name
@ -121,7 +120,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
else addr + name else addr + name
override def compareTo(other: ActorPath): Int = other match { 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 case c: ChildActorPath 1
} }
} }
@ -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 /(child: String): ActorPath = new ChildActorPath(this, child)
override def elements: Iterable[String] = { override def elements: immutable.Iterable[String] = {
@tailrec @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 r: RootActorPath acc
case _ rec(p.parent, p.name :: acc) case _ rec(p.parent, p.name :: acc)
} }

View file

@ -419,7 +419,10 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
* to the ActorSystem's EventStream * to the ActorSystem's EventStream
*/ */
@SerialVersionUID(1L) @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 { private[akka] object DeadLetterActorRef {
@SerialVersionUID(1L) @SerialVersionUID(1L)
@ -446,8 +449,11 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message) override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match { 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 d: DeadLetter
case _ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this)) 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 { protected def specialHandle(msg: Any): Boolean = msg match {
@ -530,7 +536,7 @@ private[akka] class VirtualPathContainer(
def hasChildren: Boolean = !children.isEmpty def hasChildren: Boolean = !children.isEmpty
def foreachChild(f: ActorRef Unit) = { def foreachChild(f: ActorRef Unit): Unit = {
val iter = children.values.iterator val iter = children.values.iterator
while (iter.hasNext) f(iter.next) while (iter.hasNext) f(iter.next)
} }

View file

@ -8,8 +8,9 @@ import akka.dispatch._
import akka.routing._ import akka.routing._
import akka.event._ import akka.event._
import akka.util.{ Switch, Helpers } import akka.util.{ Switch, Helpers }
import akka.japi.Util.immutableSeq
import akka.util.Collections.EmptyImmutableSeq
import scala.util.{ Success, Failure } import scala.util.{ Success, Failure }
import scala.util.control.NonFatal
import scala.concurrent.{ Future, Promise } import scala.concurrent.{ Future, Promise }
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
@ -271,10 +272,7 @@ trait ActorRefFactory {
* *
* For maximum performance use a collection with efficient head & tail operations. * For maximum performance use a collection with efficient head & tail operations.
*/ */
def actorFor(path: java.lang.Iterable[String]): ActorRef = { def actorFor(path: java.lang.Iterable[String]): ActorRef = provider.actorFor(lookupRoot, immutableSeq(path))
import scala.collection.JavaConverters._
provider.actorFor(lookupRoot, path.asScala)
}
/** /**
* Construct an [[akka.actor.ActorSelection]] from the given path, which is * 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 def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras
private def guardianSupervisorStrategyConfigurator = private def guardianSupervisorStrategyConfigurator =
dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Seq()).get dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, EmptyImmutableSeq).get
/** /**
* Overridable supervision strategy to be used by the /user guardian. * Overridable supervision strategy to be used by the /user guardian.

View file

@ -6,20 +6,20 @@ package akka.actor
import akka.event._ import akka.event._
import akka.dispatch._ import akka.dispatch._
import akka.pattern.ask import akka.japi.Util.immutableSeq
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.concurrent.duration.Duration import scala.collection.immutable
import java.io.Closeable import scala.concurrent.duration.{ FiniteDuration, Duration }
import scala.concurrent.{ Await, Awaitable, CanAwait, Future } import scala.concurrent.{ Await, Awaitable, CanAwait, Future }
import scala.util.{ Failure, Success }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.util._ import akka.util._
import java.io.Closeable
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap } import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException } import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.actor.dungeon.ChildrenContainer import akka.actor.dungeon.ChildrenContainer
import scala.concurrent.duration.FiniteDuration
import util.{ Failure, Success }
object ActorSystem { object ActorSystem {
@ -144,7 +144,7 @@ object ActorSystem {
final val LogLevel: String = getString("akka.loglevel") final val LogLevel: String = getString("akka.loglevel")
final val StdoutLogLevel: String = getString("akka.stdout-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] = immutableSeq(getStringList("akka.event-handlers"))
final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS)) final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start") final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")
@ -273,10 +273,7 @@ abstract class ActorSystem extends ActorRefFactory {
/** /**
* ''Java API'': Recursively create a descendants path by appending all child names. * ''Java API'': Recursively create a descendants path by appending all child names.
*/ */
def descendant(names: java.lang.Iterable[String]): ActorPath = { def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
import scala.collection.JavaConverters._
/(names.asScala)
}
/** /**
* Start-up time in milliseconds since the epoch. * Start-up time in milliseconds since the epoch.
@ -536,7 +533,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
val scheduler: Scheduler = createScheduler() val scheduler: Scheduler = createScheduler()
val provider: ActorRefProvider = { val provider: ActorRefProvider = {
val arguments = Seq( val arguments = Vector(
classOf[String] -> name, classOf[String] -> name,
classOf[Settings] -> settings, classOf[Settings] -> settings,
classOf[EventStream] -> eventStream, classOf[EventStream] -> eventStream,
@ -676,9 +673,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null
private def loadExtensions() { private def loadExtensions() {
import scala.collection.JavaConverters.collectionAsScalaIterableConverter immutableSeq(settings.config.getStringList("akka.extensions")) foreach { fqcn
settings.config.getStringList("akka.extensions").asScala foreach { fqcn dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match {
dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()) } match {
case Success(p: ExtensionIdProvider) registerExtension(p.lookup()) case Success(p: ExtensionIdProvider) registerExtension(p.lookup())
case Success(p: ExtensionId[_]) registerExtension(p) case Success(p: ExtensionId[_]) registerExtension(p)
case Success(other) log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn) case Success(other) log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)

View file

@ -5,7 +5,8 @@ package akka.actor
import java.net.URI import java.net.URI
import java.net.URISyntaxException import java.net.URISyntaxException
import java.net.MalformedURLException 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 * The address specifies the physical location under which an Actor can be
@ -71,7 +72,7 @@ private[akka] trait PathUtils {
} }
object RelativeActorPath extends PathUtils { object RelativeActorPath extends PathUtils {
def unapply(addr: String): Option[Iterable[String]] = { def unapply(addr: String): Option[immutable.Seq[String]] = {
try { try {
val uri = new URI(addr) val uri = new URI(addr)
if (uri.isAbsolute) None if (uri.isAbsolute) None
@ -119,13 +120,12 @@ object AddressFromURIString {
* Given an ActorPath it returns the Address and the path elements if the path is well-formed * Given an ActorPath it returns the Address and the path elements if the path is well-formed
*/ */
object ActorPathExtractor extends PathUtils { object ActorPathExtractor extends PathUtils {
def unapply(addr: String): Option[(Address, Iterable[String])] = def unapply(addr: String): Option[(Address, immutable.Iterable[String])] =
try { try {
val uri = new URI(addr) val uri = new URI(addr)
if (uri.getRawPath == null) None uri.getRawPath match {
else AddressFromURIString.unapply(uri) match { case null None
case None None case path AddressFromURIString.unapply(uri).map((_, split(path).drop(1)))
case Some(addr) Some((addr, split(uri.getRawPath).drop(1)))
} }
} catch { } catch {
case _: URISyntaxException None case _: URISyntaxException None

View file

@ -7,10 +7,11 @@ package akka.actor
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import com.typesafe.config._ import com.typesafe.config._
import akka.routing._ import akka.routing._
import akka.japi.Util.immutableSeq
import java.util.concurrent.{ TimeUnit } import java.util.concurrent.{ TimeUnit }
import akka.util.WildcardTree import akka.util.WildcardTree
import java.util.concurrent.atomic.AtomicReference 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 * 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 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") val nrOfInstances = deployment.getInt("nr-of-instances")
@ -160,7 +161,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
val vnodes = deployment.getInt("virtual-nodes-factor") val vnodes = deployment.getInt("virtual-nodes-factor")
ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes) ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes)
case fqn case fqn
val args = Seq(classOf[Config] -> deployment) val args = List(classOf[Config] -> deployment)
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({ dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({
case exception throw new IllegalArgumentException( case exception throw new IllegalArgumentException(
("Cannot instantiate router [%s], defined in [%s], " + ("Cannot instantiate router [%s], defined in [%s], " +

View file

@ -3,7 +3,7 @@
*/ */
package akka.actor package akka.actor
import scala.util.control.NonFatal import scala.collection.immutable
import java.lang.reflect.InvocationTargetException import java.lang.reflect.InvocationTargetException
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.util.Try import scala.util.Try
@ -25,7 +25,7 @@ abstract class DynamicAccess {
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name)) * 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 * 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, * `args` argument. The exact usage of args depends on which type is requested,
* see the relevant requesting code for details. * 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. * 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) 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 { Try {
val types = args.map(_._1).toArray val types = args.map(_._1).toArray
val values = args.map(_._2).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) 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 } } 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) } getClassFor(fqcn) flatMap { c createInstanceFor(c, args) }
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = { override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {

View file

@ -98,5 +98,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext
def this(clazz: Class[T]) = this()(ClassTag(clazz)) def this(clazz: Class[T]) = this()(ClassTag(clazz))
override def lookup(): ExtensionId[T] = this 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
} }

View file

@ -521,7 +521,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
* Main actor receive() method * Main actor receive() method
* ******************************************* * *******************************************
*/ */
override final def receive: Receive = { override def receive: Receive = {
case TimeoutMarker(gen) case TimeoutMarker(gen)
if (generation == gen) { if (generation == gen) {
processMsg(StateTimeout, "state timeout") processMsg(StateTimeout, "state timeout")

View file

@ -5,10 +5,13 @@ package akka.actor
import language.implicitConversions import language.implicitConversions
import java.util.concurrent.TimeUnit
import scala.collection.mutable.ArrayBuffer
import java.lang.{ Iterable JIterable } 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 import scala.concurrent.duration.Duration
/** /**
* INTERNAL API * INTERNAL API
*/ */
@ -170,7 +173,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
* Implicit conversion from `Seq` of Throwables to a `Decider`. * Implicit conversion from `Seq` of Throwables to a `Decider`.
* This maps the given Throwables to restarts, otherwise escalates. * 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 = makeDecider(trapExit)
type Decider = PartialFunction[Throwable, Directive] type Decider = PartialFunction[Throwable, Directive]
type JDecider = akka.japi.Function[Throwable, Directive] type JDecider = akka.japi.Function[Throwable, Directive]
@ -180,25 +183,16 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
* Decider builder which just checks whether one of * Decider builder which just checks whether one of
* the given Throwables matches the cause and restarts, otherwise escalates. * the given Throwables matches the cause and restarts, otherwise escalates.
*/ */
def makeDecider(trapExit: Array[Class[_]]): Decider = def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = {
{ case x if (trapExit exists (_ isInstance x)) Restart else Escalate } 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: 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 = {
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
makeDecider(trapExit.asScala.toSeq)
} }
/**
* 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 = makeDecider(immutableSeq(trapExit))
/** /**
* Decider builder for Iterables of cause-directive pairs, e.g. a map obtained * Decider builder for Iterables of cause-directive pairs, e.g. a map obtained
* from configuration; will sort the pairs so that the most specific type is * from configuration; will sort the pairs so that the most specific type is
@ -222,14 +216,14 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
* *
* INTERNAL API * 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) (new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca)
buf.indexWhere(_._1 isAssignableFrom ca._1) match { buf.indexWhere(_._1 isAssignableFrom ca._1) match {
case -1 buf append ca case -1 buf append ca
case x buf insert (x, ca) case x buf insert (x, ca)
} }
buf buf
} }.to[immutable.IndexedSeq]
private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] = private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] =
if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None
@ -338,10 +332,6 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) 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 * this is a performance optimization to avoid re-allocating the pairs upon
* every call to requestRestartPermission, assuming that strategies are shared * every call to requestRestartPermission, assuming that strategies are shared
@ -380,9 +370,6 @@ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) 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 * this is a performance optimization to avoid re-allocating the pairs upon
* every call to requestRestartPermission, assuming that strategies are shared * every call to requestRestartPermission, assuming that strategies are shared

View file

@ -6,6 +6,7 @@ package akka.actor
import language.higherKinds import language.higherKinds
import language.postfixOps import language.postfixOps
import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future } import scala.concurrent.{ ExecutionContext, Future }
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.control.NonFatal import scala.util.control.NonFatal
@ -122,7 +123,7 @@ object IO {
* @return a new SocketHandle that can be used to perform actions on the * @return a new SocketHandle that can be used to perform actions on the
* new connection's SocketChannel. * 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) val socket = SocketHandle(socketOwner, ioManager)
ioManager ! Accept(socket, this, options) ioManager ! Accept(socket, this, options)
socket socket
@ -250,7 +251,7 @@ object IO {
* *
* Normally sent using IOManager.listen() * 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 * Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is
@ -272,7 +273,7 @@ object IO {
* *
* Normally sent using [[akka.actor.IO.ServerHandle]].accept() * 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 * Message to an [[akka.actor.IOManager]] to create a SocketChannel connected
@ -280,7 +281,7 @@ object IO {
* *
* Normally sent using IOManager.connect() * 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 * 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 * @param option Seq of [[akka.actor.IO.ServerSocketOptions]] to setup on socket
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created 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) val server = IO.ServerHandle(owner, actor)
actor ! IO.Listen(server, address, options) actor ! IO.Listen(server, address, options)
server server
@ -847,7 +848,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
* @param owner the ActorRef that will receive messages from the IOManagerActor * @param owner the ActorRef that will receive messages from the IOManagerActor
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket * @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 * 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 * @param owner the ActorRef that will receive messages from the IOManagerActor
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket * @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) 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 * @param owner the ActorRef that will receive messages from the IOManagerActor
* @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket * @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) val socket = IO.SocketHandle(owner, actor)
actor ! IO.Connect(socket, address, options) actor ! IO.Connect(socket, address, options)
socket 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 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 { options foreach {
case IO.KeepAlive(on) forwardFailure(socket.setKeepAlive(on)) case IO.KeepAlive(on) forwardFailure(socket.setKeepAlive(on))
case IO.OOBInline(on) forwardFailure(socket.setOOBInline(on)) case IO.OOBInline(on) forwardFailure(socket.setOOBInline(on))

View file

@ -189,15 +189,16 @@ private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: Rep
def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer
def getChildByName(name: String): Option[ChildRestartStats] = None def getChildByName(name: String): Option[ChildRestartStats] = None
def tell(message: Any, sender: ActorRef): Unit = { def tell(message: Any, sender: ActorRef): Unit = {
val useSender = if (sender eq Actor.noSender) system.deadLetters else sender
if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) { if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
try { try {
if (self.underlying eq this) queue enqueue Envelope(message, sender, system) if (self.underlying eq this) queue enqueue Envelope(message, useSender, system)
else self.underlying.tell(message, sender) else self.underlying.tell(message, useSender)
} finally { } finally {
lock.unlock() lock.unlock()
} }
} else { } else {
system.deadLetters ! DeadLetter(message, sender, self) system.deadLetters ! DeadLetter(message, useSender, self)
} }
} }
def sendSystemMessage(msg: SystemMessage): Unit = { def sendSystemMessage(msg: SystemMessage): Unit = {

View file

@ -203,8 +203,8 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter)
} }
override def close(): Unit = { override def close(): Unit = {
import scala.collection.JavaConverters._ val i = hashedWheelTimer.stop().iterator()
hashedWheelTimer.stop().asScala foreach execDirectly while (i.hasNext) execDirectly(i.next())
} }
} }

View file

@ -16,13 +16,13 @@ import akka.AkkaException
* def receive = { * def receive = {
* case "open" * case "open"
* unstashAll() * unstashAll()
* context.become { * context.become({
* case "write" // do writing... * case "write" // do writing...
* case "close" * case "close"
* unstashAll() * unstashAll()
* context.unbecome() * context.unbecome()
* case msg stash() * case msg stash()
* } * }, discardOld = false)
* case "done" // done * case "done" // done
* case msg stash() * case msg stash()
* } * }

View file

@ -4,22 +4,25 @@
package akka.actor package akka.actor
import language.existentials 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.control.NonFatal
import scala.util.{ Try, Success, Failure }
import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
import scala.concurrent.{ Await, Future } 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.util.Reflect.instantiator
import akka.serialization.{ JavaSerializer, SerializationExtension }
import akka.dispatch._ import akka.dispatch._
import java.util.concurrent.atomic.{ AtomicReference AtomVar } import java.util.concurrent.atomic.{ AtomicReference AtomVar }
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import scala.reflect.ClassTag
import akka.serialization.{ JavaSerializer, SerializationExtension }
import java.io.ObjectStreamException import java.io.ObjectStreamException
import scala.util.{ Try, Success, Failure } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
import scala.concurrent.duration.FiniteDuration
/** /**
* A TypedActorFactory is something that can created TypedActor instances. * A TypedActorFactory is something that can created TypedActor instances.
@ -439,8 +442,8 @@ object TypedProps {
* @return a sequence of interfaces that the specified class implements, * @return a sequence of interfaces that the specified class implements,
* or a sequence containing only itself, if itself is an interface. * or a sequence containing only itself, if itself is an interface.
*/ */
def extractInterfaces(clazz: Class[_]): Seq[Class[_]] = def extractInterfaces(clazz: Class[_]): immutable.Seq[Class[_]] =
if (clazz.isInterface) Seq[Class[_]](clazz) else clazz.getInterfaces.toList if (clazz.isInterface) immutableSingletonSeq(clazz) else immutableSeq(clazz.getInterfaces)
/** /**
* Uses the supplied class as the factory for the TypedActor implementation, * Uses the supplied class as the factory for the TypedActor implementation,
@ -489,7 +492,7 @@ object TypedProps {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class TypedProps[T <: AnyRef] protected[TypedProps] ( case class TypedProps[T <: AnyRef] protected[TypedProps] (
interfaces: Seq[Class[_]], interfaces: immutable.Seq[Class[_]],
creator: () T, creator: () T,
dispatcher: String = TypedProps.defaultDispatcherId, dispatcher: String = TypedProps.defaultDispatcherId,
deploy: Deploy = Props.defaultDeploy, deploy: Deploy = Props.defaultDeploy,

View file

@ -29,7 +29,9 @@ trait Creators { this: ActorDSL.type ⇒
* for quickly trying things out in the REPL. It makes the following keywords * for quickly trying things out in the REPL. It makes the following keywords
* available: * 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` * - `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 * stack is cleared upon restart. Use `unbecome()` to pop an element off
* this stack. * 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 * Pop the active behavior from the behavior stack of this actor. This stack

View file

@ -5,14 +5,12 @@
package akka.actor.dungeon package akka.actor.dungeon
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.JavaConverters.asJavaIterableConverter
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.collection.immutable
import akka.actor._ import akka.actor._
import akka.actor.ActorCell
import akka.actor.ActorPath.ElementRegex import akka.actor.ActorPath.ElementRegex
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.util.{ Unsafe, Helpers } import akka.util.{ Unsafe, Helpers }
import akka.actor.ChildNameReserved
private[akka] trait Children { this: ActorCell private[akka] trait Children { this: ActorCell
@ -24,8 +22,9 @@ private[akka] trait Children { this: ActorCell ⇒
def childrenRefs: ChildrenContainer = def childrenRefs: ChildrenContainer =
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer] Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer]
final def children: Iterable[ActorRef] = childrenRefs.children final def children: immutable.Iterable[ActorRef] = childrenRefs.children
final def getChildren(): java.lang.Iterable[ActorRef] = children.asJava 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 child(name: String): Option[ActorRef] = Option(getChild(name))
final def getChild(name: String): ActorRef = childrenRefs.getByName(name) match { 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 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] = { protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = {
childrenRefs match { childrenRefs match {

View file

@ -4,10 +4,11 @@
package akka.actor.dungeon package akka.actor.dungeon
import scala.collection.immutable.TreeMap import scala.collection.immutable
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef } import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
import akka.dispatch.SystemMessage import akka.dispatch.SystemMessage
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
/** /**
* INTERNAL API * INTERNAL API
@ -20,8 +21,8 @@ private[akka] trait ChildrenContainer {
def getByName(name: String): Option[ChildStats] def getByName(name: String): Option[ChildStats]
def getByRef(actor: ActorRef): Option[ChildRestartStats] def getByRef(actor: ActorRef): Option[ChildRestartStats]
def children: Iterable[ActorRef] def children: immutable.Iterable[ActorRef]
def stats: Iterable[ChildRestartStats] def stats: immutable.Iterable[ChildRestartStats]
def shallDie(actor: ActorRef): ChildrenContainer def shallDie(actor: ActorRef): ChildrenContainer
@ -49,6 +50,18 @@ private[akka] object ChildrenContainer {
case class Creation() extends SuspendReason with WaitingForChildren case class Creation() extends SuspendReason with WaitingForChildren
case object Termination extends SuspendReason 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 { trait WaitingForChildren {
private var todo: SystemMessage = null private var todo: SystemMessage = null
def enqueue(message: SystemMessage) = { message.next = todo; todo = message } def enqueue(message: SystemMessage) = { message.next = todo; todo = message }
@ -56,13 +69,13 @@ private[akka] object ChildrenContainer {
} }
trait EmptyChildrenContainer extends 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 add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, stats))
override def remove(child: ActorRef): ChildrenContainer = this override def remove(child: ActorRef): ChildrenContainer = this
override def getByName(name: String): Option[ChildRestartStats] = None override def getByName(name: String): Option[ChildRestartStats] = None
override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
override def children: Iterable[ActorRef] = Nil override def children: immutable.Iterable[ActorRef] = EmptyImmutableSeq
override def stats: Iterable[ChildRestartStats] = Nil override def stats: immutable.Iterable[ChildRestartStats] = EmptyImmutableSeq
override def shallDie(actor: ActorRef): ChildrenContainer = this override def shallDie(actor: ActorRef): ChildrenContainer = this
override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved)) override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
override def unreserve(name: String): ChildrenContainer = this override def unreserve(name: String): ChildrenContainer = this
@ -95,7 +108,7 @@ private[akka] object ChildrenContainer {
* calling context.stop(child) and processing the ChildTerminated() system * calling context.stop(child) and processing the ChildTerminated() system
* message). * 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)) override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(c.updated(name, stats))
@ -108,9 +121,11 @@ private[akka] object ChildrenContainer {
case _ None 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) override def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
@ -130,7 +145,7 @@ private[akka] object ChildrenContainer {
} }
object NormalChildrenContainer { object NormalChildrenContainer {
def apply(c: TreeMap[String, ChildStats]): ChildrenContainer = def apply(c: immutable.TreeMap[String, ChildStats]): ChildrenContainer =
if (c.isEmpty) EmptyChildrenContainer if (c.isEmpty) EmptyChildrenContainer
else new NormalChildrenContainer(c) 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 * type of container, depending on whether or not children are left and whether or not
* the reason was Terminating. * 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 { extends ChildrenContainer {
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats)) override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats))
@ -166,9 +181,11 @@ private[akka] object ChildrenContainer {
case _ None 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) override def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)

View file

@ -10,7 +10,7 @@ import akka.dispatch._
import akka.event.Logging.{ Warning, Error, Debug } import akka.event.Logging.{ Warning, Error, Debug }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.event.Logging import akka.event.Logging
import scala.Some import scala.collection.immutable
import akka.dispatch.ChildTerminated import akka.dispatch.ChildTerminated
import akka.actor.PreRestartException import akka.actor.PreRestartException
import akka.actor.Failed 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)) publish(Error(t, self.path.toString, clazz(actor), message))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
if (!isFailed) try { if (!isFailed) try {

View file

@ -420,7 +420,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
case "unbounded" UnboundedMailbox() case "unbounded" UnboundedMailbox()
case "bounded" new BoundedMailbox(prerequisites.settings, config) case "bounded" new BoundedMailbox(prerequisites.settings, config)
case fqcn 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({ prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({
case exception case exception
throw new IllegalArgumentException( 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 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 "thread-pool-executor" new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
case fqcn case fqcn
val args = Seq( val args = List(
classOf[Config] -> config, classOf[Config] -> config,
classOf[DispatcherPrerequisites] -> prerequisites) classOf[DispatcherPrerequisites] -> prerequisites)
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({ prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({

View file

@ -147,7 +147,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
case "BalancingDispatcher" new BalancingDispatcherConfigurator(cfg, prerequisites) case "BalancingDispatcher" new BalancingDispatcherConfigurator(cfg, prerequisites)
case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites) case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites)
case fqn case fqn
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) val args = List(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({ prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({
case exception case exception
throw new IllegalArgumentException( throw new IllegalArgumentException(

View file

@ -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]] = { def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = {
implicit val ec = executor 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
} }
/** /**

View file

@ -10,6 +10,7 @@ import java.util.concurrent.ConcurrentSkipListSet
import java.util.Comparator import java.util.Comparator
import akka.util.{ Subclassification, SubclassifiedIndex } import akka.util.{ Subclassification, SubclassifiedIndex }
import scala.collection.immutable.TreeSet import scala.collection.immutable.TreeSet
import scala.collection.immutable
/** /**
* Represents the base type for EventBuses * Represents the base type for EventBuses
@ -167,12 +168,12 @@ trait SubchannelClassification { this: EventBus ⇒
recv foreach (publish(event, _)) 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) { cache = (cache /: changes) {
case (m, (c, cs)) m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) -- cs) 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) { cache = (cache /: changes) {
case (m, (c, cs)) m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs) case (m, (c, cs)) m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs)
} }
@ -265,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 @tailrec
def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = { def dissociateAsMonitored(monitored: ActorRef): immutable.Iterable[ActorRef] = {
val current = mappings get monitored val current = mappings get monitored
current match { current match {
case null empty case null empty

View file

@ -9,12 +9,13 @@ import akka.actor._
import akka.{ ConfigurationException, AkkaException } import akka.{ ConfigurationException, AkkaException }
import akka.actor.ActorSystem.Settings import akka.actor.ActorSystem.Settings
import akka.util.{ Timeout, ReentrantGuard } import akka.util.{ Timeout, ReentrantGuard }
import scala.concurrent.duration._
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.util.control.NoStackTrace
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import scala.annotation.implicitNotFound
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.Await 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 * 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 // 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 * Obtain LoggingAdapter for the given actor system and source object. This
@ -877,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)) protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
} }
private[akka] object NoLogging extends LoggingAdapter { /**
def isErrorEnabled = false * NoLogging is a LoggingAdapter that does absolutely nothing no logging at all.
def isWarningEnabled = false */
def isInfoEnabled = false object NoLogging extends LoggingAdapter {
def isDebugEnabled = false
protected def notifyError(message: String): Unit = () /**
protected def notifyError(cause: Throwable, message: String): Unit = () * Java API to return the reference to NoLogging
protected def notifyWarning(message: String): Unit = () * @return The NoLogging instance
protected def notifyInfo(message: String): Unit = () */
protected def notifyDebug(message: String): Unit = () 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 = ()
} }

View file

@ -5,10 +5,12 @@
package akka.japi package akka.japi
import language.implicitConversions import language.implicitConversions
import scala.Some
import scala.collection.immutable
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import scala.runtime.AbstractPartialFunction import scala.runtime.AbstractPartialFunction
import akka.util.Collections.EmptyImmutableSeq
import java.util.Collections.{ emptyList, singletonList } import java.util.Collections.{ emptyList, singletonList }
/** /**
@ -174,9 +176,40 @@ object Option {
* This class hold common utilities for Java * This class hold common utilities for Java
*/ */
object Util { object Util {
/**
* Returns a ClassTag describing the provided Class.
*
* Java API
*/
def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz) def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq /**
* 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[_]]): Seq[Class[_]] = classes.toSeq /**
*
*/
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
} }

View file

@ -4,7 +4,7 @@
package akka.routing package akka.routing
import scala.collection.immutable.SortedMap import scala.collection.immutable
import scala.reflect.ClassTag import scala.reflect.ClassTag
import java.util.Arrays import java.util.Arrays
@ -18,7 +18,7 @@ import java.util.Arrays
* hash, i.e. make sure it is different for different nodes. * 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._ import ConsistentHash._
@ -106,7 +106,7 @@ class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtual
object ConsistentHash { object ConsistentHash {
def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = { 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)), (for (node nodes; vnode 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)),
virtualNodesFactor) virtualNodesFactor)
} }
@ -120,8 +120,10 @@ object ConsistentHash {
apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]])) apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]]))
} }
private def nodeHashFor(node: Any, vnode: Int): Int = private def nodeHashFor(node: Any, vnode: Int): Int = {
hashFor((node + ":" + vnode).getBytes("UTF-8")) val baseStr = node.toString + ":"
hashFor(baseStr + vnode)
}
private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes) private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes)

View file

@ -3,30 +3,29 @@
*/ */
package akka.routing package akka.routing
import scala.collection.JavaConverters.iterableAsScalaIterableConverter import scala.collection.immutable
import akka.japi.Util.immutableSeq
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.actor.Props
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.event.Logging import akka.event.Logging
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.Address
import akka.actor.ExtendedActorSystem
object ConsistentHashingRouter { object ConsistentHashingRouter {
/** /**
* Creates a new ConsistentHashingRouter, routing to the specified routees * 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)) new ConsistentHashingRouter(routees = routees map (_.path.toString))
/** /**
* Java API to create router with the supplied 'routees' actors. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = { def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees))
import scala.collection.JavaConverters._
apply(routees.asScala)
}
/** /**
* If you don't define the `hashMapping` when * If you don't define the `hashMapping` when
@ -144,7 +143,7 @@ object ConsistentHashingRouter {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class ConsistentHashingRouter( 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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy, val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy,
val virtualNodesFactor: Int = 0, val virtualNodesFactor: Int = 0,
@ -163,7 +162,7 @@ case class ConsistentHashingRouter(
* @param routeePaths string representation of the actor paths of the routees that will be looked up * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * Constructor that sets the resizer to be used.
@ -225,7 +224,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
def virtualNodesFactor: Int def virtualNodesFactor: Int
@ -238,20 +237,22 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
} }
val log = Logging(routeeProvider.context.system, routeeProvider.context.self) val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.rootPath.address
val vnodes = val vnodes =
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
else virtualNodesFactor else virtualNodesFactor
// tuple of routees and the ConsistentHash, updated together in updateConsistentHash // 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() updateConsistentHash()
// update consistentHash when routees has changed // update consistentHash when routees has changed
// changes to routees are rare and when no changes this is a quick operation // 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 oldConsistentHashTuple = consistentHashRef.get
val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple
val currentRoutees = routeeProvider.routees val currentRoutees = routeeProvider.routees map { ConsistentActorRef(_, selfAddress) }
if (currentRoutees ne oldConsistentHashRoutees) { if (currentRoutees ne oldConsistentHashRoutees) {
// when other instance, same content, no need to re-hash, but try to set routees // when other instance, same content, no need to re-hash, but try to set routees
val consistentHash = val consistentHash =
@ -267,9 +268,9 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
val currentConsistenHash = updateConsistentHash() val currentConsistenHash = updateConsistentHash()
if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters
else hashData match { else hashData match {
case bytes: Array[Byte] currentConsistenHash.nodeFor(bytes) case bytes: Array[Byte] currentConsistenHash.nodeFor(bytes).actorRef
case str: String currentConsistenHash.nodeFor(str) case str: String currentConsistenHash.nodeFor(str).actorRef
case x: AnyRef currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get) case x: AnyRef currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get).actorRef
} }
} catch { } catch {
case NonFatal(e) case NonFatal(e)
@ -294,4 +295,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
}
}
} }

View file

@ -5,18 +5,20 @@ package akka.routing
import language.implicitConversions import language.implicitConversions
import language.postfixOps import language.postfixOps
import akka.actor._
import scala.concurrent.duration._ import scala.collection.immutable
import akka.ConfigurationException
import akka.pattern.pipe
import com.typesafe.config.Config
import scala.collection.JavaConverters.iterableAsScalaIterableConverter 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.atomic.{ AtomicLong, AtomicBoolean }
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.dispatch.Dispatchers
import scala.annotation.tailrec 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 * 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 private val resizeCounter = new AtomicLong
@volatile @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 def routees = _routees
@volatile @volatile
@ -75,14 +77,11 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
* end of construction * end of construction
*/ */
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match { def applyRoute(sender: ActorRef, message: Any): immutable.Iterable[Destination] = message match {
case _: AutoReceivedMessage Destination(self, self) :: Nil case _: AutoReceivedMessage Destination(self, self) :: Nil
case CurrentRoutees case CurrentRoutees sender ! RouterRoutees(_routees); Nil
sender ! RouterRoutees(_routees) case msg if route.isDefinedAt(sender, msg) route(sender, message)
Nil case _ Nil
case _
if (route.isDefinedAt(sender, message)) route(sender, message)
else 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 * Not thread safe, but intended to be called from protected points, such as
* `RouterConfig.createRoute` and `Resizer.resize` * `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 _routees = _routees ++ newRoutees
// subscribe to Terminated messages for all route destinations, to be handled by Router actor // subscribe to Terminated messages for all route destinations, to be handled by Router actor
newRoutees foreach watch 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 * Not thread safe, but intended to be called from protected points, such as
* `Resizer.resize` * `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) } _routees = abandonedRoutees.foldLeft(_routees) { (xs, x) unwatch(x); xs.filterNot(_ == x) }
} }
@ -195,7 +194,7 @@ trait RouterConfig {
*/ */
def withFallback(other: RouterConfig): RouterConfig = this 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, _)) 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. * 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 * Not thread safe, but intended to be called from protected points, such as
* `RouterConfig.createRoute` and `Resizer.resize`. * `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. * 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`. * `RouterConfig.createRoute` and `Resizer.resize`.
* Java API. * 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. * 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 * Not thread safe, but intended to be called from protected points, such as
* `Resizer.resize`. * `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. * 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`. * `Resizer.resize`.
* JAVA API * 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. * 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. * Looks up routes with specified paths and registers them.
* JAVA API * 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. * Creates new routees from specified `Props` and registers them.
*/ */
def createRoutees(nrOfInstances: Int): Unit = { def createRoutees(nrOfInstances: Int): Unit =
if (nrOfInstances <= 0) throw new IllegalArgumentException( if (nrOfInstances <= 0) throw new IllegalArgumentException("Must specify nrOfInstances or routees for [%s]" format context.self.path.toString)
"Must specify nrOfInstances or routees for [%s]" format context.self.path.toString) else registerRoutees(immutable.IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
else
registerRoutees(IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
}
/** /**
* Remove specified number of routees by unregister them * 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 * Give concurrent messages a chance to be placed in mailbox before
* sending PoisonPill. * 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 (abandon.nonEmpty) {
if (stopDelay <= Duration.Zero) { if (stopDelay <= Duration.Zero) {
abandon foreach (_ ! PoisonPill) abandon foreach (_ ! PoisonPill)
@ -314,7 +310,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
/** /**
* All routees of the router * All routees of the router
*/ */
def routees: IndexedSeq[ActorRef] = routedCell.routees def routees: immutable.IndexedSeq[ActorRef] = routedCell.routees
/** /**
* All routees of the router * All routees of the router
@ -335,7 +331,7 @@ abstract class CustomRouterConfig extends RouterConfig {
val customRoute = createCustomRoute(routeeProvider) 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 { 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) if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProvider)) finally ab.set(false)
case Terminated(child) case Terminated(child)
ref.removeRoutees(IndexedSeq(child)) ref.removeRoutees(child :: Nil)
if (ref.routees.isEmpty) context.stop(self) if (ref.routees.isEmpty) context.stop(self)
}: Receive) orElse routerReceive }: 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. * Message used to carry information about what routees the router is currently using.
*/ */
@SerialVersionUID(1L) @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, * 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 * 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)) new RoundRobinRouter(routees = routees map (_.path.toString))
/** /**
* Java API to create router with the supplied 'routees' actors. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = { def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter =
import scala.collection.JavaConverters._ apply(immutableSeq(routees))
apply(routees.asScala)
}
} }
/** /**
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort. * 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]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
extends RouterConfig with RoundRobinLike { 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * Constructor that sets the resizer to be used.
@ -602,7 +602,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
def createRoute(routeeProvider: RouteeProvider): Route = { def createRoute(routeeProvider: RouteeProvider): Route = {
if (resizer.isEmpty) { if (resizer.isEmpty) {
@ -622,7 +622,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
case (sender, message) case (sender, message)
message match { message match {
case Broadcast(msg) toAll(sender, routeeProvider.routees) 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 * 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. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = { def create(routees: java.lang.Iterable[ActorRef]): RandomRouter =
import scala.collection.JavaConverters._ apply(immutableSeq(routees))
apply(routees.asScala)
}
} }
/** /**
* A Router that randomly selects one of the target connections to send a message to. * 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]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
extends RouterConfig with RandomLike { 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * 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 trait RandomLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
def createRoute(routeeProvider: RouteeProvider): Route = { def createRoute(routeeProvider: RouteeProvider): Route = {
if (resizer.isEmpty) { if (resizer.isEmpty) {
@ -756,7 +754,7 @@ trait RandomLike { this: RouterConfig ⇒
case (sender, message) case (sender, message)
message match { message match {
case Broadcast(msg) toAll(sender, routeeProvider.routees) 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 * 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)) new SmallestMailboxRouter(routees = routees map (_.path.toString))
/** /**
* Java API to create router with the supplied 'routees' actors. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter = { def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter =
import scala.collection.JavaConverters._ apply(immutableSeq(routees))
apply(routees.asScala)
}
} }
/** /**
* A Router that tries to send to the non-suspended routee with fewest messages in mailbox. * 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]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
extends RouterConfig with SmallestMailboxLike { 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * 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 trait SmallestMailboxLike { this: RouterConfig
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
/** /**
* Returns true if the actor is currently processing a message. * 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 // 4. An ActorRef with unknown mailbox size that isn't processing anything
// 5. An ActorRef with a known mailbox size // 5. An ActorRef with a known mailbox size
// 6. An ActorRef without any messages // 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, proposedTarget: ActorRef = routeeProvider.context.system.deadLetters,
currentScore: Long = Long.MaxValue, currentScore: Long = Long.MaxValue,
at: Int = 0, at: Int = 0,
@ -985,7 +981,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
case (sender, message) case (sender, message)
message match { message match {
case Broadcast(msg) toAll(sender, routeeProvider.routees) 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 * 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. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = { def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter =
import scala.collection.JavaConverters._ apply(immutableSeq(routees))
apply(routees.asScala)
}
} }
/** /**
* A Router that uses broadcasts a message to all its connections. * A Router that uses broadcasts a message to all its connections.
@ -1047,7 +1041,7 @@ object BroadcastRouter {
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
extends RouterConfig with BroadcastLike { 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * Constructor that sets the resizer to be used.
@ -1102,7 +1096,7 @@ trait BroadcastLike { this: RouterConfig ⇒
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
def createRoute(routeeProvider: RouteeProvider): Route = { def createRoute(routeeProvider: RouteeProvider): Route = {
if (resizer.isEmpty) { if (resizer.isEmpty) {
@ -1120,16 +1114,14 @@ object ScatterGatherFirstCompletedRouter {
/** /**
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration * 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) new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
/** /**
* Java API to create router with the supplied 'routees' actors. * Java API to create router with the supplied 'routees' actors.
*/ */
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = { def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
import scala.collection.JavaConverters._ apply(immutableSeq(routees), within)
apply(routees.asScala, within)
}
} }
/** /**
* Simple router that broadcasts the message to all routees, and replies with the first response. * 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]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
@SerialVersionUID(1L) @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, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId, val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy) 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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. * Constructor that sets the resizer to be used.
@ -1234,7 +1226,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] def routees: immutable.Iterable[String]
def within: FiniteDuration def within: FiniteDuration
@ -1394,7 +1386,7 @@ case class DefaultResizer(
* @param routees The current actor in the resizer * @param routees The current actor in the resizer
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero) * @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 currentSize = routees.size
val press = pressure(routees) val press = pressure(routees)
val delta = filter(press, currentSize) val delta = filter(press, currentSize)
@ -1422,7 +1414,7 @@ case class DefaultResizer(
* @param routees the current resizer of routees * @param routees the current resizer of routees
* @return number of busy routees, between 0 and routees.size * @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 { routees count {
case a: ActorRefWithCell case a: ActorRefWithCell
a.underlying match { a.underlying match {

View file

@ -4,10 +4,12 @@
package akka package akka
import scala.collection.immutable
package object routing { package object routing {
/** /**
* Routing logic, partial function from (sender, message) to a * Routing logic, partial function from (sender, message) to a
* set of destinations. * set of destinations.
*/ */
type Route = PartialFunction[(akka.actor.ActorRef, Any), Iterable[Destination]] type Route = PartialFunction[(akka.actor.ActorRef, Any), immutable.Iterable[Destination]]
} }

View file

@ -4,14 +4,14 @@
package akka.serialization package akka.serialization
import akka.AkkaException
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess } import akka.actor.{ Extension, ExtendedActorSystem, Address }
import akka.event.Logging import akka.event.Logging
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import java.io.NotSerializableException import java.io.NotSerializableException
import util.{ Try, DynamicVariable } import scala.util.{ Try, DynamicVariable }
import scala.collection.immutable
object Serialization { object Serialization {
@ -27,17 +27,13 @@ object Serialization {
val currentTransportAddress = new DynamicVariable[Address](null) val currentTransportAddress = new DynamicVariable[Address](null)
class Settings(val config: Config) { 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._ private final def configToMap(path: String): Map[String, String] = {
import config._ import scala.collection.JavaConverters._
config.getConfig(path).root.unwrapped.asScala.mapValues(_.toString).toMap
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) }
} }
} }
@ -62,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. * 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. * Returns either the resulting object or an Exception if one was thrown.
*/ */
def deserialize(bytes: Array[Byte], def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]]): Try[AnyRef] =
serializerId: Int, Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
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. * 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. * You can specify an optional ClassLoader to load the object into.
* Returns either the resulting object or an Exception if one was thrown. * 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. * Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
@ -95,9 +91,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
*/ */
def serializerFor(clazz: Class[_]): Serializer = def serializerFor(clazz: Class[_]): Serializer =
serializerMap.get(clazz) match { serializerMap.get(clazz) match {
case null case null // bindings are ordered from most specific to least specific
// bindings are ordered from most specific to least specific def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean =
def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean =
possibilities.size == 1 || possibilities.size == 1 ||
(possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) || (possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) ||
(possibilities forall (_._2 == possibilities(0)._2)) (possibilities forall (_._2 == possibilities(0)._2))
@ -122,8 +117,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* loading is performed by the systems [[akka.actor.DynamicAccess]]. * loading is performed by the systems [[akka.actor.DynamicAccess]].
*/ */
def serializerOf(serializerFQN: String): Try[Serializer] = def serializerOf(serializerFQN: String): Try[Serializer] =
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) recoverWith { system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] -> system)) recoverWith {
case _ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()) case _ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil)
} }
/** /**
@ -137,21 +132,21 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* bindings is a Seq of tuple representing the mapping from Class to Serializer. * 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. * It is primarily ordered by the most specific classes first, and secondly in the configured order.
*/ */
private[akka] val bindings: Seq[ClassSerializer] = 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))) 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 * Sort so that subtypes always precede their supertypes, but without
* obeying any order between unrelated subtypes (insert sort). * obeying any order between unrelated subtypes (insert sort).
*/ */
private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] = private def sort(in: Iterable[ClassSerializer]): immutable.Seq[ClassSerializer] =
(new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ((new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca)
buf.indexWhere(_._1 isAssignableFrom ca._1) match { buf.indexWhere(_._1 isAssignableFrom ca._1) match {
case -1 buf append ca case -1 buf append ca
case x buf insert (x, ca) case x buf insert (x, ca)
} }
buf buf
} }).to[immutable.Seq]
/** /**
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer * serializerMap is a Map whose keys is the class that is serializable and values is the serializer

View file

@ -0,0 +1,54 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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
}
}

View file

@ -1,45 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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)
}
}

View file

@ -75,7 +75,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
def findValue(key: K)(f: (V) Boolean): Option[V] = def findValue(key: K)(f: (V) Boolean): Option[V] =
container get key match { container get key match {
case null None case null None
case set set.iterator.asScala.find(f) case set set.iterator.asScala find f
} }
/** /**

View file

@ -3,6 +3,8 @@
*/ */
package akka.util package akka.util
import scala.collection.immutable
/** /**
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`! * Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
*/ */
@ -55,7 +57,7 @@ private[akka] object SubclassifiedIndex {
} }
private[SubclassifiedIndex] def emptyMergeMap[K, V] = internalEmptyMergeMap.asInstanceOf[Map[K, Set[V]]] 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]())
} }
/** /**
@ -74,7 +76,7 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V]
import SubclassifiedIndex._ import SubclassifiedIndex._
type Changes = Seq[(K, Set[V])] type Changes = immutable.Seq[(K, Set[V])]
protected var subkeys = Vector.empty[Nonroot[K, 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 = private def mergeChangesByKey(changes: Changes): Changes =
(emptyMergeMap[K, V] /: changes) { (emptyMergeMap[K, V] /: changes) {
case (m, (k, s)) m.updated(k, m(k) ++ s) case (m, (k, s)) m.updated(k, m(k) ++ s)
}.toSeq }.to[immutable.Seq]
} }

View file

@ -5,27 +5,9 @@
package akka.util; package akka.util;
import java.lang.reflect.Field;
/** /**
* INTERNAL API * INTERNAL API
*/ */
public final class Unsafe { public final class Unsafe {
public final static sun.misc.Unsafe instance; public final static sun.misc.Unsafe instance = scala.concurrent.util.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);
}
}
} }

View file

@ -4,16 +4,15 @@
package akka.camel package akka.camel
import internal._ import akka.camel.internal._
import akka.actor._ import akka.actor._
import akka.ConfigurationException
import org.apache.camel.ProducerTemplate import org.apache.camel.ProducerTemplate
import org.apache.camel.impl.DefaultCamelContext import org.apache.camel.impl.DefaultCamelContext
import org.apache.camel.model.RouteDefinition import org.apache.camel.model.RouteDefinition
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.ConfigurationException import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.concurrent.duration.Duration
import java.util.concurrent.TimeUnit._ import java.util.concurrent.TimeUnit._
import scala.concurrent.duration.FiniteDuration
/** /**
* Camel trait encapsulates the underlying camel machinery. * 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 StreamingCache: Boolean = config.getBoolean("akka.camel.streamingCache")
final val Conversions: (String, RouteDefinition) RouteDefinition = { final val Conversions: (String, RouteDefinition) RouteDefinition = {
import scala.collection.JavaConverters.asScalaSetConverter
val specifiedConversions = { val specifiedConversions = {
import scala.collection.JavaConverters.asScalaSetConverter
val section = config.getConfig("akka.camel.conversions") val section = config.getConfig("akka.camel.conversions")
section.entrySet.asScala.map(e (e.getKey, section.getString(e.getKey))) section.entrySet.asScala.map(e (e.getKey, section.getString(e.getKey)))
} }

View file

@ -1,13 +1,18 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.camel package akka.camel
import language.postfixOps
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import scala.concurrent.{ Promise, Await, Future }
import scala.collection.immutable
import akka.camel.TestSupport.NonSharedCamelSystem import akka.camel.TestSupport.NonSharedCamelSystem
import akka.actor.{ ActorRef, Props, Actor } import akka.actor.{ ActorRef, Props, Actor }
import akka.routing.BroadcastRouter import akka.routing.BroadcastRouter
import concurrent.{ Promise, Await, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.Timeout import akka.util.Timeout
import org.apache.camel.model.RouteDefinition 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) activations.size must be(2 * number * number)
// must be the size of the activated activated producers and consumers // must be the size of the activated activated producers and consumers
deactivations.size must be(2 * number * number) 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[_])) { def assertContainsSameElements(lists: (Seq[_], Seq[_])) {
val (a, b) = lists val (a, b) = lists
a.intersect(b).size must be(a.size) a.intersect(b).size must be(a.size)

View file

@ -75,7 +75,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
val failureDetector: FailureDetector = { val failureDetector: FailureDetector = {
import settings.{ FailureDetectorImplementationClass fqcn } import settings.{ FailureDetectorImplementationClass fqcn }
system.dynamicAccess.createInstanceFor[FailureDetector]( 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) case e throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString)
}).get }).get
} }

View file

@ -9,8 +9,8 @@ import scala.concurrent.duration._
import scala.collection.immutable.{ SortedSet, Map } import scala.collection.immutable.{ SortedSet, Map }
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import scala.util.{ Try, Success, Failure } import scala.util.{ Try, Success, Failure }
import scala.math.ScalaNumericConversions import scala.math.ScalaNumericAnyConversions
import scala.runtime.{ RichLong, RichDouble, RichInt } import runtime.{ ScalaNumberProxy, RichLong, RichDouble, RichInt }
import akka.actor._ import akka.actor._
import akka.event.LoggingAdapter 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 * @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 { 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. * 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. * and defer to BigInt or BigDecimal.
* *
* @param xn the new data point * @param xn the new data point
* @return an new [[akka.cluster.DataStream]] with the updated yn and timestamp * @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), nl copy(ewma = BigInt(α * nl + 1 - α * ewma.longValue()), timestamp = newTimestamp),
nd copy(ewma = BigDecimal(α * nd + 1 - α * ewma.doubleValue()), 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 { 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 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 * @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. * 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 { 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 * 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. * 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 Some(v) if defined(v) Metric(name, value, None)
case _ Metric(name, None, None) case _ Metric(name, None, None)
} }
@ -409,13 +409,13 @@ private[cluster] trait MetricNumericConverter {
* <ul><li>JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned</li> * <ul><li>JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned</li>
* <li>SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)</li></ul> * <li>SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)</li></ul>
*/ */
def defined(value: ScalaNumericConversions): Boolean = def defined(value: ScalaNumericAnyConversions): Boolean =
convert(value) fold (a value.underlying != -1, b !(b.isNaN || b.isInfinite)) convert(value) fold (a value.underlying != -1, b !(b.isNaN || b.isInfinite))
/** /**
* May involve rounding or truncation. * 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: BigInt Left(n.longValue())
case n: BigDecimal Right(n.doubleValue()) case n: BigDecimal Right(n.doubleValue())
case n: RichInt Left(n.abs) case n: RichInt Left(n.abs)
@ -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. * (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 Metric with
* undefined value is returned.
* Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default. * 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( def systemLoadAverage: Metric = Metric("system-load-average",
LoadAverage.get.invoke(sigar.get).asInstanceOf[Array[Double]].toSeq.head) getOrElse osMBean.getSystemLoadAverage))) 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 * (JMX) Returns the number of available processors
@ -552,7 +557,7 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe
*/ */
private[cluster] object MetricsCollector { private[cluster] object MetricsCollector {
def apply(address: Address, log: LoggingAdapter, dynamicAccess: DynamicAccess): 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 Success(identity) new MetricsCollector(Some(identity), address)
case Failure(e) case Failure(e)
log.debug(e.toString) log.debug(e.toString)

View file

@ -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 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 { final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { case AddressFromURIString(addr) addr }.toIndexedSeq
case AddressFromURIString(addr) addr
}.toIndexedSeq
final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS) 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 PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS) final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)

View file

@ -5,16 +5,13 @@ package akka.cluster.routing
import java.lang.IllegalStateException import java.lang.IllegalStateException
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.collection.immutable.SortedSet import scala.collection.immutable
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.ConfigurationException import akka.ConfigurationException
import akka.actor.Actor
import akka.actor.ActorContext import akka.actor.ActorContext
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ActorSystemImpl
import akka.actor.Address import akka.actor.Address
import akka.actor.Deploy import akka.actor.Deploy
import akka.actor.InternalActorRef
import akka.actor.Props import akka.actor.Props
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.cluster.Cluster import akka.cluster.Cluster
@ -51,7 +48,7 @@ final case class ClusterRouterConfig(local: RouterConfig, settings: ClusterRoute
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor // 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 }: Route) orElse localRoute
} }
@ -156,7 +153,7 @@ private[akka] class ClusterRouteeProvider(
// need this counter as instance variable since Resizer may call createRoutees several times // need this counter as instance variable since Resizer may call createRoutees several times
private val childNameCounter = new AtomicInteger 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]" throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
format context.self.path.toString) format context.self.path.toString)
@ -183,7 +180,7 @@ private[akka] class ClusterRouteeProvider(
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false) context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
} }
// must register each one, since registered routees are used in selectDeploymentTarget // must register each one, since registered routees are used in selectDeploymentTarget
registerRoutees(Some(ref)) registerRoutees(List(ref))
// recursion until all created // recursion until all created
doCreateRoutees() doCreateRoutees()
@ -202,7 +199,7 @@ private[akka] class ClusterRouteeProvider(
} else { } else {
// find the node with least routees // find the node with least routees
val numberOfRouteesPerNode: Map[Address, Int] = 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) val address = fullAddress(x)
acc + (address -> (acc(address) + 1)) acc + (address -> (acc(address) + 1))
} }
@ -222,27 +219,26 @@ private[akka] class ClusterRouteeProvider(
case a a case a a
} }
private[routing] def availableNodes: SortedSet[Address] = { private[routing] def availableNodes: immutable.SortedSet[Address] = {
import Member.addressOrdering import Member.addressOrdering
val currentNodes = nodes val currentNodes = nodes
if (currentNodes.isEmpty && settings.allowLocalRoutees) if (currentNodes.isEmpty && settings.allowLocalRoutees)
//use my own node, cluster information not updated yet //use my own node, cluster information not updated yet
SortedSet(cluster.selfAddress) immutable.SortedSet(cluster.selfAddress)
else else
currentNodes currentNodes
} }
@volatile @volatile
private[routing] var nodes: SortedSet[Address] = { private[routing] var nodes: immutable.SortedSet[Address] = {
import Member.addressOrdering import Member.addressOrdering
cluster.readView.members.collect { cluster.readView.members.collect {
case m if isAvailable(m) m.address 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) m.status == MemberStatus.Up && (settings.allowLocalRoutees || m.address != cluster.selfAddress)
}
} }

View file

@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.actor.Address import akka.actor.Address
import scala.collection.immutable
case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -51,7 +52,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
cluster.down(thirdAddress) cluster.down(thirdAddress)
enterBarrier("down-third-node") enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
clusterView.members.exists(_.address == thirdAddress) must be(false) clusterView.members.exists(_.address == thirdAddress) must be(false)
} }
@ -62,7 +63,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
runOn(second, fourth) { runOn(second, fourth) {
enterBarrier("down-third-node") enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
} }
enterBarrier("await-completion") enterBarrier("await-completion")

View file

@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.actor.Address import akka.actor.Address
import scala.collection.immutable
case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -49,7 +50,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
markNodeAsUnavailable(thirdAddress) markNodeAsUnavailable(thirdAddress)
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
clusterView.members.exists(_.address == thirdAddress) must be(false) clusterView.members.exists(_.address == thirdAddress) must be(false)
} }
@ -60,7 +61,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
runOn(second, fourth) { runOn(second, fourth) {
enterBarrier("down-third-node") enterBarrier("down-third-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress)) awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
} }
enterBarrier("await-completion") enterBarrier("await-completion")

View file

@ -55,7 +55,6 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig {
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.remote.netty.execution-pool-size = 4 akka.remote.netty.execution-pool-size = 4
#akka.remote.netty.reconnection-time-window = 10s #akka.remote.netty.reconnection-time-window = 10s
akka.remote.netty.read-timeout = 5s
akka.remote.netty.write-timeout = 5s akka.remote.netty.write-timeout = 5s
akka.remote.netty.backoff-timeout = 500ms akka.remote.netty.backoff-timeout = 500ms
akka.remote.netty.connection-timeout = 500ms akka.remote.netty.connection-timeout = 500ms
@ -148,7 +147,7 @@ abstract class LargeClusterSpec
runOn(runOnRoles: _*) { runOn(runOnRoles: _*) {
systems.size must be(nodesPerDatacenter) // make sure it is initialized 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] ++ val startGossipCounts = Map.empty[Cluster, Long] ++
clusterNodes.map(c (c -> c.readView.latestStats.receivedGossipCount)) clusterNodes.map(c (c -> c.readView.latestStats.receivedGossipCount))
def gossipCount(c: Cluster): Long = { def gossipCount(c: Cluster): Long = {
@ -260,7 +259,7 @@ abstract class LargeClusterSpec
if (bulk.nonEmpty) { if (bulk.nonEmpty) {
val totalNodes = nodesPerDatacenter * 4 + bulk.size val totalNodes = nodesPerDatacenter * 4 + bulk.size
within(expectedMaxDuration(totalNodes)) { 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, join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
enterBarrier("fifth-datacenter-joined-" + bulk.size) enterBarrier("fifth-datacenter-joined-" + bulk.size)
@ -270,7 +269,7 @@ abstract class LargeClusterSpec
for (i 0 until oneByOne.size) { for (i 0 until oneByOne.size) {
val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1 val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1
within(expectedMaxDuration(totalNodes)) { 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, join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter) runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
enterBarrier("fifth-datacenter-joined-" + (bulk.size + i)) enterBarrier("fifth-datacenter-joined-" + (bulk.size + i))

View file

@ -11,6 +11,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.actor._ import akka.actor._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") 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 --- // --- 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) { runOn(fourth) {
@ -69,7 +70,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
runOn(second, third) { runOn(second, third) {
enterBarrier("down-fourth-node") enterBarrier("down-fourth-node")
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds) awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds)
} }
enterBarrier("await-completion-1") 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 --- // --- 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) { runOn(second) {
@ -99,7 +100,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
runOn(third) { runOn(third) {
enterBarrier("down-second-node") enterBarrier("down-second-node")
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds) awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30 seconds)
} }
enterBarrier("await-completion-2") enterBarrier("await-completion-2")

View file

@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val controller = role("controller") val controller = role("controller")
@ -42,7 +43,7 @@ abstract class LeaderElectionSpec(multiNodeConfig: LeaderElectionMultiNodeConfig
import multiNodeConfig._ import multiNodeConfig._
// sorted in the order used by the cluster // 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 { "A cluster of four nodes" must {

View file

@ -4,21 +4,21 @@
package akka.cluster package akka.cluster
import language.implicitConversions import language.implicitConversions
import org.scalatest.Suite
import org.scalatest.exceptions.TestFailedException
import com.typesafe.config.Config import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.{ Address, ExtendedActorSystem }
import akka.remote.testconductor.RoleName import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec } import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec }
import akka.testkit._ import akka.testkit._
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import scala.concurrent.duration._ import akka.actor.{ ActorSystem, Address }
import org.scalatest.Suite
import org.scalatest.exceptions.TestFailedException
import java.util.concurrent.ConcurrentHashMap
import akka.actor.ActorPath
import akka.actor.RootActorPath
import akka.event.Logging.ErrorLevel import akka.event.Logging.ErrorLevel
import akka.actor.ActorSystem import scala.concurrent.duration._
import scala.collection.immutable
import java.util.concurrent.ConcurrentHashMap
object MultiNodeClusterSpec { object MultiNodeClusterSpec {
@ -158,7 +158,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
* nodes (roles). First node will be started first * nodes (roles). First node will be started first
* and others will join the 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 * 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 * First node will be started first and others will join
* the first. * the first.
*/ */
def awaitClusterUp(roles: RoleName*): Unit = { def awaitClusterUp(roles: RoleName*): Unit = awaitStartCluster(true, roles.to[immutable.Seq])
awaitStartCluster(true, roles.toSeq)
}
private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = { private def awaitStartCluster(upConvergence: Boolean = true, roles: immutable.Seq[RoleName]): Unit = {
runOn(roles.head) { runOn(roles.head) {
// make sure that the node-to-join is started before other join // make sure that the node-to-join is started before other join
startClusterNode() startClusterNode()
@ -196,21 +194,20 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) members(i).address must be(a) } expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) members(i).address must be(a) }
} }
def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) { def assertLeader(nodesInCluster: RoleName*): Unit =
assertLeaderIn(nodesInCluster) if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq])
}
/** /**
* Assert that the cluster has elected the correct leader * Assert that the cluster has elected the correct leader
* out of all nodes in the cluster. First * out of all nodes in the cluster. First
* member in the cluster ring is expected leader. * 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) nodesInCluster.length must not be (0)
val expectedLeader = roleOfLeader(nodesInCluster) val expectedLeader = roleOfLeader(nodesInCluster)
val leader = clusterView.leader val leader = clusterView.leader
val isLeader = leader == Some(clusterView.selfAddress) 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)) "expectedLeader [%s], got leader [%s], members [%s]".format(expectedLeader, leader, clusterView.members))
clusterView.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving)) clusterView.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving))
} }
@ -221,7 +218,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
*/ */
def awaitUpConvergence( def awaitUpConvergence(
numberOfMembers: Int, numberOfMembers: Int,
canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address], canNotBePartOfMemberRing: immutable.Seq[Address] = Nil,
timeout: FiniteDuration = 20.seconds): Unit = { timeout: FiniteDuration = 20.seconds): Unit = {
within(timeout) { within(timeout) {
awaitCond(clusterView.members.size == numberOfMembers) awaitCond(clusterView.members.size == numberOfMembers)
@ -239,7 +236,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
def awaitSeenSameState(addresses: Address*): Unit = def awaitSeenSameState(addresses: Address*): Unit =
awaitCond((addresses.toSet -- clusterView.seenBy).isEmpty) 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.length must not be (0)
nodesInCluster.sorted.head nodesInCluster.sorted.head
} }

View file

@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -65,7 +66,7 @@ abstract class SingletonClusterSpec(multiNodeConfig: SingletonClusterMultiNodeCo
markNodeAsUnavailable(secondAddress) markNodeAsUnavailable(secondAddress)
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds) awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = List(secondAddress), 30.seconds)
clusterView.isSingletonCluster must be(true) clusterView.isSingletonCluster must be(true)
awaitCond(clusterView.isLeader) awaitCond(clusterView.isLeader)
} }

View file

@ -9,9 +9,10 @@ import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import scala.concurrent.duration._
import akka.actor.Address import akka.actor.Address
import akka.remote.testconductor.Direction import akka.remote.testconductor.Direction
import scala.concurrent.duration._
import scala.collection.immutable
case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -53,8 +54,8 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig)
muteMarkingAsUnreachable() muteMarkingAsUnreachable()
val side1 = IndexedSeq(first, second) val side1 = Vector(first, second)
val side2 = IndexedSeq(third, fourth, fifth) val side2 = Vector(third, fourth, fifth)
"A cluster of 5 members" must { "A cluster of 5 members" must {

View file

@ -6,13 +6,14 @@ package akka.cluster
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.actor.Address import akka.actor.Address
import akka.remote.testconductor.{ RoleName, Direction } import akka.remote.testconductor.{ RoleName, Direction }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig { case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -45,7 +46,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
muteMarkingAsUnreachable() 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) roles.filterNot(_ == role)
} }
@ -125,7 +126,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
} }
runOn(allBut(victim): _*) { runOn(allBut(victim): _*) {
awaitUpConvergence(roles.size - 1, Seq(victim)) awaitUpConvergence(roles.size - 1, List(victim))
} }
endBarrier endBarrier

View file

@ -7,7 +7,7 @@ package akka.cluster
import akka.actor.Address import akka.actor.Address
import akka.testkit._ import akka.testkit._
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import scala.collection.immutable.TreeMap import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
@ -27,7 +27,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
val conn = Address("akka", "", "localhost", 2552) val conn = Address("akka", "", "localhost", 2552)
val conn2 = Address("akka", "", "localhost", 2553) 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)) var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) acc ::: List[Long](acc.last + c))
def timeGenerator(): Long = { def timeGenerator(): Long = {
val currentTime = times.head val currentTime = times.head
@ -73,7 +73,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
"return realistic phi values" in { "return realistic phi values" in {
val fd = createFailureDetector() 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) { for ((timeDiff, expectedPhi) test) {
fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1)) fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1))
} }

View file

@ -5,14 +5,16 @@
package akka.cluster package akka.cluster
import scala.language.postfixOps import scala.language.postfixOps
import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.Await import scala.concurrent.Await
import scala.util.{ Success, Try, Failure }
import akka.actor._ import akka.actor._
import akka.testkit._ import akka.testkit._
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import util.{ Success, Try, Failure }
object MetricsEnabledSpec { object MetricsEnabledSpec {
val config = """ val config = """
@ -109,8 +111,7 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl
"collect JMX metrics" in { "collect JMX metrics" in {
// heap max may be undefined depending on the OS // heap max may be undefined depending on the OS
// systemLoadAverage is JMX is SIGAR not present // systemLoadAverage is JMX if SIGAR not present, but not available on all OS
collector.systemLoadAverage.isDefined must be(true)
collector.used.isDefined must be(true) collector.used.isDefined must be(true)
collector.committed.isDefined must be(true) collector.committed.isDefined must be(true)
collector.processors.isDefined must be(true) collector.processors.isDefined must be(true)
@ -208,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)) 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] = { def collectNodeMetrics(nodes: Set[NodeMetrics]): immutable.Seq[Metric] =
var r: Seq[Metric] = Seq.empty nodes.foldLeft(Vector[Metric]()) {
nodes.foreach(n r ++= n.metrics.filter(_.isDefined)) case (r, n) r ++ n.metrics.filter(_.isDefined)
r }
}
} }
trait AbstractClusterMetricsSpec extends DefaultTimeout { trait AbstractClusterMetricsSpec extends DefaultTimeout {

View file

@ -29,6 +29,7 @@ The Current List of Modules
.. toctree:: .. toctree::
reliable-proxy reliable-proxy
throttle
Suggested Way of Using these Contributions Suggested Way of Using these Contributions
------------------------------------------ ------------------------------------------

View file

@ -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 dont 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 throttlers 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 Cordingleys `blog post <http://www.cordinc.com/blog/2010/04/java-multichannel-asynchronous.html>`_
discusses a throttler with stronger guarantees (it solves problem 2 from above).
Future versions of this module may feature throttlers with better guarantees.

Binary file not shown.

After

Width:  |  Height:  |  Size: 3.9 KiB

View file

@ -0,0 +1,296 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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 <em>throttler</em> is an actor that is defined through a <em>target actor</em> and a <em>rate</em>
* (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 <em>below</em>
* the threshold rate.
*
* <em>Example:</em> 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 <em>distribute</em> its messages to a set of worker actors so that individual messages
* can be handled in parallel.
*
* @see [[akka.contrib.throttle.TimerBasedThrottler]]
*/
trait Throttler { this: 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(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))
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: A `TimerBasedThrottler` is in state `Active` iff the timer is running.
private[throttle] sealed trait State
private[throttle] case object Idle extends State
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
* <a href='http://letitcrash.com/post/28901663062/throttling-messages-in-akka-2'>this blog post</a>):
*
* - 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)
// 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)
// 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) using d.copy(callsLeftInThisPeriod = rate.numberOfCalls)
// 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)
}
}

View file

@ -122,8 +122,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
enterBarrier("test2b") enterBarrier("test2b")
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Send, -1) testConductor.throttle(local, remote, Direction.Send, -1).await
expectTransition(Active, Idle) within(5 seconds) { expectTransition(Active, Idle) }
} }
runOn(remote) { runOn(remote) {
within(1 second) { within(1 second) {
@ -152,8 +152,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
enterBarrier("test3a") enterBarrier("test3a")
runOn(local) { runOn(local) {
testConductor.throttle(local, remote, Direction.Receive, -1) testConductor.throttle(local, remote, Direction.Receive, -1).await
expectTransition(Active, Idle) within(5 seconds) { expectTransition(Active, Idle) }
} }
enterBarrier("test3b") enterBarrier("test3b")

View file

@ -0,0 +1,205 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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
import akka.testkit._
object TimerBasedThrottlerSpec {
class EchoActor extends Actor {
def receive = {
case x sender ! x
}
}
}
@RunWith(classOf[JUnitRunner])
class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSpec")) with ImplicitSender
with WordSpec with MustMatchers with BeforeAndAfterAll {
override def afterAll {
system.shutdown()
}
"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
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.dilated))))
// 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.dilated))))
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")
}
}
"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.dilated))))
throttler ! SetTarget(Some(echo))
throttler ! "1"
throttler ! "2"
throttler ! "3"
throttler ! SetTarget(None)
within(1 second) {
expectMsg("1")
expectMsg("2")
expectMsg("3")
expectNoMsg()
}
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()
}
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.dilated))))
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()
}
expectNoMsg(1 second)
throttler ! SetTarget(Some(echo))
within(1 seconds) {
expectMsg("4")
expectMsg("5")
expectMsg("6")
expectNoMsg()
}
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.dilated))))
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()
}
within(1 second) {
expectMsg("4")
expectMsg("5")
expectMsg("6")
expectNoMsg()
}
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.dilated))))
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()
}
within(1 second) {
expectMsg("5")
expectMsg("6")
expectMsg("7")
expectMsg("8")
expectNoMsg()
}
within(1 second) {
expectMsg("9")
}
}
}
}

View file

@ -46,7 +46,7 @@ package object dataflow {
implicit class DataflowPromise[T](val promise: Promise[T]) extends AnyVal { 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. * completed. See Promise.success(value) for semantics.
* *
* @param value The value which denotes the successful value of the Promise * @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. * 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 * @param ec An ExecutionContext which will be used to execute callbacks registered in this method
* @return A Future representing the result of this operation * @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. * 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 * @param ec An ExecutionContext which will be used to execute callbacks registered in this method
* @return A Future representing the result of this operation * @return A Future representing the result of this operation
*/ */

View file

@ -5,7 +5,7 @@ Duration
######## ########
Durations are used throughout the Akka library, wherefore this concept is 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`, Values of this type may represent infinite (:obj:`Duration.Inf`,
:obj:`Duration.MinusInf`) or finite durations, or be :obj:`Duration.Undefined`. :obj:`Duration.MinusInf`) or finite durations, or be :obj:`Duration.Undefined`.

View file

@ -17,13 +17,12 @@ built using `Sphinx`_.
Sphinx Sphinx
====== ======
More to come... For more details see `The Sphinx Documentation <http://sphinx.pocoo.org/contents.html>`_
reStructuredText reStructuredText
================ ================
More to come... For more details see `The reST Quickref <http://docutils.sourceforge.net/docs/user/rst/quickref.html>`_
Sections Sections
-------- --------
@ -75,16 +74,17 @@ First install `Sphinx`_. See below.
Building Building
-------- --------
:: For the html version of the docs::
cd akka-docs sbt sphinx:generate-html
make html open <project-dir>/akka-docs/target/sphinx/html/index.html
open _build/html/index.html
make pdf For the pdf version of the docs::
open _build/latex/Akka.pdf
sbt sphinx:generate-pdf
open <project-dir>/akka-docs/target/sphinx/latex/Akka.pdf
Installing Sphinx on OS X Installing Sphinx on OS X
------------------------- -------------------------

View file

@ -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 .. includecode:: ../../../project/plugins.sbt#sbt-multi-jvm
You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm`` 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 settings and config. Please note that MultiJvm test sources are located in ``src/multi-jvm/...``,
multi-JVM testing (Simplified for clarity): and not in ``src/test/...``.
Here is an example Build.scala file that uses the MultiJvm plugin:
.. parsed-literal:: .. parsed-literal::
import sbt._ import sbt._
import Keys._ import Keys._
import com.typesafe.sbt.SbtMultiJvm 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( lazy val buildSettings = Defaults.defaultSettings ++ multiJvmSettings ++ Seq(
id = "akka-remote-tests", organization := "example",
base = file("akka-remote-tests"), version := "1.0",
dependencies = Seq(remote, actorTests % "test->test", scalaVersion := "@scalaVersion@",
testkit % "test->test"), // make sure that the artifacts don't have the scala version in the name
settings = defaultSettings ++ Seq( crossPaths := false
// 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 ++ lazy val example = Project(
SbtMultiJvm.multiJvmSettings ++ Seq( id = "example",
organization := "com.typesafe.akka", base = file("."),
version := "@version@", settings = buildSettings ++
scalaVersion := "@scalaVersion@", Seq(libraryDependencies ++= Dependencies.example)
crossPaths := false ) configs(MultiJvm)
)
lazy val defaultSettings = buildSettings ++ Seq( lazy val multiJvmSettings = SbtMultiJvm.multiJvmSettings ++ Seq(
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" // 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-B2" % "test" cross CrossVersion.full,
"junit" % "junit" % "4.5" % "test"
)
}
}
You can specify JVM options for the forked JVMs:: You can specify JVM options for the forked JVMs::
@ -73,7 +86,7 @@ You can specify JVM options for the forked JVMs::
Running tests 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. 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 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 Creating application tests
========================== ==========================
The tests are discovered, and combined, through a naming convention. MultiJvm tests are The tests are discovered, and combined, through a naming convention. MultiJvm test sources
located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern: are located in ``src/multi-jvm/...``. A test is named with the following pattern:
.. code-block:: none .. code-block:: none
@ -162,14 +175,26 @@ spawned, one for each node. It will look like this:
[success] Total time: ... [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 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" multiJvmMarker in MultiJvm := "ClusterTest"
Your tests should now be named ``{TestName}ClusterTest{NodeName}``. Your tests should now be named ``{TestName}ClusterTest{NodeName}``.

View file

@ -108,8 +108,8 @@ Actor Best Practices
#. Top-level actors are the innermost part of your Error Kernel, so create them #. Top-level actors are the innermost part of your Error Kernel, so create them
sparingly and prefer truly hierarchical systems. This has benefits wrt. sparingly and prefer truly hierarchical systems. This has benefits wrt.
fault-handling (both considering the granularity of configuration and the fault-handling (both considering the granularity of configuration and the
performance) and it also reduces the number of blocking calls made, since performance) and it also reduces the strain on the guardian actor, which is
the creation of top-level actors involves synchronous messaging. a single point of contention if over-used.
Blocking Needs Careful Management Blocking Needs Careful Management
--------------------------------- ---------------------------------

View file

@ -27,10 +27,11 @@ import akka.testkit.ErrorFilter;
import akka.testkit.EventFilter; import akka.testkit.EventFilter;
import akka.testkit.TestEvent; import akka.testkit.TestEvent;
import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.concurrent.TimeUnit.SECONDS;
import static akka.japi.Util.immutableSeq;
import akka.japi.Function; import akka.japi.Function;
import scala.Option; import scala.Option;
import scala.collection.JavaConverters; import scala.collection.JavaConverters;
import scala.collection.Seq; import scala.collection.immutable.Seq;
import org.junit.Test; import org.junit.Test;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -219,8 +220,7 @@ public class FaultHandlingTestBase {
//#testkit //#testkit
public <A> Seq<A> seq(A... args) { public <A> Seq<A> seq(A... args) {
return JavaConverters.collectionAsScalaIterableConverter( return immutableSeq(args);
java.util.Arrays.asList(args)).asScala().toSeq();
} }
//#testkit //#testkit
} }

View file

@ -351,24 +351,23 @@ public class UntypedActorDocTestBase {
static static
//#stash //#stash
public class ActorWithProtocol extends UntypedActorWithStash { public class ActorWithProtocol extends UntypedActorWithStash {
private Boolean isOpen = false;
public void onReceive(Object msg) { public void onReceive(Object msg) {
if (isOpen) { if (msg.equals("open")) {
if (msg.equals("write")) { unstashAll();
// do writing... getContext().become(new Procedure<Object>() {
} else if (msg.equals("close")) { public void apply(Object msg) throws Exception {
unstashAll(); if (msg.equals("write")) {
isOpen = false; // do writing...
} else { } else if (msg.equals("close")) {
stash(); unstashAll();
} getContext().unbecome();
} else {
stash();
}
}
}, false); // add behavior on top instead of replacing
} else { } else {
if (msg.equals("open")) { stash();
unstashAll();
isOpen = true;
} else {
stash();
}
} }
} }
} }

View file

@ -32,9 +32,9 @@ public class UntypedActorSwapper {
@Override @Override
public void apply(Object message) { public void apply(Object message) {
log.info("Ho"); 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 { } else {
unhandled(message); unhandled(message);
} }

View file

@ -119,5 +119,4 @@ public class LoggingDocTestBase {
} }
} }
//#deadletter-actor //#deadletter-actor
} }

View file

@ -11,6 +11,7 @@ import static docs.jrouting.CustomRouterDocTestBase.Message.RepublicanVote;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import org.junit.After; import org.junit.After;
@ -69,7 +70,7 @@ public class CustomRouterDocTestBase {
//#supervision //#supervision
final SupervisorStrategy strategy = final SupervisorStrategy strategy =
new OneForOneStrategy(5, Duration.create("1 minute"), new OneForOneStrategy(5, Duration.create("1 minute"),
new Class<?>[] { Exception.class }); Collections.<Class<? extends Throwable>>singletonList(Exception.class));
final ActorRef router = system.actorOf(new Props(MyActor.class) final ActorRef router = system.actorOf(new Props(MyActor.class)
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy))); .withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
//#supervision //#supervision
@ -179,16 +180,14 @@ public class CustomRouterDocTestBase {
//#crRoutingLogic //#crRoutingLogic
return new CustomRoute() { return new CustomRoute() {
@Override @Override
public Iterable<Destination> destinationsFor(ActorRef sender, Object msg) { public scala.collection.immutable.Seq<Destination> destinationsFor(ActorRef sender, Object msg) {
switch ((Message) msg) { switch ((Message) msg) {
case DemocratVote: case DemocratVote:
case DemocratCountResult: case DemocratCountResult:
return Arrays.asList( return akka.japi.Util.immutableSingletonSeq(new Destination(sender, democratActor));
new Destination[] { new Destination(sender, democratActor) });
case RepublicanVote: case RepublicanVote:
case RepublicanCountResult: case RepublicanCountResult:
return Arrays.asList( return akka.japi.Util.immutableSingletonSeq(new Destination(sender, republicanActor));
new Destination[] { new Destination(sender, republicanActor) });
default: default:
throw new IllegalArgumentException("Unknown message: " + msg); throw new IllegalArgumentException("Unknown message: " + msg);
} }

View file

@ -549,7 +549,8 @@ Upgrade
Akka supports hotswapping the Actors message loop (e.g. its implementation) at Akka supports hotswapping the Actors message loop (e.g. its implementation) at
runtime. Use the ``getContext().become`` method from within the Actor. 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:: .. warning::
@ -563,26 +564,19 @@ To hotswap the Actor using ``getContext().become``:
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java .. includecode:: code/docs/actor/UntypedActorDocTestBase.java
:include: hot-swap-actor :include: hot-swap-actor
The ``become`` method is useful for many different things, such as to implement This variant of the :meth:`become` method is useful for many different things,
a Finite State Machine (FSM). 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 you do not use
:meth:`unbecome`, 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 .. 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 Stash
===== =====
@ -627,9 +621,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 result, even a very large number of messages may be stashed without a
major impact on performance. major impact on performance.
Note that the stash is not persisted across restarts of an actor, Note that the stash is part of the ephemeral actor state, unlike the
unlike the actor's mailbox. Therefore, it should be managed like other mailbox. Therefore, it should be managed like other parts of the
parts of the actor's state which have the same property. actor's state which have the same property. The :class:`Stash` traits
implementation of :meth:`preRestart` will call ``unstashAll()``, which is
usually the desired behavior.
Killing an Actor Killing an Actor

View file

@ -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 <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_ and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
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.<collection-type>]`` method.

View file

@ -8,3 +8,4 @@ Migration Guides
migration-guide-1.3.x-2.0.x migration-guide-1.3.x-2.0.x
migration-guide-2.0.x-2.1.x migration-guide-2.0.x-2.1.x
migration-guide-2.1.x-2.2.x

View file

@ -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 have to either declare an implicit :class:`ActorSystem`, or you can give the
factory explicitly (see further below). 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 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 Hook`_ below), where later invocations of the methods shown below will replace
the contents of the respective hooks: the contents of the respective hooks:
@ -653,11 +664,10 @@ Upgrade
------- -------
Akka supports hotswapping the Actors message loop (e.g. its implementation) at Akka supports hotswapping the Actors message loop (e.g. its implementation) at
runtime: Invoke the ``context.become`` method from within the Actor. runtime: invoke the ``context.become`` method from within the Actor.
:meth:`become` takes a ``PartialFunction[Any, Unit]`` that implements the new
Become takes a ``PartialFunction[Any, Unit]`` that implements message handler. The hotswapped code is kept in a Stack which can be pushed and
the new message handler. The hotswapped code is kept in a Stack which can be popped.
pushed and popped.
.. warning:: .. warning::
@ -667,38 +677,26 @@ To hotswap the Actor behavior using ``become``:
.. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor .. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor
The ``become`` method is useful for many different things, but a particular nice This variant of the :meth:`become` method is useful for many different things,
example of it is in example where it is used to implement a Finite State Machine such as to implement a Finite State Machine (FSM, for an example see `Dining
(FSM): `Dining Hakkers`_. Hakkers`_). It will replace the current 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.
.. _Dining Hakkers: @github@/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala .. _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 .. includecode:: code/docs/actor/ActorDocSpec.scala#swapper
Encoding Scala Actors nested receives without accidentally leaking memory Encoding Scala Actors nested receives without accidentally leaking memory
------------------------------------------------------------------------- -------------------------------------------------------------------------
See this `Unnested receive example <https://github.com/akka/akka/blob/master/akka-docs/scala/code/docs/actor/UnnestedReceives.scala>`_. See this `Unnested receive example <@github@/akka-docs/rst/scala/code/docs/actor/UnnestedReceives.scala>`_.
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()
}
Stash Stash
@ -752,9 +750,11 @@ major impact on performance.
callback. This means it's not possible to write callback. This means it's not possible to write
``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``. ``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``.
Note that the stash is not persisted across restarts of an actor, Note that the stash is part of the ephemeral actor state, unlike the
unlike the actor's mailbox. Therefore, it should be managed like other mailbox. Therefore, it should be managed like other parts of the
parts of the actor's state which have the same property. actor's state which have the same property. The :class:`Stash` traits
implementation of :meth:`preRestart` will call ``unstashAll()``, which is
usually the desired behavior.
Killing an Actor Killing an Actor

View file

@ -96,11 +96,11 @@ class Swapper extends Actor {
def receive = { def receive = {
case Swap case Swap
log.info("Hi") log.info("Hi")
become { become({
case Swap case Swap
log.info("Ho") log.info("Ho")
unbecome() // resets the latest 'become' (just for fun) unbecome() // resets the latest 'become' (just for fun)
} }, discardOld = false) // push on top instead of replace
} }
} }
@ -316,13 +316,13 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
def receive = { def receive = {
case "open" case "open"
unstashAll() unstashAll()
context.become { context.become({
case "write" // do writing... case "write" // do writing...
case "close" case "close"
unstashAll() unstashAll()
context.unbecome() context.unbecome()
case msg stash() case msg stash()
} }, discardOld = false) // stack on top instead of replacing
case msg stash() case msg stash()
} }
} }

View file

@ -8,6 +8,7 @@ import language.postfixOps
import akka.testkit.{ AkkaSpec MyFavoriteTestFrameWorkPlusAkkaTestKit } import akka.testkit.{ AkkaSpec MyFavoriteTestFrameWorkPlusAkkaTestKit }
//#test-code //#test-code
import akka.actor.Props import akka.actor.Props
import scala.collection.immutable
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
@ -24,7 +25,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
case object Flush case object Flush
// sent events // sent events
case class Batch(obj: Seq[Any]) case class Batch(obj: immutable.Seq[Any])
//#simple-events //#simple-events
//#simple-state //#simple-state
// states // states
@ -34,7 +35,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
sealed trait Data sealed trait Data
case object Uninitialized extends 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-state
//#simple-fsm //#simple-fsm
class Buncher extends Actor with FSM[State, Data] { class Buncher extends Actor with FSM[State, Data] {
@ -193,12 +194,12 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
buncher ! SetTarget(testActor) buncher ! SetTarget(testActor)
buncher ! Queue(42) buncher ! Queue(42)
buncher ! Queue(43) buncher ! Queue(43)
expectMsg(Batch(Seq(42, 43))) expectMsg(Batch(immutable.Seq(42, 43)))
buncher ! Queue(44) buncher ! Queue(44)
buncher ! Flush buncher ! Flush
buncher ! Queue(45) buncher ! Queue(45)
expectMsg(Batch(Seq(44))) expectMsg(Batch(immutable.Seq(44)))
expectMsg(Batch(Seq(45))) expectMsg(Batch(immutable.Seq(45)))
} }
"batch not if uninitialized" in { "batch not if uninitialized" in {

View file

@ -44,21 +44,23 @@ class DataflowDocSpec extends WordSpec with MustMatchers {
} }
"demonstrate the use of dataflow variables" in { "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 //#dataflow-variable-a
val v1, v2 = Promise[Int]()
flow { flow {
val v1, v2 = Promise[Int]()
// v1 will become the value of v2 + 10 when v2 gets a value // v1 will become the value of v2 + 10 when v2 gets a value
v1 << v2() + 10 v1 << v2() + 10
v2 << flow { 5 } // As you can see, no blocking!
v1() + v2() v1() + v2()
} onComplete println } onComplete println
flow { v2 << 5 } // As you can see, no blocking above!
//#dataflow-variable-a //#dataflow-variable-a
Await.result(result.future, 10.seconds) must be === 20
} }
"demonstrate the difference between for and flow" in { "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 //#for-vs-flow
val f1, f2 = Future { 1 } val f1, f2 = Future { 1 }
@ -68,6 +70,7 @@ class DataflowDocSpec extends WordSpec with MustMatchers {
usingFor onComplete println usingFor onComplete println
usingFlow onComplete println usingFlow onComplete println
//#for-vs-flow //#for-vs-flow
Await.result(result.future, 10.seconds) must be === 2
} }
} }

View file

@ -22,6 +22,7 @@ import akka.testkit.DefaultTimeout
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.testkit.TestKit import akka.testkit.TestKit
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
/** /**
* a Test to show some TestKit examples * a Test to show some TestKit examples
@ -38,8 +39,8 @@ class TestKitUsageSpec
val filterRef = system.actorOf(Props(new FilteringActor(testActor))) val filterRef = system.actorOf(Props(new FilteringActor(testActor)))
val randomHead = Random.nextInt(6) val randomHead = Random.nextInt(6)
val randomTail = Random.nextInt(10) val randomTail = Random.nextInt(10)
val headList = Seq().padTo(randomHead, "0") val headList = immutable.Seq().padTo(randomHead, "0")
val tailList = Seq().padTo(randomTail, "1") val tailList = immutable.Seq().padTo(randomTail, "1")
val seqRef = val seqRef =
system.actorOf(Props(new SequencingActor(testActor, headList, tailList))) 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 * like to test that the interesting value is received and that you cant
* be bothered with the rest * 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 { extends Actor {
def receive = { def receive = {
case msg { case msg {

View file

@ -5,13 +5,13 @@ package docs.zeromq
import language.postfixOps import language.postfixOps
import akka.actor.{ Actor, Props }
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable
import akka.actor.{ Actor, Props }
import akka.testkit._ import akka.testkit._
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension } import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import akka.zeromq.{ SocketType, Bind }
object ZeromqDocSpec { object ZeromqDocSpec {
@ -52,12 +52,12 @@ object ZeromqDocSpec {
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed, val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed,
currentHeap.getMax)).get currentHeap.getMax)).get
// the first frame is the topic, second is the message // 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 // use akka SerializationExtension to convert to bytes
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get
// the first frame is the topic, second is the message // 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 //#health
@ -146,7 +146,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
val payload = Array.empty[Byte] val payload = Array.empty[Byte]
//#pub-topic //#pub-topic
pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload))) pubSocket ! ZMQMessage(Frame("foo.bar"), Frame(payload))
//#pub-topic //#pub-topic
system.stop(subSocket) system.stop(subSocket)

View file

@ -124,6 +124,14 @@ obvious that an actor is actually created:
:include: simple-fsm :include: simple-fsm
:exclude: fsm-body :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 :class:`FSM` trait takes two type parameters:
#. the supertype of all state names, usually a sealed trait with case objects #. the supertype of all state names, usually a sealed trait with case objects

View file

@ -713,8 +713,8 @@ Some `Specs2 <http://specs2.org>`_ users have contributed examples of how to wor
actually beneficial also for the third point—is to apply the TestKit together actually beneficial also for the third point—is to apply the TestKit together
with :class:`org.specs2.specification.Scope`. with :class:`org.specs2.specification.Scope`.
* The Specification traits provide a :class:`Duration` DSL which uses partly * The Specification traits provide a :class:`Duration` DSL which uses partly
the same method names as :class:`scala.concurrent.util.Duration`, resulting in ambiguous the same method names as :class:`scala.concurrent.duration.Duration`, resulting in ambiguous
implicits if ``akka.util.duration._`` is imported. There are two work-arounds: implicits if ``scala.concurrent.duration._`` is imported. There are two work-arounds:
* either use the Specification variant of Duration and supply an implicit * either use the Specification variant of Duration and supply an implicit
conversion to the Akka Duration. This conversion is not supplied with the conversion to the Akka Duration. This conversion is not supplied with the

Some files were not shown because too many files have changed in this diff Show more