Merge branch 'master' into wip-2547-metrics-router-patriknw
Conflicts: akka-actor/src/main/scala/akka/actor/Deployer.scala akka-cluster/src/main/scala/akka/cluster/ClusterMetricsCollector.scala akka-cluster/src/test/scala/akka/cluster/MetricsCollectorSpec.scala
This commit is contained in:
commit
1914be7069
110 changed files with 888 additions and 798 deletions
|
|
@ -15,7 +15,8 @@ public class CustomRouteTest {
|
|||
// only to test compilability
|
||||
public void testRoute() {
|
||||
final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1)));
|
||||
final scala.Function1<scala.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);
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,14 +4,15 @@
|
|||
package akka.actor
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.testkit._
|
||||
import org.scalatest.junit.JUnitSuite
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.JavaConverters
|
||||
import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue }
|
||||
import java.util.concurrent.{ RejectedExecutionException, ConcurrentLinkedQueue }
|
||||
import akka.util.Timeout
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.concurrent.Future
|
||||
import akka.pattern.ask
|
||||
|
||||
|
|
@ -102,8 +103,6 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
|
|||
}
|
||||
|
||||
"run termination callbacks in order" in {
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf)
|
||||
val result = new ConcurrentLinkedQueue[Int]
|
||||
val count = 10
|
||||
|
|
@ -121,13 +120,11 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
|
|||
Await.ready(latch, 5 seconds)
|
||||
|
||||
val expected = (for (i ← 1 to count) yield i).reverse
|
||||
result.asScala.toSeq must be(expected)
|
||||
|
||||
immutableSeq(result) must be(expected)
|
||||
}
|
||||
|
||||
"awaitTermination after termination callbacks" in {
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
|
||||
@volatile
|
||||
var callbackWasRun = false
|
||||
|
|
|
|||
|
|
@ -6,24 +6,22 @@ package akka.actor
|
|||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import java.net.URLEncoder
|
||||
import scala.collection.immutable
|
||||
|
||||
class RelativeActorPathSpec extends WordSpec with MustMatchers {
|
||||
|
||||
def elements(path: String): Seq[String] = path match {
|
||||
case RelativeActorPath(elem) ⇒ elem.toSeq
|
||||
case _ ⇒ Nil
|
||||
}
|
||||
def elements(path: String): immutable.Seq[String] = RelativeActorPath.unapply(path).getOrElse(Nil)
|
||||
|
||||
"RelativeActorPath" must {
|
||||
"match single name" in {
|
||||
elements("foo") must be(Seq("foo"))
|
||||
elements("foo") must be(List("foo"))
|
||||
}
|
||||
"match path separated names" in {
|
||||
elements("foo/bar/baz") must be(Seq("foo", "bar", "baz"))
|
||||
elements("foo/bar/baz") must be(List("foo", "bar", "baz"))
|
||||
}
|
||||
"match url encoded name" in {
|
||||
val name = URLEncoder.encode("akka://ClusterSystem@127.0.0.1:2552", "UTF-8")
|
||||
elements(name) must be(Seq(name))
|
||||
elements(name) must be(List(name))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,18 +5,19 @@ package akka.actor
|
|||
|
||||
import language.postfixOps
|
||||
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
||||
import akka.util.Timeout
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Await, Future, Promise }
|
||||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import annotation.tailrec
|
||||
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
|
||||
import akka.util.Timeout
|
||||
import akka.japi.{ Option ⇒ JOption }
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.dispatch.{ Dispatchers }
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.pattern.ask
|
||||
import akka.serialization.JavaSerializer
|
||||
import akka.actor.TypedActor._
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.lang.IllegalStateException
|
||||
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
|
||||
|
||||
|
|
@ -35,9 +36,9 @@ object TypedActorSpec {
|
|||
}
|
||||
"""
|
||||
|
||||
class CyclicIterator[T](val items: Seq[T]) extends Iterator[T] {
|
||||
class CyclicIterator[T](val items: immutable.Seq[T]) extends Iterator[T] {
|
||||
|
||||
private[this] val current: AtomicReference[Seq[T]] = new AtomicReference(items)
|
||||
private[this] val current = new AtomicReference(items)
|
||||
|
||||
def hasNext = items != Nil
|
||||
|
||||
|
|
|
|||
|
|
@ -12,17 +12,18 @@ import java.io.PrintWriter
|
|||
import java.text.SimpleDateFormat
|
||||
import java.util.Date
|
||||
import scala.collection.mutable.{ Map ⇒ MutableMap }
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.Logging
|
||||
|
||||
trait BenchResultRepository {
|
||||
def add(stats: Stats)
|
||||
|
||||
def get(name: String): Seq[Stats]
|
||||
def get(name: String): immutable.Seq[Stats]
|
||||
|
||||
def get(name: String, load: Int): Option[Stats]
|
||||
|
||||
def getWithHistorical(name: String, load: Int): Seq[Stats]
|
||||
def getWithHistorical(name: String, load: Int): immutable.Seq[Stats]
|
||||
|
||||
def isBaseline(stats: Stats): Boolean
|
||||
|
||||
|
|
@ -38,9 +39,9 @@ object BenchResultRepository {
|
|||
}
|
||||
|
||||
class FileBenchResultRepository extends BenchResultRepository {
|
||||
private val statsByName = MutableMap[String, Seq[Stats]]()
|
||||
private val statsByName = MutableMap[String, immutable.Seq[Stats]]()
|
||||
private val baselineStats = MutableMap[Key, Stats]()
|
||||
private val historicalStats = MutableMap[Key, Seq[Stats]]()
|
||||
private val historicalStats = MutableMap[Key, immutable.Seq[Stats]]()
|
||||
private def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir")
|
||||
private val serDir = resultDir + "/ser"
|
||||
private def serDirExists: Boolean = new File(serDir).exists
|
||||
|
|
@ -51,13 +52,13 @@ class FileBenchResultRepository extends BenchResultRepository {
|
|||
case class Key(name: String, load: Int)
|
||||
|
||||
def add(stats: Stats): Unit = synchronized {
|
||||
val values = statsByName.getOrElseUpdate(stats.name, IndexedSeq.empty)
|
||||
val values = statsByName.getOrElseUpdate(stats.name, Vector.empty)
|
||||
statsByName(stats.name) = values :+ stats
|
||||
save(stats)
|
||||
}
|
||||
|
||||
def get(name: String): Seq[Stats] = synchronized {
|
||||
statsByName.getOrElse(name, IndexedSeq.empty)
|
||||
def get(name: String): immutable.Seq[Stats] = synchronized {
|
||||
statsByName.getOrElse(name, Vector.empty)
|
||||
}
|
||||
|
||||
def get(name: String, load: Int): Option[Stats] = synchronized {
|
||||
|
|
@ -68,13 +69,13 @@ class FileBenchResultRepository extends BenchResultRepository {
|
|||
baselineStats.get(Key(stats.name, stats.load)) == Some(stats)
|
||||
}
|
||||
|
||||
def getWithHistorical(name: String, load: Int): Seq[Stats] = synchronized {
|
||||
def getWithHistorical(name: String, load: Int): immutable.Seq[Stats] = synchronized {
|
||||
val key = Key(name, load)
|
||||
val historical = historicalStats.getOrElse(key, IndexedSeq.empty)
|
||||
val historical = historicalStats.getOrElse(key, Vector.empty)
|
||||
val baseline = baselineStats.get(key)
|
||||
val current = get(name, load)
|
||||
|
||||
val limited = (IndexedSeq.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical)
|
||||
val limited = (Vector.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical)
|
||||
limited.sortBy(_.timestamp)
|
||||
}
|
||||
|
||||
|
|
@ -94,7 +95,7 @@ class FileBenchResultRepository extends BenchResultRepository {
|
|||
}
|
||||
val historical = load(historicalFiles)
|
||||
for (h ← historical) {
|
||||
val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), IndexedSeq.empty)
|
||||
val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), Vector.empty)
|
||||
historicalStats(Key(h.name, h.load)) = values :+ h
|
||||
}
|
||||
}
|
||||
|
|
@ -120,7 +121,7 @@ class FileBenchResultRepository extends BenchResultRepository {
|
|||
}
|
||||
}
|
||||
|
||||
private def load(files: Iterable[File]): Seq[Stats] = {
|
||||
private def load(files: Iterable[File]): immutable.Seq[Stats] = {
|
||||
val result =
|
||||
for (f ← files) yield {
|
||||
var in: ObjectInputStream = null
|
||||
|
|
@ -132,11 +133,11 @@ class FileBenchResultRepository extends BenchResultRepository {
|
|||
case e: Throwable ⇒
|
||||
None
|
||||
} finally {
|
||||
if (in ne null) try { in.close() } catch { case ignore: Exception ⇒ }
|
||||
if (in ne null) try in.close() catch { case ignore: Exception ⇒ }
|
||||
}
|
||||
}
|
||||
|
||||
result.flatten.toSeq.sortBy(_.timestamp)
|
||||
result.flatten.toVector.sortBy(_.timestamp)
|
||||
}
|
||||
|
||||
loadFiles()
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@ package akka.performance.workbench
|
|||
import java.io.UnsupportedEncodingException
|
||||
import java.net.URLEncoder
|
||||
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Generates URLs to Google Chart API http://code.google.com/apis/chart/
|
||||
|
|
@ -16,7 +16,7 @@ object GoogleChartBuilder {
|
|||
/**
|
||||
* Builds a bar chart for tps in the statistics.
|
||||
*/
|
||||
def tpsChartUrl(statsByTimestamp: TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = {
|
||||
def tpsChartUrl(statsByTimestamp: immutable.TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = {
|
||||
if (statsByTimestamp.isEmpty) ""
|
||||
else {
|
||||
val loads = statsByTimestamp.values.head.map(_.load)
|
||||
|
|
@ -46,7 +46,7 @@ object GoogleChartBuilder {
|
|||
//sb.append("&")
|
||||
|
||||
// legend
|
||||
val legendStats = statsByTimestamp.values.map(_.head).toSeq
|
||||
val legendStats = statsByTimestamp.values.toVector.map(_.head)
|
||||
appendLegend(legendStats, sb, legend)
|
||||
sb.append("&")
|
||||
// bar spacing
|
||||
|
|
@ -60,10 +60,7 @@ object GoogleChartBuilder {
|
|||
val loadStr = loads.mkString(",")
|
||||
sb.append("chd=t:")
|
||||
val maxValue = allStats.map(_.tps).max
|
||||
val tpsSeries: Iterable[String] =
|
||||
for (statsSeq ← statsByTimestamp.values) yield {
|
||||
statsSeq.map(_.tps).mkString(",")
|
||||
}
|
||||
val tpsSeries: Iterable[String] = for (statsSeq ← statsByTimestamp.values) yield statsSeq.map(_.tps).mkString(",")
|
||||
sb.append(tpsSeries.mkString("|"))
|
||||
|
||||
// y range
|
||||
|
|
@ -83,7 +80,7 @@ object GoogleChartBuilder {
|
|||
/**
|
||||
* Builds a bar chart for all percentiles and the mean in the statistics.
|
||||
*/
|
||||
def percentilesAndMeanChartUrl(statistics: Seq[Stats], title: String, legend: Stats ⇒ String): String = {
|
||||
def percentilesAndMeanChartUrl(statistics: immutable.Seq[Stats], title: String, legend: Stats ⇒ String): String = {
|
||||
if (statistics.isEmpty) ""
|
||||
else {
|
||||
val current = statistics.last
|
||||
|
|
@ -146,13 +143,13 @@ object GoogleChartBuilder {
|
|||
}
|
||||
}
|
||||
|
||||
private def percentileLabels(percentiles: TreeMap[Int, Long], sb: StringBuilder) {
|
||||
private def percentileLabels(percentiles: immutable.TreeMap[Int, Long], sb: StringBuilder) {
|
||||
sb.append("chxl=1:|")
|
||||
val s = percentiles.keys.toList.map(_ + "%").mkString("|")
|
||||
sb.append(s)
|
||||
}
|
||||
|
||||
private def appendLegend(statistics: Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) {
|
||||
private def appendLegend(statistics: immutable.Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) {
|
||||
val legends = statistics.map(legend(_))
|
||||
sb.append("chdl=")
|
||||
val s = legends.map(urlEncode(_)).mkString("|")
|
||||
|
|
@ -166,7 +163,7 @@ object GoogleChartBuilder {
|
|||
sb.append(s)
|
||||
}
|
||||
|
||||
private def dataSeries(allPercentiles: Seq[TreeMap[Int, Long]], meanValues: Seq[Double], sb: StringBuilder) {
|
||||
private def dataSeries(allPercentiles: immutable.Seq[immutable.TreeMap[Int, Long]], meanValues: immutable.Seq[Double], sb: StringBuilder) {
|
||||
val percentileSeries =
|
||||
for {
|
||||
percentiles ← allPercentiles
|
||||
|
|
@ -181,7 +178,7 @@ object GoogleChartBuilder {
|
|||
sb.append(series.mkString("|"))
|
||||
}
|
||||
|
||||
private def dataSeries(values: Seq[Double], sb: StringBuilder) {
|
||||
private def dataSeries(values: immutable.Seq[Double], sb: StringBuilder) {
|
||||
val series = values.map(formatDouble(_))
|
||||
sb.append(series.mkString("|"))
|
||||
}
|
||||
|
|
@ -198,7 +195,7 @@ object GoogleChartBuilder {
|
|||
}
|
||||
}
|
||||
|
||||
def latencyAndThroughputChartUrl(statistics: Seq[Stats], title: String): String = {
|
||||
def latencyAndThroughputChartUrl(statistics: immutable.Seq[Stats], title: String): String = {
|
||||
if (statistics.isEmpty) ""
|
||||
else {
|
||||
val sb = new StringBuilder
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ import java.text.SimpleDateFormat
|
|||
import java.util.Date
|
||||
import akka.actor.ActorSystem
|
||||
import akka.event.Logging
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.immutable
|
||||
|
||||
class Report(
|
||||
system: ActorSystem,
|
||||
|
|
@ -19,7 +19,7 @@ class Report(
|
|||
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
|
||||
val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss")
|
||||
|
||||
def html(statistics: Seq[Stats]) {
|
||||
def html(statistics: immutable.Seq[Stats]) {
|
||||
|
||||
val current = statistics.last
|
||||
val sb = new StringBuilder
|
||||
|
|
@ -80,13 +80,13 @@ class Report(
|
|||
chartUrl
|
||||
}
|
||||
|
||||
def comparePercentilesAndMeanChart(stats: Stats): Seq[String] = {
|
||||
def comparePercentilesAndMeanChart(stats: Stats): immutable.Seq[String] = {
|
||||
for {
|
||||
compareName ← compareResultWith.toSeq
|
||||
compareName ← compareResultWith.to[immutable.Seq]
|
||||
compareStats ← resultRepository.get(compareName, stats.load)
|
||||
} yield {
|
||||
val chartTitle = stats.name + " vs. " + compareName + ", " + stats.load + " clients" + ", Percentiles and Mean (microseconds)"
|
||||
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(Seq(compareStats, stats), chartTitle, _.name)
|
||||
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(List(compareStats, stats), chartTitle, _.name)
|
||||
chartUrl
|
||||
}
|
||||
}
|
||||
|
|
@ -102,17 +102,17 @@ class Report(
|
|||
}
|
||||
}
|
||||
|
||||
def compareWithHistoricalTpsChart(statistics: Seq[Stats]): Option[String] = {
|
||||
def compareWithHistoricalTpsChart(statistics: immutable.Seq[Stats]): Option[String] = {
|
||||
|
||||
if (statistics.isEmpty) {
|
||||
None
|
||||
} else {
|
||||
val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp)
|
||||
val statsByTimestamp = TreeMap[Long, Seq[Stats]]() ++
|
||||
val statsByTimestamp = immutable.TreeMap[Long, Seq[Stats]]() ++
|
||||
(for (ts ← histTimestamps) yield {
|
||||
val seq =
|
||||
for (stats ← statistics) yield {
|
||||
val withHistorical: Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load)
|
||||
val withHistorical: immutable.Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load)
|
||||
val cell = withHistorical.find(_.timestamp == ts)
|
||||
cell.getOrElse(Stats(stats.name, stats.load, ts))
|
||||
}
|
||||
|
|
@ -131,7 +131,7 @@ class Report(
|
|||
chartUrl
|
||||
}
|
||||
|
||||
def formatResultsTable(statsSeq: Seq[Stats]): String = {
|
||||
def formatResultsTable(statsSeq: immutable.Seq[Stats]): String = {
|
||||
|
||||
val name = statsSeq.head.name
|
||||
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ class CustomRouteSpec extends AkkaSpec {
|
|||
provider.createRoutees(1)
|
||||
|
||||
{
|
||||
case (sender, message: String) ⇒ Seq(Destination(sender, target))
|
||||
case (sender, message: String) ⇒ List(Destination(sender, target))
|
||||
case (sender, message) ⇒ toAll(sender, provider.routees)
|
||||
}
|
||||
}
|
||||
|
|
@ -43,8 +43,8 @@ class CustomRouteSpec extends AkkaSpec {
|
|||
val r = Await.result(router.ask(CurrentRoutees)(1 second).
|
||||
mapTo[RouterRoutees], 1 second)
|
||||
r.routees.size must be(1)
|
||||
route(testActor -> "hallo") must be(Seq(Destination(testActor, target)))
|
||||
route(testActor -> 12) must be(Seq(Destination(testActor, r.routees.head)))
|
||||
route(testActor -> "hallo") must be(List(Destination(testActor, target)))
|
||||
route(testActor -> 12) must be(List(Destination(testActor, r.routees.head)))
|
||||
//#test-route
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,10 +10,9 @@ import akka.testkit.TestEvent._
|
|||
import akka.actor.Props
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.pattern.ask
|
||||
import java.util.concurrent.TimeoutException
|
||||
import scala.util.Try
|
||||
|
||||
object ResizerSpec {
|
||||
|
|
@ -61,10 +60,10 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
lowerBound = 2,
|
||||
upperBound = 3)
|
||||
|
||||
val c1 = resizer.capacity(IndexedSeq.empty[ActorRef])
|
||||
val c1 = resizer.capacity(immutable.IndexedSeq.empty[ActorRef])
|
||||
c1 must be(2)
|
||||
|
||||
val current = IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor]))
|
||||
val current = immutable.IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor]))
|
||||
val c2 = resizer.capacity(current)
|
||||
c2 must be(0)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ import language.postfixOps
|
|||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor._
|
||||
import scala.collection.mutable.LinkedList
|
||||
import scala.collection.immutable
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -17,7 +17,7 @@ import akka.pattern.{ ask, pipe }
|
|||
import java.util.concurrent.ConcurrentHashMap
|
||||
import com.typesafe.config.Config
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.util.Timeout
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
|
||||
object RoutingSpec {
|
||||
|
||||
|
|
@ -54,11 +54,10 @@ object RoutingSpec {
|
|||
class MyRouter(config: Config) extends RouterConfig {
|
||||
val foo = config.getString("foo")
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
val routees = IndexedSeq(routeeProvider.context.actorOf(Props[Echo]))
|
||||
routeeProvider.registerRoutees(routees)
|
||||
routeeProvider.registerRoutees(List(routeeProvider.context.actorOf(Props[Echo])))
|
||||
|
||||
{
|
||||
case (sender, message) ⇒ Nil
|
||||
case (sender, message) ⇒ EmptyImmutableSeq
|
||||
}
|
||||
}
|
||||
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||
|
|
@ -251,15 +250,15 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
//lets create some connections.
|
||||
var actors = new LinkedList[ActorRef]
|
||||
var counters = new LinkedList[AtomicInteger]
|
||||
@volatile var actors = immutable.IndexedSeq[ActorRef]()
|
||||
@volatile var counters = immutable.IndexedSeq[AtomicInteger]()
|
||||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters.get(i).get.addAndGet(msg)
|
||||
case msg: Int ⇒ counters(i).addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
actors = actors :+ actor
|
||||
|
|
@ -278,10 +277,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
//now wait some and do validations.
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
for (i ← 0 until connectionCount) {
|
||||
val counter = counters.get(i).get
|
||||
counter.get must be((iterationCount * (i + 1)))
|
||||
}
|
||||
for (i ← 0 until connectionCount)
|
||||
counters(i).get must be((iterationCount * (i + 1)))
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ import scala.collection.Seq;
|
|||
public class JAPI {
|
||||
|
||||
public static <T> Seq<T> seq(T... ts) {
|
||||
return Util.arrayToSeq(ts);
|
||||
return Util.immutableSeq(ts);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.actor
|
|||
|
||||
import java.io.{ ObjectOutputStream, NotSerializableException }
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.TreeSet
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.dungeon.ChildrenContainer
|
||||
|
|
@ -108,7 +108,7 @@ trait ActorContext extends ActorRefFactory {
|
|||
* val goodLookup = context.actorFor("kid")
|
||||
* }}}
|
||||
*/
|
||||
def children: Iterable[ActorRef]
|
||||
def children: immutable.Iterable[ActorRef]
|
||||
|
||||
/**
|
||||
* Get the child with the given name if it exists.
|
||||
|
|
@ -287,7 +287,7 @@ private[akka] object ActorCell {
|
|||
|
||||
final val emptyBehaviorStack: List[Actor.Receive] = Nil
|
||||
|
||||
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
|
||||
final val emptyActorRefSet: Set[ActorRef] = immutable.TreeSet.empty
|
||||
}
|
||||
|
||||
//ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit)
|
||||
|
|
|
|||
|
|
@ -3,6 +3,8 @@
|
|||
*/
|
||||
package akka.actor
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import akka.japi.Util.immutableSeq
|
||||
import java.net.MalformedURLException
|
||||
|
||||
object ActorPath {
|
||||
|
|
@ -20,6 +22,8 @@ object ActorPath {
|
|||
* http://www.ietf.org/rfc/rfc2396.txt
|
||||
*/
|
||||
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
|
||||
|
||||
private[akka] final val emptyActorPath: immutable.Iterable[String] = List("")
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -68,23 +72,18 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
|
|||
/**
|
||||
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
||||
*/
|
||||
def descendant(names: java.lang.Iterable[String]): ActorPath = {
|
||||
import scala.collection.JavaConverters._
|
||||
/(names.asScala)
|
||||
}
|
||||
def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
|
||||
|
||||
/**
|
||||
* Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
||||
*/
|
||||
def elements: Iterable[String]
|
||||
def elements: immutable.Iterable[String]
|
||||
|
||||
/**
|
||||
* ''Java API'': Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
||||
*/
|
||||
def getElements: java.lang.Iterable[String] = {
|
||||
import scala.collection.JavaConverters._
|
||||
elements.asJava
|
||||
}
|
||||
def getElements: java.lang.Iterable[String] =
|
||||
scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava
|
||||
|
||||
/**
|
||||
* Walk up the tree to obtain and return the RootActorPath.
|
||||
|
|
@ -112,7 +111,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
|||
|
||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
|
||||
override val elements: Iterable[String] = List("")
|
||||
override def elements: immutable.Iterable[String] = ActorPath.emptyActorPath
|
||||
|
||||
override val toString: String = address + name
|
||||
|
||||
|
|
@ -121,7 +120,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
|||
else addr + name
|
||||
|
||||
override def compareTo(other: ActorPath): Int = other match {
|
||||
case r: RootActorPath ⇒ toString compareTo r.toString
|
||||
case r: RootActorPath ⇒ toString compareTo r.toString // FIXME make this cheaper by comparing address and name in isolation
|
||||
case c: ChildActorPath ⇒ 1
|
||||
}
|
||||
}
|
||||
|
|
@ -134,9 +133,9 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
|||
|
||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
|
||||
override def elements: Iterable[String] = {
|
||||
override def elements: immutable.Iterable[String] = {
|
||||
@tailrec
|
||||
def rec(p: ActorPath, acc: List[String]): Iterable[String] = p match {
|
||||
def rec(p: ActorPath, acc: List[String]): immutable.Iterable[String] = p match {
|
||||
case r: RootActorPath ⇒ acc
|
||||
case _ ⇒ rec(p.parent, p.name :: acc)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,8 +8,9 @@ import akka.dispatch._
|
|||
import akka.routing._
|
||||
import akka.event._
|
||||
import akka.util.{ Switch, Helpers }
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import scala.util.{ Success, Failure }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
|
|
@ -271,10 +272,7 @@ trait ActorRefFactory {
|
|||
*
|
||||
* For maximum performance use a collection with efficient head & tail operations.
|
||||
*/
|
||||
def actorFor(path: java.lang.Iterable[String]): ActorRef = {
|
||||
import scala.collection.JavaConverters._
|
||||
provider.actorFor(lookupRoot, path.asScala)
|
||||
}
|
||||
def actorFor(path: java.lang.Iterable[String]): ActorRef = provider.actorFor(lookupRoot, immutableSeq(path))
|
||||
|
||||
/**
|
||||
* Construct an [[akka.actor.ActorSelection]] from the given path, which is
|
||||
|
|
@ -480,7 +478,7 @@ class LocalActorRefProvider(
|
|||
def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras
|
||||
|
||||
private def guardianSupervisorStrategyConfigurator =
|
||||
dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Seq()).get
|
||||
dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, EmptyImmutableSeq).get
|
||||
|
||||
/**
|
||||
* Overridable supervision strategy to be used by the “/user” guardian.
|
||||
|
|
|
|||
|
|
@ -6,20 +6,20 @@ package akka.actor
|
|||
|
||||
import akka.event._
|
||||
import akka.dispatch._
|
||||
import akka.pattern.ask
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration.Duration
|
||||
import java.io.Closeable
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.{ FiniteDuration, Duration }
|
||||
import scala.concurrent.{ Await, Awaitable, CanAwait, Future }
|
||||
import scala.util.{ Failure, Success }
|
||||
import scala.util.control.NonFatal
|
||||
import akka.util._
|
||||
import java.io.Closeable
|
||||
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
|
||||
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import akka.actor.dungeon.ChildrenContainer
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import util.{ Failure, Success }
|
||||
|
||||
object ActorSystem {
|
||||
|
||||
|
|
@ -144,7 +144,7 @@ object ActorSystem {
|
|||
|
||||
final val LogLevel: String = getString("akka.loglevel")
|
||||
final val StdoutLogLevel: String = getString("akka.stdout-loglevel")
|
||||
final val EventHandlers: Seq[String] = getStringList("akka.event-handlers").asScala
|
||||
final val EventHandlers: immutable.Seq[String] = immutableSeq(getStringList("akka.event-handlers"))
|
||||
final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
|
||||
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")
|
||||
|
||||
|
|
@ -273,10 +273,7 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
/**
|
||||
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
||||
*/
|
||||
def descendant(names: java.lang.Iterable[String]): ActorPath = {
|
||||
import scala.collection.JavaConverters._
|
||||
/(names.asScala)
|
||||
}
|
||||
def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
|
||||
|
||||
/**
|
||||
* 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 provider: ActorRefProvider = {
|
||||
val arguments = Seq(
|
||||
val arguments = Vector(
|
||||
classOf[String] -> name,
|
||||
classOf[Settings] -> settings,
|
||||
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
|
||||
|
||||
private def loadExtensions() {
|
||||
import scala.collection.JavaConverters.collectionAsScalaIterableConverter
|
||||
settings.config.getStringList("akka.extensions").asScala foreach { fqcn ⇒
|
||||
dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()) } match {
|
||||
immutableSeq(settings.config.getStringList("akka.extensions")) foreach { fqcn ⇒
|
||||
dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match {
|
||||
case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup())
|
||||
case Success(p: ExtensionId[_]) ⇒ registerExtension(p)
|
||||
case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||
|
|
|
|||
|
|
@ -5,7 +5,8 @@ package akka.actor
|
|||
import java.net.URI
|
||||
import java.net.URISyntaxException
|
||||
import java.net.MalformedURLException
|
||||
import annotation.tailrec
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* The address specifies the physical location under which an Actor can be
|
||||
|
|
@ -71,7 +72,7 @@ private[akka] trait PathUtils {
|
|||
}
|
||||
|
||||
object RelativeActorPath extends PathUtils {
|
||||
def unapply(addr: String): Option[Iterable[String]] = {
|
||||
def unapply(addr: String): Option[immutable.Seq[String]] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.isAbsolute) None
|
||||
|
|
@ -119,13 +120,12 @@ object AddressFromURIString {
|
|||
* Given an ActorPath it returns the Address and the path elements if the path is well-formed
|
||||
*/
|
||||
object ActorPathExtractor extends PathUtils {
|
||||
def unapply(addr: String): Option[(Address, Iterable[String])] =
|
||||
def unapply(addr: String): Option[(Address, immutable.Iterable[String])] =
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getRawPath == null) None
|
||||
else AddressFromURIString.unapply(uri) match {
|
||||
case None ⇒ None
|
||||
case Some(addr) ⇒ Some((addr, split(uri.getRawPath).drop(1)))
|
||||
uri.getRawPath match {
|
||||
case null ⇒ None
|
||||
case path ⇒ AddressFromURIString.unapply(uri).map((_, split(path).drop(1)))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
|
|
|
|||
|
|
@ -7,10 +7,11 @@ package akka.actor
|
|||
import scala.concurrent.duration.Duration
|
||||
import com.typesafe.config._
|
||||
import akka.routing._
|
||||
import akka.japi.Util.immutableSeq
|
||||
import java.util.concurrent.{ TimeUnit }
|
||||
import akka.util.WildcardTree
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import annotation.tailrec
|
||||
import scala.annotation.tailrec
|
||||
|
||||
/**
|
||||
* This class represents deployment configuration for a given actor path. It is
|
||||
|
|
@ -151,7 +152,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
* @param deployment the deployment config, with defaults
|
||||
*/
|
||||
protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
|
||||
val routees = Vector() ++ deployment.getStringList("routees.paths").asScala
|
||||
val routees = immutableSeq(deployment.getStringList("routees.paths"))
|
||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
|
||||
|
||||
|
|
@ -168,7 +169,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
val vnodes = deployment.getInt("virtual-nodes-factor")
|
||||
ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes)
|
||||
case fqn ⇒
|
||||
val args = Seq(classOf[Config] -> deployment)
|
||||
val args = List(classOf[Config] -> deployment)
|
||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({
|
||||
case exception ⇒ throw new IllegalArgumentException(
|
||||
("Cannot instantiate router [%s], defined in [%s], " +
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.actor
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
import scala.collection.immutable
|
||||
import java.lang.reflect.InvocationTargetException
|
||||
import scala.reflect.ClassTag
|
||||
import scala.util.Try
|
||||
|
|
@ -25,7 +25,7 @@ abstract class DynamicAccess {
|
|||
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
|
||||
* }}}
|
||||
*/
|
||||
def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T]
|
||||
def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T]
|
||||
|
||||
/**
|
||||
* Obtain a `Class[_]` object loaded with the right class loader (i.e. the one
|
||||
|
|
@ -40,7 +40,7 @@ abstract class DynamicAccess {
|
|||
* `args` argument. The exact usage of args depends on which type is requested,
|
||||
* see the relevant requesting code for details.
|
||||
*/
|
||||
def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T]
|
||||
def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T]
|
||||
|
||||
/**
|
||||
* Obtain the Scala “object” instance for the given fully-qualified class name, if there is one.
|
||||
|
|
@ -70,7 +70,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces
|
|||
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c)
|
||||
})
|
||||
|
||||
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T] =
|
||||
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
|
||||
Try {
|
||||
val types = args.map(_._1).toArray
|
||||
val values = args.map(_._2).toArray
|
||||
|
|
@ -81,7 +81,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces
|
|||
if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t)
|
||||
} recover { case i: InvocationTargetException if i.getTargetException ne null ⇒ throw i.getTargetException }
|
||||
|
||||
override def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T] =
|
||||
override def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
|
||||
getClassFor(fqcn) flatMap { c ⇒ createInstanceFor(c, args) }
|
||||
|
||||
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {
|
||||
|
|
|
|||
|
|
@ -98,5 +98,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext
|
|||
def this(clazz: Class[T]) = this()(ClassTag(clazz))
|
||||
|
||||
override def lookup(): ExtensionId[T] = this
|
||||
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, Seq(classOf[ExtendedActorSystem] -> system)).get
|
||||
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] -> system)).get
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,10 +5,13 @@ package akka.actor
|
|||
|
||||
import language.implicitConversions
|
||||
|
||||
import java.util.concurrent.TimeUnit
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Duration
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
|
|
@ -170,7 +173,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
* Implicit conversion from `Seq` of Throwables to a `Decider`.
|
||||
* This maps the given Throwables to restarts, otherwise escalates.
|
||||
*/
|
||||
implicit def seqThrowable2Decider(trapExit: Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit)
|
||||
implicit def seqThrowable2Decider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit)
|
||||
|
||||
type Decider = PartialFunction[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
|
||||
* the given Throwables matches the cause and restarts, otherwise escalates.
|
||||
*/
|
||||
def makeDecider(trapExit: Array[Class[_]]): Decider =
|
||||
{ case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate }
|
||||
|
||||
/**
|
||||
* 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)
|
||||
def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = {
|
||||
case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate
|
||||
}
|
||||
|
||||
/**
|
||||
* Decider builder which just checks whether one of
|
||||
* the given Throwables matches the cause and restarts, otherwise escalates.
|
||||
*/
|
||||
def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(immutableSeq(trapExit))
|
||||
|
||||
/**
|
||||
* 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
|
||||
|
|
@ -222,14 +216,14 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
||||
private[akka] def sort(in: Iterable[CauseDirective]): immutable.Seq[CauseDirective] =
|
||||
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒
|
||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||
case -1 ⇒ buf append ca
|
||||
case x ⇒ buf insert (x, ca)
|
||||
}
|
||||
buf
|
||||
}
|
||||
}.to[immutable.IndexedSeq]
|
||||
|
||||
private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] =
|
||||
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]]) =
|
||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||
|
||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) =
|
||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||
|
||||
/*
|
||||
* this is a performance optimization to avoid re-allocating the pairs upon
|
||||
* every call to requestRestartPermission, assuming that strategies are shared
|
||||
|
|
@ -380,9 +370,6 @@ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
|
|||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
|
||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||
|
||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) =
|
||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||
|
||||
/*
|
||||
* this is a performance optimization to avoid re-allocating the pairs upon
|
||||
* every call to requestRestartPermission, assuming that strategies are shared
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package akka.actor
|
|||
import language.higherKinds
|
||||
import language.postfixOps
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.util.control.NonFatal
|
||||
|
|
@ -122,7 +123,7 @@ object IO {
|
|||
* @return a new SocketHandle that can be used to perform actions on the
|
||||
* new connection's SocketChannel.
|
||||
*/
|
||||
def accept(options: Seq[SocketOption] = Seq.empty)(implicit socketOwner: ActorRef): SocketHandle = {
|
||||
def accept(options: immutable.Seq[SocketOption] = Nil)(implicit socketOwner: ActorRef): SocketHandle = {
|
||||
val socket = SocketHandle(socketOwner, ioManager)
|
||||
ioManager ! Accept(socket, this, options)
|
||||
socket
|
||||
|
|
@ -250,7 +251,7 @@ object IO {
|
|||
*
|
||||
* Normally sent using IOManager.listen()
|
||||
*/
|
||||
case class Listen(server: ServerHandle, address: SocketAddress, options: Seq[ServerSocketOption] = Seq.empty) extends IOMessage
|
||||
case class Listen(server: ServerHandle, address: SocketAddress, options: immutable.Seq[ServerSocketOption] = Nil) extends IOMessage
|
||||
|
||||
/**
|
||||
* Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is
|
||||
|
|
@ -272,7 +273,7 @@ object IO {
|
|||
*
|
||||
* Normally sent using [[akka.actor.IO.ServerHandle]].accept()
|
||||
*/
|
||||
case class Accept(socket: SocketHandle, server: ServerHandle, options: Seq[SocketOption] = Seq.empty) extends IOMessage
|
||||
case class Accept(socket: SocketHandle, server: ServerHandle, options: immutable.Seq[SocketOption] = Nil) extends IOMessage
|
||||
|
||||
/**
|
||||
* Message to an [[akka.actor.IOManager]] to create a SocketChannel connected
|
||||
|
|
@ -280,7 +281,7 @@ object IO {
|
|||
*
|
||||
* Normally sent using IOManager.connect()
|
||||
*/
|
||||
case class Connect(socket: SocketHandle, address: SocketAddress, options: Seq[SocketOption] = Seq.empty) extends IOMessage
|
||||
case class Connect(socket: SocketHandle, address: SocketAddress, options: immutable.Seq[SocketOption] = Nil) extends IOMessage
|
||||
|
||||
/**
|
||||
* Message from an [[akka.actor.IOManager]] that the SocketChannel has
|
||||
|
|
@ -832,7 +833,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
|||
* @param option Seq of [[akka.actor.IO.ServerSocketOptions]] to setup on socket
|
||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||
*/
|
||||
def listen(address: SocketAddress, options: Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = {
|
||||
def listen(address: SocketAddress, options: immutable.Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = {
|
||||
val server = IO.ServerHandle(owner, actor)
|
||||
actor ! IO.Listen(server, address, options)
|
||||
server
|
||||
|
|
@ -847,7 +848,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
|||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||
*/
|
||||
def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Seq.empty)
|
||||
def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Nil)
|
||||
|
||||
/**
|
||||
* Create a ServerSocketChannel listening on a host and port. Messages will
|
||||
|
|
@ -860,7 +861,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
|||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||
*/
|
||||
def listen(host: String, port: Int, options: Seq[IO.ServerSocketOption] = Seq.empty)(implicit owner: ActorRef): IO.ServerHandle =
|
||||
def listen(host: String, port: Int, options: immutable.Seq[IO.ServerSocketOption] = Nil)(implicit owner: ActorRef): IO.ServerHandle =
|
||||
listen(new InetSocketAddress(host, port), options)(owner)
|
||||
|
||||
/**
|
||||
|
|
@ -873,7 +874,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
|||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||
* @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket
|
||||
*/
|
||||
def connect(address: SocketAddress, options: Seq[IO.SocketOption] = Seq.empty)(implicit owner: ActorRef): IO.SocketHandle = {
|
||||
def connect(address: SocketAddress, options: immutable.Seq[IO.SocketOption] = Nil)(implicit owner: ActorRef): IO.SocketHandle = {
|
||||
val socket = IO.SocketHandle(owner, actor)
|
||||
actor ! IO.Connect(socket, address, options)
|
||||
socket
|
||||
|
|
@ -991,7 +992,7 @@ final class IOManagerActor(val settings: Settings) extends Actor with ActorLoggi
|
|||
|
||||
private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) }
|
||||
|
||||
private def setSocketOptions(socket: java.net.Socket, options: Seq[IO.SocketOption]) {
|
||||
private def setSocketOptions(socket: java.net.Socket, options: immutable.Seq[IO.SocketOption]) {
|
||||
options foreach {
|
||||
case IO.KeepAlive(on) ⇒ forwardFailure(socket.setKeepAlive(on))
|
||||
case IO.OOBInline(on) ⇒ forwardFailure(socket.setOOBInline(on))
|
||||
|
|
|
|||
|
|
@ -203,8 +203,8 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter)
|
|||
}
|
||||
|
||||
override def close(): Unit = {
|
||||
import scala.collection.JavaConverters._
|
||||
hashedWheelTimer.stop().asScala foreach execDirectly
|
||||
val i = hashedWheelTimer.stop().iterator()
|
||||
while (i.hasNext) execDirectly(i.next())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,22 +4,25 @@
|
|||
package akka.actor
|
||||
|
||||
import language.existentials
|
||||
import akka.japi.{ Creator, Option ⇒ JOption }
|
||||
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
|
||||
import akka.util.Timeout
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.{ Try, Success, Failure }
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.reflect.ClassTag
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import akka.japi.{ Creator, Option ⇒ JOption }
|
||||
import akka.japi.Util.{ immutableSeq, immutableSingletonSeq }
|
||||
import akka.util.Timeout
|
||||
import akka.util.Reflect.instantiator
|
||||
import akka.serialization.{ JavaSerializer, SerializationExtension }
|
||||
import akka.dispatch._
|
||||
import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
|
||||
import java.util.concurrent.TimeoutException
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import scala.reflect.ClassTag
|
||||
import akka.serialization.{ JavaSerializer, SerializationExtension }
|
||||
import java.io.ObjectStreamException
|
||||
import scala.util.{ Try, Success, Failure }
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
|
||||
|
||||
/**
|
||||
* A TypedActorFactory is something that can created TypedActor instances.
|
||||
|
|
@ -439,8 +442,8 @@ object TypedProps {
|
|||
* @return a sequence of interfaces that the specified class implements,
|
||||
* or a sequence containing only itself, if itself is an interface.
|
||||
*/
|
||||
def extractInterfaces(clazz: Class[_]): Seq[Class[_]] =
|
||||
if (clazz.isInterface) Seq[Class[_]](clazz) else clazz.getInterfaces.toList
|
||||
def extractInterfaces(clazz: Class[_]): immutable.Seq[Class[_]] =
|
||||
if (clazz.isInterface) immutableSingletonSeq(clazz) else immutableSeq(clazz.getInterfaces)
|
||||
|
||||
/**
|
||||
* Uses the supplied class as the factory for the TypedActor implementation,
|
||||
|
|
@ -489,7 +492,7 @@ object TypedProps {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class TypedProps[T <: AnyRef] protected[TypedProps] (
|
||||
interfaces: Seq[Class[_]],
|
||||
interfaces: immutable.Seq[Class[_]],
|
||||
creator: () ⇒ T,
|
||||
dispatcher: String = TypedProps.defaultDispatcherId,
|
||||
deploy: Deploy = Props.defaultDeploy,
|
||||
|
|
|
|||
|
|
@ -5,14 +5,12 @@
|
|||
package akka.actor.dungeon
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters.asJavaIterableConverter
|
||||
import scala.util.control.NonFatal
|
||||
import scala.collection.immutable
|
||||
import akka.actor._
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.ActorPath.ElementRegex
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.util.{ Unsafe, Helpers }
|
||||
import akka.actor.ChildNameReserved
|
||||
|
||||
private[akka] trait Children { this: ActorCell ⇒
|
||||
|
||||
|
|
@ -24,8 +22,9 @@ private[akka] trait Children { this: ActorCell ⇒
|
|||
def childrenRefs: ChildrenContainer =
|
||||
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer]
|
||||
|
||||
final def children: Iterable[ActorRef] = childrenRefs.children
|
||||
final def getChildren(): java.lang.Iterable[ActorRef] = children.asJava
|
||||
final def children: immutable.Iterable[ActorRef] = childrenRefs.children
|
||||
final def getChildren(): java.lang.Iterable[ActorRef] =
|
||||
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
|
||||
|
||||
final def child(name: String): Option[ActorRef] = Option(getChild(name))
|
||||
final def getChild(name: String): ActorRef = childrenRefs.getByName(name) match {
|
||||
|
|
@ -141,7 +140,7 @@ private[akka] trait Children { this: ActorCell ⇒
|
|||
|
||||
protected def getChildByRef(ref: ActorRef): Option[ChildRestartStats] = childrenRefs.getByRef(ref)
|
||||
|
||||
protected def getAllChildStats: Iterable[ChildRestartStats] = childrenRefs.stats
|
||||
protected def getAllChildStats: immutable.Iterable[ChildRestartStats] = childrenRefs.stats
|
||||
|
||||
protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = {
|
||||
childrenRefs match {
|
||||
|
|
|
|||
|
|
@ -4,10 +4,11 @@
|
|||
|
||||
package akka.actor.dungeon
|
||||
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.immutable
|
||||
|
||||
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
||||
import akka.dispatch.SystemMessage
|
||||
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -20,8 +21,8 @@ private[akka] trait ChildrenContainer {
|
|||
def getByName(name: String): Option[ChildStats]
|
||||
def getByRef(actor: ActorRef): Option[ChildRestartStats]
|
||||
|
||||
def children: Iterable[ActorRef]
|
||||
def stats: Iterable[ChildRestartStats]
|
||||
def children: immutable.Iterable[ActorRef]
|
||||
def stats: immutable.Iterable[ChildRestartStats]
|
||||
|
||||
def shallDie(actor: ActorRef): ChildrenContainer
|
||||
|
||||
|
|
@ -49,6 +50,18 @@ private[akka] object ChildrenContainer {
|
|||
case class Creation() extends SuspendReason with WaitingForChildren
|
||||
case object Termination extends SuspendReason
|
||||
|
||||
class ChildRestartsIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] {
|
||||
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats]
|
||||
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||
override final def valuesIterator = stats.valuesIterator
|
||||
}
|
||||
|
||||
class ChildrenIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] {
|
||||
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child
|
||||
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||
override final def valuesIterator = stats.valuesIterator
|
||||
}
|
||||
|
||||
trait WaitingForChildren {
|
||||
private var todo: SystemMessage = null
|
||||
def enqueue(message: SystemMessage) = { message.next = todo; todo = message }
|
||||
|
|
@ -56,13 +69,13 @@ private[akka] object ChildrenContainer {
|
|||
}
|
||||
|
||||
trait EmptyChildrenContainer extends ChildrenContainer {
|
||||
val emptyStats = TreeMap.empty[String, ChildStats]
|
||||
val emptyStats = immutable.TreeMap.empty[String, ChildStats]
|
||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, stats))
|
||||
override def remove(child: ActorRef): ChildrenContainer = this
|
||||
override def getByName(name: String): Option[ChildRestartStats] = None
|
||||
override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
|
||||
override def children: Iterable[ActorRef] = Nil
|
||||
override def stats: Iterable[ChildRestartStats] = Nil
|
||||
override def children: immutable.Iterable[ActorRef] = EmptyImmutableSeq
|
||||
override def stats: immutable.Iterable[ChildRestartStats] = EmptyImmutableSeq
|
||||
override def shallDie(actor: ActorRef): ChildrenContainer = this
|
||||
override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
|
||||
override def unreserve(name: String): ChildrenContainer = this
|
||||
|
|
@ -95,7 +108,7 @@ private[akka] object ChildrenContainer {
|
|||
* calling context.stop(child) and processing the ChildTerminated() system
|
||||
* message).
|
||||
*/
|
||||
class NormalChildrenContainer(val c: TreeMap[String, ChildStats]) extends ChildrenContainer {
|
||||
class NormalChildrenContainer(val c: immutable.TreeMap[String, ChildStats]) extends ChildrenContainer {
|
||||
|
||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(c.updated(name, stats))
|
||||
|
||||
|
|
@ -108,9 +121,11 @@ private[akka] object ChildrenContainer {
|
|||
case _ ⇒ None
|
||||
}
|
||||
|
||||
override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child }
|
||||
override def children: immutable.Iterable[ActorRef] =
|
||||
if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c)
|
||||
|
||||
override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c }
|
||||
override def stats: immutable.Iterable[ChildRestartStats] =
|
||||
if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c)
|
||||
|
||||
override def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
|
||||
|
||||
|
|
@ -130,7 +145,7 @@ private[akka] object ChildrenContainer {
|
|||
}
|
||||
|
||||
object NormalChildrenContainer {
|
||||
def apply(c: TreeMap[String, ChildStats]): ChildrenContainer =
|
||||
def apply(c: immutable.TreeMap[String, ChildStats]): ChildrenContainer =
|
||||
if (c.isEmpty) EmptyChildrenContainer
|
||||
else new NormalChildrenContainer(c)
|
||||
}
|
||||
|
|
@ -145,7 +160,7 @@ private[akka] object ChildrenContainer {
|
|||
* type of container, depending on whether or not children are left and whether or not
|
||||
* the reason was “Terminating”.
|
||||
*/
|
||||
case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
|
||||
case class TerminatingChildrenContainer(c: immutable.TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
|
||||
extends ChildrenContainer {
|
||||
|
||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats))
|
||||
|
|
@ -166,9 +181,11 @@ private[akka] object ChildrenContainer {
|
|||
case _ ⇒ None
|
||||
}
|
||||
|
||||
override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child }
|
||||
override def children: immutable.Iterable[ActorRef] =
|
||||
if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c)
|
||||
|
||||
override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c }
|
||||
override def stats: immutable.Iterable[ChildRestartStats] =
|
||||
if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c)
|
||||
|
||||
override def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
|
||||
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.dispatch._
|
|||
import akka.event.Logging.{ Warning, Error, Debug }
|
||||
import scala.util.control.NonFatal
|
||||
import akka.event.Logging
|
||||
import scala.Some
|
||||
import scala.collection.immutable
|
||||
import akka.dispatch.ChildTerminated
|
||||
import akka.actor.PreRestartException
|
||||
import akka.actor.Failed
|
||||
|
|
@ -160,7 +160,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
|
|||
}
|
||||
}
|
||||
|
||||
final def handleInvokeFailure(childrenNotToSuspend: Iterable[ActorRef], t: Throwable, message: String): Unit = {
|
||||
final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable, message: String): Unit = {
|
||||
publish(Error(t, self.path.toString, clazz(actor), message))
|
||||
// prevent any further messages to be processed until the actor has been restarted
|
||||
if (!isFailed) try {
|
||||
|
|
|
|||
|
|
@ -420,7 +420,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
case "unbounded" ⇒ UnboundedMailbox()
|
||||
case "bounded" ⇒ new BoundedMailbox(prerequisites.settings, config)
|
||||
case fqcn ⇒
|
||||
val args = Seq(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config)
|
||||
val args = List(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config)
|
||||
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({
|
||||
case exception ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
@ -436,7 +436,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
|||
case null | "" | "fork-join-executor" ⇒ new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites)
|
||||
case "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
|
||||
case fqcn ⇒
|
||||
val args = Seq(
|
||||
val args = List(
|
||||
classOf[Config] -> config,
|
||||
classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||
case fqn ⇒
|
||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
val args = List(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({
|
||||
case exception ⇒
|
||||
throw new IllegalArgumentException(
|
||||
|
|
|
|||
|
|
@ -95,7 +95,7 @@ object Futures {
|
|||
*/
|
||||
def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = {
|
||||
implicit val ec = executor
|
||||
Future.find[T](futures.asScala)(predicate.apply(_))(executor).map(JOption.fromScalaOption(_))
|
||||
Future.find[T](futures.asScala)(predicate.apply(_))(executor) map JOption.fromScalaOption
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import java.util.concurrent.ConcurrentSkipListSet
|
|||
import java.util.Comparator
|
||||
import akka.util.{ Subclassification, SubclassifiedIndex }
|
||||
import scala.collection.immutable.TreeSet
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Represents the base type for EventBuses
|
||||
|
|
@ -167,12 +168,12 @@ trait SubchannelClassification { this: EventBus ⇒
|
|||
recv foreach (publish(event, _))
|
||||
}
|
||||
|
||||
private def removeFromCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||
private def removeFromCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||
cache = (cache /: changes) {
|
||||
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) -- cs)
|
||||
}
|
||||
|
||||
private def addToCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||
private def addToCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||
cache = (cache /: changes) {
|
||||
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs)
|
||||
}
|
||||
|
|
@ -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
|
||||
def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = {
|
||||
def dissociateAsMonitored(monitored: ActorRef): immutable.Iterable[ActorRef] = {
|
||||
val current = mappings get monitored
|
||||
current match {
|
||||
case null ⇒ empty
|
||||
|
|
|
|||
|
|
@ -9,12 +9,13 @@ import akka.actor._
|
|||
import akka.{ ConfigurationException, AkkaException }
|
||||
import akka.actor.ActorSystem.Settings
|
||||
import akka.util.{ Timeout, ReentrantGuard }
|
||||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.util.control.NoStackTrace
|
||||
import java.util.concurrent.TimeoutException
|
||||
import scala.annotation.implicitNotFound
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import annotation.implicitNotFound
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
/**
|
||||
* This trait brings log level handling to the EventStream: it reads the log
|
||||
|
|
@ -448,7 +449,7 @@ object Logging {
|
|||
}
|
||||
|
||||
// these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type
|
||||
val AllLogLevels: Seq[LogLevel] = Seq(ErrorLevel, WarningLevel, InfoLevel, DebugLevel)
|
||||
val AllLogLevels: immutable.Seq[LogLevel] = Vector(ErrorLevel, WarningLevel, InfoLevel, DebugLevel)
|
||||
|
||||
/**
|
||||
* Obtain LoggingAdapter for the given actor system and source object. This
|
||||
|
|
|
|||
|
|
@ -5,10 +5,12 @@
|
|||
package akka.japi
|
||||
|
||||
import language.implicitConversions
|
||||
import scala.Some
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.reflect.ClassTag
|
||||
import scala.util.control.NoStackTrace
|
||||
import scala.runtime.AbstractPartialFunction
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import java.util.Collections.{ emptyList, singletonList }
|
||||
|
||||
/**
|
||||
|
|
@ -174,9 +176,40 @@ object Option {
|
|||
* This class hold common utilities for Java
|
||||
*/
|
||||
object Util {
|
||||
|
||||
/**
|
||||
* Returns a ClassTag describing the provided Class.
|
||||
*
|
||||
* Java API
|
||||
*/
|
||||
def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
|
||||
|
||||
def arrayToSeq[T](arr: Array[T]): 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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable.SortedMap
|
||||
import scala.collection.immutable
|
||||
import scala.reflect.ClassTag
|
||||
import java.util.Arrays
|
||||
|
||||
|
|
@ -18,7 +18,7 @@ import java.util.Arrays
|
|||
* hash, i.e. make sure it is different for different nodes.
|
||||
*
|
||||
*/
|
||||
class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtualNodesFactor: Int) {
|
||||
class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], val virtualNodesFactor: Int) {
|
||||
|
||||
import ConsistentHash._
|
||||
|
||||
|
|
@ -106,7 +106,7 @@ class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtual
|
|||
|
||||
object ConsistentHash {
|
||||
def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = {
|
||||
new ConsistentHash(SortedMap.empty[Int, T] ++
|
||||
new ConsistentHash(immutable.SortedMap.empty[Int, T] ++
|
||||
(for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)),
|
||||
virtualNodesFactor)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,11 +3,11 @@
|
|||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
import scala.collection.immutable
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.Logging
|
||||
import akka.serialization.SerializationExtension
|
||||
|
|
@ -19,16 +19,13 @@ object ConsistentHashingRouter {
|
|||
/**
|
||||
* Creates a new ConsistentHashingRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): ConsistentHashingRouter =
|
||||
def apply(routees: immutable.Iterable[ActorRef]): ConsistentHashingRouter =
|
||||
new ConsistentHashingRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees))
|
||||
|
||||
/**
|
||||
* If you don't define the `hashMapping` when
|
||||
|
|
@ -146,7 +143,7 @@ object ConsistentHashingRouter {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class ConsistentHashingRouter(
|
||||
nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy,
|
||||
val virtualNodesFactor: Int = 0,
|
||||
|
|
@ -165,7 +162,7 @@ case class ConsistentHashingRouter(
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -227,7 +224,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
|
|||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def virtualNodesFactor: Int
|
||||
|
||||
|
|
|
|||
|
|
@ -5,18 +5,20 @@ package akka.routing
|
|||
|
||||
import language.implicitConversions
|
||||
import language.postfixOps
|
||||
import akka.actor._
|
||||
import scala.concurrent.duration._
|
||||
import akka.ConfigurationException
|
||||
import akka.pattern.pipe
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor._
|
||||
import akka.ConfigurationException
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.pattern.pipe
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config.Config
|
||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
||||
import java.util.concurrent.TimeUnit
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import akka.dispatch.Dispatchers
|
||||
import scala.annotation.tailrec
|
||||
import concurrent.ExecutionContext
|
||||
|
||||
/**
|
||||
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
|
||||
|
|
@ -50,7 +52,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
|||
private val resizeCounter = new AtomicLong
|
||||
|
||||
@volatile
|
||||
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
||||
private var _routees: immutable.IndexedSeq[ActorRef] = immutable.IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
||||
def routees = _routees
|
||||
|
||||
@volatile
|
||||
|
|
@ -75,14 +77,11 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
|||
* end of construction
|
||||
*/
|
||||
|
||||
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
|
||||
def applyRoute(sender: ActorRef, message: Any): immutable.Iterable[Destination] = message match {
|
||||
case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil
|
||||
case CurrentRoutees ⇒
|
||||
sender ! RouterRoutees(_routees)
|
||||
Nil
|
||||
case _ ⇒
|
||||
if (route.isDefinedAt(sender, message)) route(sender, message)
|
||||
else Nil
|
||||
case CurrentRoutees ⇒ sender ! RouterRoutees(_routees); Nil
|
||||
case msg if route.isDefinedAt(sender, msg) ⇒ route(sender, message)
|
||||
case _ ⇒ Nil
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -91,7 +90,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `RouterConfig.createRoute` and `Resizer.resize`
|
||||
*/
|
||||
private[akka] def addRoutees(newRoutees: Iterable[ActorRef]): Unit = {
|
||||
private[akka] def addRoutees(newRoutees: immutable.Iterable[ActorRef]): Unit = {
|
||||
_routees = _routees ++ newRoutees
|
||||
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
||||
newRoutees foreach watch
|
||||
|
|
@ -103,7 +102,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `Resizer.resize`
|
||||
*/
|
||||
private[akka] def removeRoutees(abandonedRoutees: Iterable[ActorRef]): Unit = {
|
||||
private[akka] def removeRoutees(abandonedRoutees: immutable.Iterable[ActorRef]): Unit = {
|
||||
_routees = abandonedRoutees.foldLeft(_routees) { (xs, x) ⇒ unwatch(x); xs.filterNot(_ == x) }
|
||||
}
|
||||
|
||||
|
|
@ -195,7 +194,7 @@ trait RouterConfig {
|
|||
*/
|
||||
def withFallback(other: RouterConfig): RouterConfig = this
|
||||
|
||||
protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] =
|
||||
protected def toAll(sender: ActorRef, routees: immutable.Iterable[ActorRef]): immutable.Iterable[Destination] =
|
||||
routees.map(Destination(sender, _))
|
||||
|
||||
/**
|
||||
|
|
@ -207,7 +206,7 @@ trait RouterConfig {
|
|||
/**
|
||||
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
|
||||
*/
|
||||
def verifyConfig(): Unit = {}
|
||||
def verifyConfig(): Unit = ()
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -226,7 +225,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||
*/
|
||||
def registerRoutees(routees: Iterable[ActorRef]): Unit = routedCell.addRoutees(routees)
|
||||
def registerRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.addRoutees(routees)
|
||||
|
||||
/**
|
||||
* Adds the routees to the router.
|
||||
|
|
@ -235,7 +234,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||
* Java API.
|
||||
*/
|
||||
def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(routees.asScala)
|
||||
def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(immutableSeq(routees))
|
||||
|
||||
/**
|
||||
* Removes routees from the router. This method doesn't stop the routees.
|
||||
|
|
@ -243,7 +242,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `Resizer.resize`.
|
||||
*/
|
||||
def unregisterRoutees(routees: Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees)
|
||||
def unregisterRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees)
|
||||
|
||||
/**
|
||||
* Removes routees from the router. This method doesn't stop the routees.
|
||||
|
|
@ -252,28 +251,25 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
* `Resizer.resize`.
|
||||
* JAVA API
|
||||
*/
|
||||
def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(routees.asScala)
|
||||
def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(immutableSeq(routees))
|
||||
|
||||
/**
|
||||
* Looks up routes with specified paths and registers them.
|
||||
*/
|
||||
def registerRouteesFor(paths: Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_)))
|
||||
def registerRouteesFor(paths: immutable.Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_)))
|
||||
|
||||
/**
|
||||
* Looks up routes with specified paths and registers them.
|
||||
* JAVA API
|
||||
*/
|
||||
def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(paths.asScala)
|
||||
def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(immutableSeq(paths))
|
||||
|
||||
/**
|
||||
* Creates new routees from specified `Props` and registers them.
|
||||
*/
|
||||
def createRoutees(nrOfInstances: Int): Unit = {
|
||||
if (nrOfInstances <= 0) throw new IllegalArgumentException(
|
||||
"Must specify nrOfInstances or routees for [%s]" format context.self.path.toString)
|
||||
else
|
||||
registerRoutees(IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
|
||||
}
|
||||
def createRoutees(nrOfInstances: Int): Unit =
|
||||
if (nrOfInstances <= 0) throw new IllegalArgumentException("Must specify nrOfInstances or routees for [%s]" format context.self.path.toString)
|
||||
else registerRoutees(immutable.IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
|
||||
|
||||
/**
|
||||
* Remove specified number of routees by unregister them
|
||||
|
|
@ -296,7 +292,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
* Give concurrent messages a chance to be placed in mailbox before
|
||||
* sending PoisonPill.
|
||||
*/
|
||||
protected def delayedStop(scheduler: Scheduler, abandon: Iterable[ActorRef], stopDelay: FiniteDuration): Unit = {
|
||||
protected def delayedStop(scheduler: Scheduler, abandon: immutable.Iterable[ActorRef], stopDelay: FiniteDuration): Unit = {
|
||||
if (abandon.nonEmpty) {
|
||||
if (stopDelay <= Duration.Zero) {
|
||||
abandon foreach (_ ! PoisonPill)
|
||||
|
|
@ -314,7 +310,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
|||
/**
|
||||
* All routees of the router
|
||||
*/
|
||||
def routees: IndexedSeq[ActorRef] = routedCell.routees
|
||||
def routees: immutable.IndexedSeq[ActorRef] = routedCell.routees
|
||||
|
||||
/**
|
||||
* All routees of the router
|
||||
|
|
@ -335,7 +331,7 @@ abstract class CustomRouterConfig extends RouterConfig {
|
|||
val customRoute = createCustomRoute(routeeProvider)
|
||||
|
||||
{
|
||||
case (sender, message) ⇒ customRoute.destinationsFor(sender, message).asScala
|
||||
case (sender, message) ⇒ customRoute.destinationsFor(sender, message)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -344,7 +340,13 @@ abstract class CustomRouterConfig extends RouterConfig {
|
|||
}
|
||||
|
||||
trait CustomRoute {
|
||||
def destinationsFor(sender: ActorRef, message: Any): java.lang.Iterable[Destination]
|
||||
/**
|
||||
* use akka.japi.Util.immutableSeq to convert a java.lang.Iterable to the return type needed for destinationsFor,
|
||||
* or if you just want to return a single Destination, use akka.japi.Util.immutableSingletonSeq
|
||||
*
|
||||
* Java API
|
||||
*/
|
||||
def destinationsFor(sender: ActorRef, message: Any): immutable.Seq[Destination]
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -366,7 +368,7 @@ trait Router extends Actor {
|
|||
if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProvider)) finally ab.set(false)
|
||||
|
||||
case Terminated(child) ⇒
|
||||
ref.removeRoutees(IndexedSeq(child))
|
||||
ref.removeRoutees(child :: Nil)
|
||||
if (ref.routees.isEmpty) context.stop(self)
|
||||
|
||||
}: Receive) orElse routerReceive
|
||||
|
|
@ -426,7 +428,7 @@ case object CurrentRoutees extends CurrentRoutees {
|
|||
* Message used to carry information about what routees the router is currently using.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class RouterRoutees(routees: Iterable[ActorRef])
|
||||
case class RouterRoutees(routees: immutable.Iterable[ActorRef])
|
||||
|
||||
/**
|
||||
* For every message sent to a router, its route determines a set of destinations,
|
||||
|
|
@ -494,16 +496,14 @@ object RoundRobinRouter {
|
|||
/**
|
||||
* Creates a new RoundRobinRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): RoundRobinRouter =
|
||||
def apply(routees: immutable.Iterable[ActorRef]): RoundRobinRouter =
|
||||
new RoundRobinRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
|
||||
|
|
@ -547,7 +547,7 @@ object RoundRobinRouter {
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||
extends RouterConfig with RoundRobinLike {
|
||||
|
|
@ -564,7 +564,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -602,7 +602,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
|
|||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
if (resizer.isEmpty) {
|
||||
|
|
@ -622,7 +622,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
|
|||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case msg ⇒ List(Destination(sender, getNext()))
|
||||
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -632,15 +632,13 @@ object RandomRouter {
|
|||
/**
|
||||
* Creates a new RandomRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
||||
def apply(routees: immutable.Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that randomly selects one of the target connections to send a message to.
|
||||
|
|
@ -684,7 +682,7 @@ object RandomRouter {
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
case class RandomRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||
extends RouterConfig with RandomLike {
|
||||
|
|
@ -701,7 +699,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -738,7 +736,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
|||
trait RandomLike { this: RouterConfig ⇒
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
if (resizer.isEmpty) {
|
||||
|
|
@ -756,7 +754,7 @@ trait RandomLike { this: RouterConfig ⇒
|
|||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case msg ⇒ List(Destination(sender, getNext()))
|
||||
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -766,16 +764,14 @@ object SmallestMailboxRouter {
|
|||
/**
|
||||
* Creates a new SmallestMailboxRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
def apply(routees: immutable.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
|
|
@ -828,7 +824,7 @@ object SmallestMailboxRouter {
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||
extends RouterConfig with SmallestMailboxLike {
|
||||
|
|
@ -845,7 +841,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -882,7 +878,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
|||
trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
/**
|
||||
* Returns true if the actor is currently processing a message.
|
||||
|
|
@ -954,7 +950,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
|||
// 4. An ActorRef with unknown mailbox size that isn't processing anything
|
||||
// 5. An ActorRef with a known mailbox size
|
||||
// 6. An ActorRef without any messages
|
||||
@tailrec def getNext(targets: IndexedSeq[ActorRef] = routeeProvider.routees,
|
||||
@tailrec def getNext(targets: immutable.IndexedSeq[ActorRef] = routeeProvider.routees,
|
||||
proposedTarget: ActorRef = routeeProvider.context.system.deadLetters,
|
||||
currentScore: Long = Long.MaxValue,
|
||||
at: Int = 0,
|
||||
|
|
@ -985,7 +981,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
|||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case msg ⇒ List(Destination(sender, getNext()))
|
||||
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -995,15 +991,13 @@ object BroadcastRouter {
|
|||
/**
|
||||
* Creates a new BroadcastRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||
def apply(routees: immutable.Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that uses broadcasts a message to all its connections.
|
||||
|
|
@ -1047,7 +1041,7 @@ object BroadcastRouter {
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
case class BroadcastRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||
extends RouterConfig with BroadcastLike {
|
||||
|
|
@ -1064,7 +1058,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -1102,7 +1096,7 @@ trait BroadcastLike { this: RouterConfig ⇒
|
|||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
if (resizer.isEmpty) {
|
||||
|
|
@ -1120,16 +1114,14 @@ object ScatterGatherFirstCompletedRouter {
|
|||
/**
|
||||
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||
def apply(routees: immutable.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = {
|
||||
import scala.collection.JavaConverters._
|
||||
apply(routees.asScala, within)
|
||||
}
|
||||
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||
apply(immutableSeq(routees), within)
|
||||
}
|
||||
/**
|
||||
* Simple router that broadcasts the message to all routees, and replies with the first response.
|
||||
|
|
@ -1175,7 +1167,7 @@ object ScatterGatherFirstCompletedRouter {
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: FiniteDuration,
|
||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, within: FiniteDuration,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||
|
|
@ -1196,7 +1188,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = routeePaths.asScala, within = w)
|
||||
def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = immutableSeq(routeePaths), within = w)
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -1234,7 +1226,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
|
|||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def within: FiniteDuration
|
||||
|
||||
|
|
@ -1394,7 +1386,7 @@ case class DefaultResizer(
|
|||
* @param routees The current actor in the resizer
|
||||
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero)
|
||||
*/
|
||||
def capacity(routees: IndexedSeq[ActorRef]): Int = {
|
||||
def capacity(routees: immutable.IndexedSeq[ActorRef]): Int = {
|
||||
val currentSize = routees.size
|
||||
val press = pressure(routees)
|
||||
val delta = filter(press, currentSize)
|
||||
|
|
@ -1422,7 +1414,7 @@ case class DefaultResizer(
|
|||
* @param routees the current resizer of routees
|
||||
* @return number of busy routees, between 0 and routees.size
|
||||
*/
|
||||
def pressure(routees: IndexedSeq[ActorRef]): Int = {
|
||||
def pressure(routees: immutable.IndexedSeq[ActorRef]): Int = {
|
||||
routees count {
|
||||
case a: ActorRefWithCell ⇒
|
||||
a.underlying match {
|
||||
|
|
|
|||
|
|
@ -4,10 +4,12 @@
|
|||
|
||||
package akka
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
package object routing {
|
||||
/**
|
||||
* Routing logic, partial function from (sender, message) to a
|
||||
* set of destinations.
|
||||
*/
|
||||
type Route = PartialFunction[(akka.actor.ActorRef, Any), Iterable[Destination]]
|
||||
type Route = PartialFunction[(akka.actor.ActorRef, Any), immutable.Iterable[Destination]]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,14 +4,14 @@
|
|||
|
||||
package akka.serialization
|
||||
|
||||
import akka.AkkaException
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess }
|
||||
import akka.actor.{ Extension, ExtendedActorSystem, Address }
|
||||
import akka.event.Logging
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
import java.io.NotSerializableException
|
||||
import util.{ Try, DynamicVariable }
|
||||
import scala.util.{ Try, DynamicVariable }
|
||||
import scala.collection.immutable
|
||||
|
||||
object Serialization {
|
||||
|
||||
|
|
@ -27,17 +27,13 @@ object Serialization {
|
|||
val currentTransportAddress = new DynamicVariable[Address](null)
|
||||
|
||||
class Settings(val config: Config) {
|
||||
val Serializers: Map[String, String] = configToMap("akka.actor.serializers")
|
||||
val SerializationBindings: Map[String, String] = configToMap("akka.actor.serialization-bindings")
|
||||
|
||||
private final def configToMap(path: String): Map[String, String] = {
|
||||
import scala.collection.JavaConverters._
|
||||
import config._
|
||||
|
||||
val Serializers: Map[String, String] = configToMap(getConfig("akka.actor.serializers"))
|
||||
|
||||
val SerializationBindings: Map[String, String] = configToMap(getConfig("akka.actor.serialization-bindings"))
|
||||
|
||||
private def configToMap(cfg: Config): Map[String, String] =
|
||||
cfg.root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) }
|
||||
|
||||
config.getConfig(path).root.unwrapped.asScala.mapValues(_.toString).toMap
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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.
|
||||
* Returns either the resulting object or an Exception if one was thrown.
|
||||
*/
|
||||
def deserialize(bytes: Array[Byte],
|
||||
serializerId: Int,
|
||||
clazz: Option[Class[_]]): Try[AnyRef] = Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
|
||||
def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]]): Try[AnyRef] =
|
||||
Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
|
||||
|
||||
/**
|
||||
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
|
||||
* You can specify an optional ClassLoader to load the object into.
|
||||
* Returns either the resulting object or an Exception if one was thrown.
|
||||
*/
|
||||
def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] = Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
|
||||
def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] =
|
||||
Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
|
||||
|
||||
/**
|
||||
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
||||
|
|
@ -95,9 +91,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
*/
|
||||
def serializerFor(clazz: Class[_]): Serializer =
|
||||
serializerMap.get(clazz) match {
|
||||
case null ⇒
|
||||
// bindings are ordered from most specific to least specific
|
||||
def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean =
|
||||
case null ⇒ // bindings are ordered from most specific to least specific
|
||||
def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean =
|
||||
possibilities.size == 1 ||
|
||||
(possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) ||
|
||||
(possibilities forall (_._2 == possibilities(0)._2))
|
||||
|
|
@ -122,8 +117,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
* loading is performed by the system’s [[akka.actor.DynamicAccess]].
|
||||
*/
|
||||
def serializerOf(serializerFQN: String): Try[Serializer] =
|
||||
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) recoverWith {
|
||||
case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq())
|
||||
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] -> system)) recoverWith {
|
||||
case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -137,21 +132,21 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
* bindings is a Seq of tuple representing the mapping from Class to Serializer.
|
||||
* It is primarily ordered by the most specific classes first, and secondly in the configured order.
|
||||
*/
|
||||
private[akka] val bindings: Seq[ClassSerializer] =
|
||||
sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v)))
|
||||
private[akka] val bindings: immutable.Seq[ClassSerializer] =
|
||||
sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v))).to[immutable.Seq]
|
||||
|
||||
/**
|
||||
* Sort so that subtypes always precede their supertypes, but without
|
||||
* obeying any order between unrelated subtypes (insert sort).
|
||||
*/
|
||||
private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] =
|
||||
(new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒
|
||||
private def sort(in: Iterable[ClassSerializer]): immutable.Seq[ClassSerializer] =
|
||||
((new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒
|
||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||
case -1 ⇒ buf append ca
|
||||
case x ⇒ buf insert (x, ca)
|
||||
}
|
||||
buf
|
||||
}
|
||||
}).to[immutable.Seq]
|
||||
|
||||
/**
|
||||
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer
|
||||
|
|
|
|||
54
akka-actor/src/main/scala/akka/util/Collections.scala
Normal file
54
akka-actor/src/main/scala/akka/util/Collections.scala
Normal 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
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
@ -75,7 +75,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
|
|||
def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] =
|
||||
container get key match {
|
||||
case null ⇒ None
|
||||
case set ⇒ set.iterator.asScala.find(f)
|
||||
case set ⇒ set.iterator.asScala find f
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -3,6 +3,8 @@
|
|||
*/
|
||||
package akka.util
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
|
||||
*/
|
||||
|
|
@ -74,7 +76,7 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V]
|
|||
|
||||
import SubclassifiedIndex._
|
||||
|
||||
type Changes = Seq[(K, Set[V])]
|
||||
type Changes = immutable.Seq[(K, Set[V])]
|
||||
|
||||
protected var subkeys = Vector.empty[Nonroot[K, V]]
|
||||
|
||||
|
|
@ -208,5 +210,5 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V]
|
|||
private def mergeChangesByKey(changes: Changes): Changes =
|
||||
(emptyMergeMap[K, V] /: changes) {
|
||||
case (m, (k, s)) ⇒ m.updated(k, m(k) ++ s)
|
||||
}.toSeq
|
||||
}.to[immutable.Seq]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,27 +5,9 @@
|
|||
|
||||
package akka.util;
|
||||
|
||||
import java.lang.reflect.Field;
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
public final class Unsafe {
|
||||
public final static sun.misc.Unsafe instance;
|
||||
static {
|
||||
try {
|
||||
sun.misc.Unsafe found = null;
|
||||
for(Field field : sun.misc.Unsafe.class.getDeclaredFields()) {
|
||||
if (field.getType() == sun.misc.Unsafe.class) {
|
||||
field.setAccessible(true);
|
||||
found = (sun.misc.Unsafe) field.get(null);
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (found == null) throw new IllegalStateException("Can't find instance of sun.misc.Unsafe");
|
||||
else instance = found;
|
||||
} catch(Throwable t) {
|
||||
throw new ExceptionInInitializerError(t);
|
||||
}
|
||||
}
|
||||
public final static sun.misc.Unsafe instance = scala.concurrent.util.Unsafe.instance;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,16 +4,15 @@
|
|||
|
||||
package akka.camel
|
||||
|
||||
import internal._
|
||||
import akka.camel.internal._
|
||||
import akka.actor._
|
||||
import akka.ConfigurationException
|
||||
import org.apache.camel.ProducerTemplate
|
||||
import org.apache.camel.impl.DefaultCamelContext
|
||||
import org.apache.camel.model.RouteDefinition
|
||||
import com.typesafe.config.Config
|
||||
import akka.ConfigurationException
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||
import java.util.concurrent.TimeUnit._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
/**
|
||||
* Camel trait encapsulates the underlying camel machinery.
|
||||
|
|
@ -88,8 +87,8 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess
|
|||
final val StreamingCache: Boolean = config.getBoolean("akka.camel.streamingCache")
|
||||
|
||||
final val Conversions: (String, RouteDefinition) ⇒ RouteDefinition = {
|
||||
import scala.collection.JavaConverters.asScalaSetConverter
|
||||
val specifiedConversions = {
|
||||
import scala.collection.JavaConverters.asScalaSetConverter
|
||||
val section = config.getConfig("akka.camel.conversions")
|
||||
section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey)))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,13 +1,18 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.camel
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import org.scalatest.WordSpec
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import scala.concurrent.{ Promise, Await, Future }
|
||||
import scala.collection.immutable
|
||||
import akka.camel.TestSupport.NonSharedCamelSystem
|
||||
import akka.actor.{ ActorRef, Props, Actor }
|
||||
import akka.routing.BroadcastRouter
|
||||
import concurrent.{ Promise, Await, Future }
|
||||
import scala.concurrent.duration._
|
||||
import language.postfixOps
|
||||
import akka.testkit._
|
||||
import akka.util.Timeout
|
||||
import org.apache.camel.model.RouteDefinition
|
||||
|
|
@ -58,7 +63,7 @@ class ConcurrentActivationTest extends WordSpec with MustMatchers with NonShared
|
|||
activations.size must be(2 * number * number)
|
||||
// must be the size of the activated activated producers and consumers
|
||||
deactivations.size must be(2 * number * number)
|
||||
def partitionNames(refs: Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer"))
|
||||
def partitionNames(refs: immutable.Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer"))
|
||||
def assertContainsSameElements(lists: (Seq[_], Seq[_])) {
|
||||
val (a, b) = lists
|
||||
a.intersect(b).size must be(a.size)
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.cluster
|
|||
|
||||
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
|
||||
import akka.event.Logging
|
||||
import scala.collection.immutable.Map
|
||||
import scala.collection.immutable
|
||||
import scala.annotation.tailrec
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.TimeUnit.NANOSECONDS
|
||||
|
|
@ -233,7 +233,7 @@ private[cluster] object HeartbeatHistory {
|
|||
*/
|
||||
def apply(maxSampleSize: Int): HeartbeatHistory = HeartbeatHistory(
|
||||
maxSampleSize = maxSampleSize,
|
||||
intervals = IndexedSeq.empty,
|
||||
intervals = immutable.IndexedSeq.empty,
|
||||
intervalSum = 0L,
|
||||
squaredIntervalSum = 0L)
|
||||
|
||||
|
|
@ -248,7 +248,7 @@ private[cluster] object HeartbeatHistory {
|
|||
*/
|
||||
private[cluster] case class HeartbeatHistory private (
|
||||
maxSampleSize: Int,
|
||||
intervals: IndexedSeq[Long],
|
||||
intervals: immutable.IndexedSeq[Long],
|
||||
intervalSum: Long,
|
||||
squaredIntervalSum: Long) {
|
||||
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import akka.util._
|
|||
import scala.concurrent.duration._
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import java.io.Closeable
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
|
@ -75,7 +75,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
|
|||
val failureDetector: FailureDetector = {
|
||||
import settings.{ FailureDetectorImplementationClass ⇒ fqcn }
|
||||
system.dynamicAccess.createInstanceFor[FailureDetector](
|
||||
fqcn, Seq(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({
|
||||
fqcn, List(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({
|
||||
case e ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString)
|
||||
}).get
|
||||
}
|
||||
|
|
@ -241,7 +241,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
|
|||
* in config. Especially useful from tests when Addresses are unknown
|
||||
* before startup time.
|
||||
*/
|
||||
private[cluster] def joinSeedNodes(seedNodes: IndexedSeq[Address]): Unit =
|
||||
private[cluster] def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit =
|
||||
clusterCore ! InternalClusterAction.JoinSeedNodes(seedNodes)
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -3,7 +3,6 @@
|
|||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
import com.typesafe.config.Config
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.Actor
|
||||
|
|
@ -19,15 +18,16 @@ import akka.actor.Props
|
|||
import akka.actor.Scheduler
|
||||
import akka.actor.Scope
|
||||
import akka.actor.Terminated
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.dispatch.ChildTerminated
|
||||
import akka.event.EventStream
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.remote.RemoteActorRefProvider
|
||||
import akka.remote.RemoteDeployer
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.DefaultResizer
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingRouter
|
||||
import akka.cluster.routing.MixMetricsSelector
|
||||
import akka.cluster.routing.HeapMetricsSelector
|
||||
|
|
@ -119,7 +119,7 @@ private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: Dynami
|
|||
}
|
||||
|
||||
override protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
|
||||
val routees = Vector() ++ deployment.getStringList("routees.paths").asScala
|
||||
val routees = immutableSeq(deployment.getStringList("routees.paths"))
|
||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
|
||||
|
||||
|
|
@ -131,7 +131,7 @@ private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: Dynami
|
|||
case "cpu" ⇒ CpuMetricsSelector
|
||||
case "load" ⇒ SystemLoadAverageMetricsSelector
|
||||
case fqn ⇒
|
||||
val args = Seq(classOf[Config] -> deployment)
|
||||
val args = List(classOf[Config] -> deployment)
|
||||
dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({
|
||||
case exception ⇒ throw new IllegalArgumentException(
|
||||
("Cannot instantiate metrics-selector [%s], defined in [%s], " +
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import akka.actor.{ Actor, ActorLogging, ActorRef, Address, Cancellable, Props, ReceiveTimeout, RootActorPath, Scheduler }
|
||||
|
|
@ -61,7 +61,7 @@ private[cluster] object InternalClusterAction {
|
|||
* Command to initiate the process to join the specified
|
||||
* seed nodes.
|
||||
*/
|
||||
case class JoinSeedNodes(seedNodes: IndexedSeq[Address])
|
||||
case class JoinSeedNodes(seedNodes: immutable.IndexedSeq[Address])
|
||||
|
||||
/**
|
||||
* Start message of the process to join one of the seed nodes.
|
||||
|
|
@ -256,7 +256,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
|
|||
|
||||
def initJoin(): Unit = sender ! InitJoinAck(selfAddress)
|
||||
|
||||
def joinSeedNodes(seedNodes: IndexedSeq[Address]): Unit = {
|
||||
def joinSeedNodes(seedNodes: immutable.IndexedSeq[Address]): Unit = {
|
||||
// only the node which is named first in the list of seed nodes will join itself
|
||||
if (seedNodes.isEmpty || seedNodes.head == selfAddress)
|
||||
self ! JoinTo(selfAddress)
|
||||
|
|
@ -770,7 +770,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
|
|||
*
|
||||
* @return the used [[akka.actor.Address] if any
|
||||
*/
|
||||
private def gossipToRandomNodeOf(addresses: IndexedSeq[Address]): Option[Address] = {
|
||||
private def gossipToRandomNodeOf(addresses: immutable.IndexedSeq[Address]): Option[Address] = {
|
||||
log.debug("Cluster Node [{}] - Selecting random node to gossip to [{}]", selfAddress, addresses.mkString(", "))
|
||||
// filter out myself
|
||||
val peer = selectRandomNode(addresses filterNot (_ == selfAddress))
|
||||
|
|
@ -823,7 +823,7 @@ private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Acto
|
|||
* 5. seed3 retries the join procedure and gets acks from seed2 first, and then joins to seed2
|
||||
*
|
||||
*/
|
||||
private[cluster] final class JoinSeedNodeProcess(seedNodes: IndexedSeq[Address]) extends Actor with ActorLogging {
|
||||
private[cluster] final class JoinSeedNodeProcess(seedNodes: immutable.IndexedSeq[Address]) extends Actor with ActorLogging {
|
||||
import InternalClusterAction._
|
||||
|
||||
def selfAddress = Cluster(context.system).selfAddress
|
||||
|
|
|
|||
|
|
@ -4,14 +4,15 @@
|
|||
package akka.cluster
|
||||
|
||||
import language.postfixOps
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import akka.actor.{ Actor, ActorLogging, ActorRef, Address }
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.event.EventStream
|
||||
import akka.actor.AddressTerminated
|
||||
import java.lang.Iterable
|
||||
import scala.collection.JavaConverters
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
|
||||
/**
|
||||
* Domain events published to the event bus.
|
||||
|
|
@ -30,7 +31,7 @@ object ClusterEvent {
|
|||
* Current snapshot state of the cluster. Sent to new subscriber.
|
||||
*/
|
||||
case class CurrentClusterState(
|
||||
members: SortedSet[Member] = SortedSet.empty,
|
||||
members: immutable.SortedSet[Member] = immutable.SortedSet.empty,
|
||||
unreachable: Set[Member] = Set.empty,
|
||||
convergence: Boolean = false,
|
||||
seenBy: Set[Address] = Set.empty,
|
||||
|
|
@ -49,19 +50,15 @@ object ClusterEvent {
|
|||
* Java API
|
||||
* Read only
|
||||
*/
|
||||
def getUnreachable: java.util.Set[Member] = {
|
||||
import scala.collection.JavaConverters._
|
||||
unreachable.asJava
|
||||
}
|
||||
def getUnreachable: java.util.Set[Member] =
|
||||
scala.collection.JavaConverters.setAsJavaSetConverter(unreachable).asJava
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* Read only
|
||||
*/
|
||||
def getSeenBy: java.util.Set[Address] = {
|
||||
import scala.collection.JavaConverters._
|
||||
seenBy.asJava
|
||||
}
|
||||
def getSeenBy: java.util.Set[Address] =
|
||||
scala.collection.JavaConverters.setAsJavaSetConverter(seenBy).asJava
|
||||
|
||||
/**
|
||||
* Java API
|
||||
|
|
@ -148,10 +145,8 @@ object ClusterEvent {
|
|||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getNodeMetrics: java.lang.Iterable[NodeMetrics] = {
|
||||
import scala.collection.JavaConverters._
|
||||
nodeMetrics.asJava
|
||||
}
|
||||
def getNodeMetrics: java.lang.Iterable[NodeMetrics] =
|
||||
scala.collection.JavaConverters.asJavaIterableConverter(nodeMetrics).asJava
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -168,7 +163,7 @@ object ClusterEvent {
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[cluster] def diff(oldGossip: Gossip, newGossip: Gossip): IndexedSeq[ClusterDomainEvent] = {
|
||||
private[cluster] def diff(oldGossip: Gossip, newGossip: Gossip): immutable.IndexedSeq[ClusterDomainEvent] = {
|
||||
val newMembers = newGossip.members -- oldGossip.members
|
||||
|
||||
val membersGroupedByAddress = (newGossip.members.toList ++ oldGossip.members.toList).groupBy(_.address)
|
||||
|
|
@ -203,18 +198,18 @@ object ClusterEvent {
|
|||
|
||||
val newConvergence = newGossip.convergence
|
||||
val convergenceChanged = newConvergence != oldGossip.convergence
|
||||
val convergenceEvents = if (convergenceChanged) Seq(ConvergenceChanged(newConvergence)) else Seq.empty
|
||||
val convergenceEvents = if (convergenceChanged) List(ConvergenceChanged(newConvergence)) else EmptyImmutableSeq
|
||||
|
||||
val leaderEvents =
|
||||
if (newGossip.leader != oldGossip.leader) Seq(LeaderChanged(newGossip.leader))
|
||||
else Seq.empty
|
||||
if (newGossip.leader != oldGossip.leader) List(LeaderChanged(newGossip.leader))
|
||||
else EmptyImmutableSeq
|
||||
|
||||
val newSeenBy = newGossip.seenBy
|
||||
val seenEvents =
|
||||
if (convergenceChanged || newSeenBy != oldGossip.seenBy) Seq(SeenChanged(newConvergence, newSeenBy))
|
||||
else Seq.empty
|
||||
if (convergenceChanged || newSeenBy != oldGossip.seenBy) List(SeenChanged(newConvergence, newSeenBy))
|
||||
else EmptyImmutableSeq
|
||||
|
||||
memberEvents.toIndexedSeq ++ unreachableEvents ++ downedEvents ++ unreachableDownedEvents ++ removedEvents ++
|
||||
memberEvents.toVector ++ unreachableEvents ++ downedEvents ++ unreachableDownedEvents ++ removedEvents ++
|
||||
leaderEvents ++ convergenceEvents ++ seenEvents
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@ package akka.cluster
|
|||
|
||||
import language.postfixOps
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.duration._
|
||||
import java.net.URLEncoder
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import java.lang.System.{ currentTimeMillis ⇒ newTimestamp }
|
|||
import java.lang.management.{ OperatingSystemMXBean, MemoryMXBean, ManagementFactory }
|
||||
import java.lang.reflect.InvocationTargetException
|
||||
import java.lang.reflect.Method
|
||||
import scala.collection.immutable.{ SortedSet, Map }
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import scala.runtime.{ RichLong, RichDouble, RichInt }
|
||||
|
|
@ -52,7 +52,7 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto
|
|||
/**
|
||||
* The node ring gossipped that contains only members that are Up.
|
||||
*/
|
||||
var nodes: SortedSet[Address] = SortedSet.empty
|
||||
var nodes: immutable.SortedSet[Address] = immutable.SortedSet.empty
|
||||
|
||||
/**
|
||||
* The latest metric values with their statistical data.
|
||||
|
|
@ -144,12 +144,12 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto
|
|||
/**
|
||||
* Gossip to peer nodes.
|
||||
*/
|
||||
def gossip(): Unit = selectRandomNode((nodes - selfAddress).toIndexedSeq) foreach gossipTo
|
||||
def gossip(): Unit = selectRandomNode((nodes - selfAddress).toVector) foreach gossipTo
|
||||
|
||||
def gossipTo(address: Address): Unit =
|
||||
context.actorFor(self.path.toStringWithAddress(address)) ! MetricsGossipEnvelope(selfAddress, latestGossip)
|
||||
|
||||
def selectRandomNode(addresses: IndexedSeq[Address]): Option[Address] =
|
||||
def selectRandomNode(addresses: immutable.IndexedSeq[Address]): Option[Address] =
|
||||
if (addresses.isEmpty) None else Some(addresses(ThreadLocalRandom.current nextInt addresses.size))
|
||||
|
||||
/**
|
||||
|
|
@ -163,7 +163,7 @@ private[cluster] class ClusterMetricsCollector(publisher: ActorRef) extends Acto
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[cluster] object MetricsGossip {
|
||||
val empty = MetricsGossip()
|
||||
val empty = MetricsGossip(Set.empty[NodeMetrics])
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -171,7 +171,7 @@ private[cluster] object MetricsGossip {
|
|||
*
|
||||
* @param nodes metrics per node
|
||||
*/
|
||||
private[cluster] case class MetricsGossip(nodes: Set[NodeMetrics] = Set.empty) {
|
||||
private[cluster] case class MetricsGossip(nodes: Set[NodeMetrics]) {
|
||||
|
||||
/**
|
||||
* Removes nodes if their correlating node ring members are not [[akka.cluster.MemberStatus.Up]]
|
||||
|
|
@ -385,10 +385,8 @@ case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Metric] =
|
|||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getMetrics: java.lang.Iterable[Metric] = {
|
||||
import scala.collection.JavaConverters._
|
||||
metrics.asJava
|
||||
}
|
||||
def getMetrics: java.lang.Iterable[Metric] =
|
||||
scala.collection.JavaConverters.asJavaIterableConverter(metrics).asJava
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -673,12 +671,13 @@ class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: AnyRef
|
|||
private def this(cluster: Cluster) =
|
||||
this(cluster.selfAddress,
|
||||
EWMA.alpha(cluster.settings.MetricsDecayHalfLifeDuration, cluster.settings.MetricsInterval),
|
||||
cluster.system.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty).get)
|
||||
cluster.system.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Nil).get)
|
||||
|
||||
def this(system: ActorSystem) = this(Cluster(system))
|
||||
|
||||
private val decayFactorOption = Some(decayFactor)
|
||||
|
||||
private val EmptyClassArray: Array[(Class[_])] = Array.empty[(Class[_])]
|
||||
private val LoadAverage: Option[Method] = createMethodFrom(sigar, "getLoadAverage")
|
||||
private val Cpu: Option[Method] = createMethodFrom(sigar, "getCpuPerc")
|
||||
private val CombinedCpu: Option[Method] = Try(Cpu.get.getReturnType.getMethod("getCombined")).toOption
|
||||
|
|
@ -730,7 +729,7 @@ class SigarMetricsCollector(address: Address, decayFactor: Double, sigar: AnyRef
|
|||
*/
|
||||
override def close(): Unit = Try(createMethodFrom(sigar, "close").get.invoke(sigar))
|
||||
|
||||
private def createMethodFrom(ref: AnyRef, method: String, types: Array[(Class[_])] = Array.empty[(Class[_])]): Option[Method] =
|
||||
private def createMethodFrom(ref: AnyRef, method: String, types: Array[(Class[_])] = EmptyClassArray): Option[Method] =
|
||||
Try(ref.getClass.getMethod(method, types: _*)).toOption
|
||||
|
||||
}
|
||||
|
|
@ -758,7 +757,7 @@ private[cluster] object MetricsCollector {
|
|||
|
||||
} else {
|
||||
system.dynamicAccess.createInstanceFor[MetricsCollector](
|
||||
fqcn, Seq(classOf[ActorSystem] -> system)).recover({
|
||||
fqcn, List(classOf[ActorSystem] -> system)).recover({
|
||||
case e ⇒ throw new ConfigurationException("Could not create custom metrics collector [" + fqcn + "] due to:" + e.toString)
|
||||
}).get
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.cluster
|
||||
|
||||
import java.io.Closeable
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import akka.actor.{ Actor, ActorRef, ActorSystemImpl, Address, Props }
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.actor.PoisonPill
|
||||
|
|
@ -81,7 +81,7 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
|
|||
/**
|
||||
* Current cluster members, sorted by address.
|
||||
*/
|
||||
def members: SortedSet[Member] = state.members
|
||||
def members: immutable.SortedSet[Member] = state.members
|
||||
|
||||
/**
|
||||
* Members that has been detected as unreachable.
|
||||
|
|
|
|||
|
|
@ -3,15 +3,16 @@
|
|||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable
|
||||
import com.typesafe.config.Config
|
||||
import scala.concurrent.duration.Duration
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import akka.ConfigurationException
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.actor.Address
|
||||
import akka.actor.AddressFromURIString
|
||||
import akka.dispatch.Dispatchers
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.japi.Util.immutableSeq
|
||||
|
||||
class ClusterSettings(val config: Config, val systemName: String) {
|
||||
import config._
|
||||
|
|
@ -45,9 +46,8 @@ class ClusterSettings(val config: Config, val systemName: String) {
|
|||
require(n > 0, "failure-detector.monitored-by-nr-of-members must be > 0"); n
|
||||
}
|
||||
|
||||
final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map {
|
||||
case AddressFromURIString(addr) ⇒ addr
|
||||
}.toIndexedSeq
|
||||
final val SeedNodes: immutable.IndexedSeq[Address] =
|
||||
immutableSeq(getStringList("akka.cluster.seed-nodes")).map { case AddressFromURIString(addr) ⇒ addr }.toVector
|
||||
final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS)
|
||||
final val PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
|
||||
final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
|
||||
|
|
|
|||
|
|
@ -5,14 +5,14 @@
|
|||
package akka.cluster
|
||||
|
||||
import akka.actor.Address
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import MemberStatus._
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
private[cluster] object Gossip {
|
||||
val emptyMembers: SortedSet[Member] = SortedSet.empty
|
||||
val emptyMembers: immutable.SortedSet[Member] = immutable.SortedSet.empty
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -50,7 +50,7 @@ private[cluster] object Gossip {
|
|||
*/
|
||||
private[cluster] case class Gossip(
|
||||
overview: GossipOverview = GossipOverview(),
|
||||
members: SortedSet[Member] = Gossip.emptyMembers, // sorted set of members with their status, sorted by address
|
||||
members: immutable.SortedSet[Member] = Gossip.emptyMembers, // sorted set of members with their status, sorted by address
|
||||
version: VectorClock = VectorClock()) // vector clock version
|
||||
extends ClusterMessage // is a serializable cluster message
|
||||
with Versioned[Gossip] {
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.cluster
|
|||
|
||||
import language.implicitConversions
|
||||
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import scala.collection.GenTraversableOnce
|
||||
import akka.actor.Address
|
||||
import MemberStatus._
|
||||
|
|
|
|||
|
|
@ -5,8 +5,9 @@
|
|||
package akka.cluster.routing
|
||||
|
||||
import java.util.Arrays
|
||||
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
import scala.collection.immutable
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
|
|
@ -21,6 +22,7 @@ import akka.cluster.NodeMetrics
|
|||
import akka.cluster.StandardMetrics.Cpu
|
||||
import akka.cluster.StandardMetrics.HeapMemory
|
||||
import akka.event.Logging
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.routing.Broadcast
|
||||
import akka.routing.Destination
|
||||
import akka.routing.FromConfig
|
||||
|
|
@ -70,7 +72,7 @@ object AdaptiveLoadBalancingRouter {
|
|||
@SerialVersionUID(1L)
|
||||
case class AdaptiveLoadBalancingRouter(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector(),
|
||||
nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = AdaptiveLoadBalancingRouter.defaultSupervisorStrategy)
|
||||
|
|
@ -92,7 +94,7 @@ case class AdaptiveLoadBalancingRouter(
|
|||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) =
|
||||
this(metricsSelector = selector, routees = routeePaths.asScala)
|
||||
this(metricsSelector = selector, routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -148,7 +150,7 @@ trait AdaptiveLoadBalancingRouterLike { this: RouterConfig ⇒
|
|||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def routerDispatcher: String
|
||||
|
||||
|
|
@ -280,17 +282,17 @@ case object SystemLoadAverageMetricsSelector extends CapacityMetricsSelector {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class MixMetricsSelector(
|
||||
selectors: IndexedSeq[CapacityMetricsSelector] = Vector(
|
||||
selectors: immutable.IndexedSeq[CapacityMetricsSelector] = Vector(
|
||||
HeapMetricsSelector, CpuMetricsSelector, SystemLoadAverageMetricsSelector))
|
||||
extends CapacityMetricsSelector {
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(selectors: java.lang.Iterable[CapacityMetricsSelector]) = this(selectors.asScala.toIndexedSeq)
|
||||
def this(selectors: java.lang.Iterable[CapacityMetricsSelector]) = this(immutableSeq(selectors).toVector)
|
||||
|
||||
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
|
||||
val combined: IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
|
||||
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
|
||||
// aggregated average of the capacities by address
|
||||
combined.foldLeft(Map.empty[Address, (Double, Int)].withDefaultValue((0.0, 0))) {
|
||||
case (acc, (address, capacity)) ⇒
|
||||
|
|
@ -365,7 +367,7 @@ abstract class CapacityMetricsSelector extends MetricsSelector {
|
|||
*
|
||||
* Pick routee based on its weight. Higher weight, higher probability.
|
||||
*/
|
||||
private[cluster] class WeightedRoutees(refs: IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]) {
|
||||
private[cluster] class WeightedRoutees(refs: immutable.IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]) {
|
||||
|
||||
// fill an array of same size as the refs with accumulated weights,
|
||||
// binarySearch is used to pick the right bucket from a requested value
|
||||
|
|
|
|||
|
|
@ -5,16 +5,13 @@ package akka.cluster.routing
|
|||
|
||||
import java.lang.IllegalStateException
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.collection.immutable
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSystemImpl
|
||||
import akka.actor.Address
|
||||
import akka.actor.Deploy
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.cluster.Cluster
|
||||
|
|
@ -51,7 +48,7 @@ final case class ClusterRouterConfig(local: RouterConfig, settings: ClusterRoute
|
|||
|
||||
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
||||
({
|
||||
case (sender, message: ClusterDomainEvent) ⇒ Seq(Destination(sender, routeeProvider.context.self))
|
||||
case (sender, message: ClusterDomainEvent) ⇒ List(Destination(sender, routeeProvider.context.self))
|
||||
}: Route) orElse localRoute
|
||||
}
|
||||
|
||||
|
|
@ -130,7 +127,7 @@ case class ClusterRouterSettings private[akka] (
|
|||
if (isRouteesPathDefined && maxInstancesPerNode != 1)
|
||||
throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be 1 when routeesPath is defined")
|
||||
|
||||
val routeesPathElements: Iterable[String] = routeesPath match {
|
||||
val routeesPathElements: immutable.Iterable[String] = routeesPath match {
|
||||
case RelativeActorPath(elements) ⇒ elements
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("routeesPath [%s] is not a valid relative actor path" format routeesPath)
|
||||
|
|
@ -156,7 +153,7 @@ private[akka] class ClusterRouteeProvider(
|
|||
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||
private val childNameCounter = new AtomicInteger
|
||||
|
||||
override def registerRouteesFor(paths: Iterable[String]): Unit =
|
||||
override def registerRouteesFor(paths: immutable.Iterable[String]): Unit =
|
||||
throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
|
||||
format context.self.path.toString)
|
||||
|
||||
|
|
@ -183,7 +180,7 @@ private[akka] class ClusterRouteeProvider(
|
|||
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
||||
}
|
||||
// must register each one, since registered routees are used in selectDeploymentTarget
|
||||
registerRoutees(Some(ref))
|
||||
registerRoutees(List(ref))
|
||||
|
||||
// recursion until all created
|
||||
doCreateRoutees()
|
||||
|
|
@ -222,27 +219,26 @@ private[akka] class ClusterRouteeProvider(
|
|||
case a ⇒ a
|
||||
}
|
||||
|
||||
private[routing] def availableNodes: SortedSet[Address] = {
|
||||
private[routing] def availableNodes: immutable.SortedSet[Address] = {
|
||||
import Member.addressOrdering
|
||||
val currentNodes = nodes
|
||||
if (currentNodes.isEmpty && settings.allowLocalRoutees)
|
||||
//use my own node, cluster information not updated yet
|
||||
SortedSet(cluster.selfAddress)
|
||||
immutable.SortedSet(cluster.selfAddress)
|
||||
else
|
||||
currentNodes
|
||||
}
|
||||
|
||||
@volatile
|
||||
private[routing] var nodes: SortedSet[Address] = {
|
||||
private[routing] var nodes: immutable.SortedSet[Address] = {
|
||||
import Member.addressOrdering
|
||||
cluster.readView.members.collect {
|
||||
case m if isAvailable(m) ⇒ m.address
|
||||
}
|
||||
}
|
||||
|
||||
private[routing] def isAvailable(m: Member): Boolean = {
|
||||
private[routing] def isAvailable(m: Member): Boolean =
|
||||
m.status == MemberStatus.Up && (settings.allowLocalRoutees || m.address != cluster.selfAddress)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.actor.Address
|
||||
import scala.collection.immutable
|
||||
|
||||
case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -51,7 +52,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
|
|||
cluster.down(thirdAddress)
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -62,7 +63,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
|
|||
runOn(second, fourth) {
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||
}
|
||||
|
||||
enterBarrier("await-completion")
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.actor.Address
|
||||
import scala.collection.immutable
|
||||
|
||||
case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -49,7 +50,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
|
|||
|
||||
markNodeAsUnavailable(thirdAddress)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -60,7 +61,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
|
|||
runOn(second, fourth) {
|
||||
enterBarrier("down-third-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||
}
|
||||
|
||||
enterBarrier("await-completion")
|
||||
|
|
|
|||
|
|
@ -3,6 +3,7 @@
|
|||
*/
|
||||
package akka.cluster
|
||||
|
||||
import scala.collection.immutable
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
|
|
@ -35,7 +36,7 @@ abstract class JoinSeedNodeSpec
|
|||
|
||||
import JoinSeedNodeMultiJvmSpec._
|
||||
|
||||
def seedNodes: IndexedSeq[Address] = IndexedSeq(seed1, seed2, seed3)
|
||||
def seedNodes: immutable.IndexedSeq[Address] = Vector(seed1, seed2, seed3)
|
||||
|
||||
"A cluster with seed nodes" must {
|
||||
"be able to start the seed nodes concurrently" taggedAs LongRunningTest in {
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -59,7 +60,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
|||
|
||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds)
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
|
|
@ -69,7 +70,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
|||
runOn(second, third) {
|
||||
enterBarrier("down-fourth-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds)
|
||||
}
|
||||
|
||||
enterBarrier("await-completion-1")
|
||||
|
|
@ -89,7 +90,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
|||
|
||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30.seconds)
|
||||
}
|
||||
|
||||
runOn(second) {
|
||||
|
|
@ -99,7 +100,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
|||
runOn(third) {
|
||||
enterBarrier("down-second-node")
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
|
||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30 seconds)
|
||||
}
|
||||
|
||||
enterBarrier("await-completion-2")
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val controller = role("controller")
|
||||
|
|
@ -42,7 +43,7 @@ abstract class LeaderElectionSpec(multiNodeConfig: LeaderElectionMultiNodeConfig
|
|||
import multiNodeConfig._
|
||||
|
||||
// sorted in the order used by the cluster
|
||||
lazy val sortedRoles = Seq(first, second, third, fourth).sorted
|
||||
lazy val sortedRoles = List(first, second, third, fourth).sorted
|
||||
|
||||
"A cluster of four nodes" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -4,21 +4,21 @@
|
|||
package akka.cluster
|
||||
|
||||
import language.implicitConversions
|
||||
|
||||
import org.scalatest.Suite
|
||||
import org.scalatest.exceptions.TestFailedException
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.{ Address, ExtendedActorSystem }
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec }
|
||||
import akka.testkit._
|
||||
import akka.testkit.TestEvent._
|
||||
import scala.concurrent.duration._
|
||||
import org.scalatest.Suite
|
||||
import org.scalatest.exceptions.TestFailedException
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.RootActorPath
|
||||
import akka.actor.{ ActorSystem, Address }
|
||||
import akka.event.Logging.ErrorLevel
|
||||
import akka.actor.ActorSystem
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
object MultiNodeClusterSpec {
|
||||
|
||||
|
|
@ -158,7 +158,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
|||
* nodes (roles). First node will be started first
|
||||
* and others will join the first.
|
||||
*/
|
||||
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq)
|
||||
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.to[immutable.Seq])
|
||||
|
||||
/**
|
||||
* Initialize the cluster of the specified member
|
||||
|
|
@ -166,11 +166,9 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
|||
* First node will be started first and others will join
|
||||
* the first.
|
||||
*/
|
||||
def awaitClusterUp(roles: RoleName*): Unit = {
|
||||
awaitStartCluster(true, roles.toSeq)
|
||||
}
|
||||
def awaitClusterUp(roles: RoleName*): Unit = awaitStartCluster(true, roles.to[immutable.Seq])
|
||||
|
||||
private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = {
|
||||
private def awaitStartCluster(upConvergence: Boolean = true, roles: immutable.Seq[RoleName]): Unit = {
|
||||
runOn(roles.head) {
|
||||
// make sure that the node-to-join is started before other join
|
||||
startClusterNode()
|
||||
|
|
@ -196,16 +194,15 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
|||
expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) ⇒ members(i).address must be(a) }
|
||||
}
|
||||
|
||||
def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) {
|
||||
assertLeaderIn(nodesInCluster)
|
||||
}
|
||||
def assertLeader(nodesInCluster: RoleName*): Unit =
|
||||
if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq])
|
||||
|
||||
/**
|
||||
* Assert that the cluster has elected the correct leader
|
||||
* out of all nodes in the cluster. First
|
||||
* member in the cluster ring is expected leader.
|
||||
*/
|
||||
def assertLeaderIn(nodesInCluster: Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) {
|
||||
def assertLeaderIn(nodesInCluster: immutable.Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) {
|
||||
nodesInCluster.length must not be (0)
|
||||
val expectedLeader = roleOfLeader(nodesInCluster)
|
||||
val leader = clusterView.leader
|
||||
|
|
@ -221,7 +218,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
|||
*/
|
||||
def awaitUpConvergence(
|
||||
numberOfMembers: Int,
|
||||
canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address],
|
||||
canNotBePartOfMemberRing: immutable.Seq[Address] = Nil,
|
||||
timeout: FiniteDuration = 20.seconds): Unit = {
|
||||
within(timeout) {
|
||||
awaitCond(clusterView.members.size == numberOfMembers)
|
||||
|
|
@ -239,7 +236,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
|||
def awaitSeenSameState(addresses: Address*): Unit =
|
||||
awaitCond((addresses.toSet -- clusterView.seenBy).isEmpty)
|
||||
|
||||
def roleOfLeader(nodesInCluster: Seq[RoleName] = roles): RoleName = {
|
||||
def roleOfLeader(nodesInCluster: immutable.Seq[RoleName] = roles): RoleName = {
|
||||
nodesInCluster.length must not be (0)
|
||||
nodesInCluster.sorted.head
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -65,7 +66,7 @@ abstract class SingletonClusterSpec(multiNodeConfig: SingletonClusterMultiNodeCo
|
|||
|
||||
markNodeAsUnavailable(secondAddress)
|
||||
|
||||
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
||||
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = List(secondAddress), 30.seconds)
|
||||
clusterView.isSingletonCluster must be(true)
|
||||
awaitCond(clusterView.isLeader)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,9 +9,10 @@ import com.typesafe.config.ConfigFactory
|
|||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Address
|
||||
import akka.remote.testconductor.Direction
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -53,8 +54,8 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig)
|
|||
|
||||
muteMarkingAsUnreachable()
|
||||
|
||||
val side1 = IndexedSeq(first, second)
|
||||
val side2 = IndexedSeq(third, fourth, fifth)
|
||||
val side1 = Vector(first, second)
|
||||
val side2 = Vector(third, fourth, fifth)
|
||||
|
||||
"A cluster of 5 members" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -6,13 +6,14 @@ package akka.cluster
|
|||
import language.postfixOps
|
||||
|
||||
import org.scalatest.BeforeAndAfter
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.Address
|
||||
import akka.remote.testconductor.{ RoleName, Direction }
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -45,7 +46,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
|
|||
|
||||
muteMarkingAsUnreachable()
|
||||
|
||||
def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = {
|
||||
def allBut(role: RoleName, roles: immutable.Seq[RoleName] = roles): immutable.Seq[RoleName] = {
|
||||
roles.filterNot(_ == role)
|
||||
}
|
||||
|
||||
|
|
@ -125,7 +126,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
|
|||
}
|
||||
|
||||
runOn(allBut(victim): _*) {
|
||||
awaitUpConvergence(roles.size - 1, Seq(victim))
|
||||
awaitUpConvergence(roles.size - 1, List(victim))
|
||||
}
|
||||
|
||||
endBarrier
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster
|
|||
import akka.actor.Address
|
||||
import akka.testkit._
|
||||
import akka.testkit.TestEvent._
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
|
|
@ -27,7 +27,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
val conn = Address("akka", "", "localhost", 2552)
|
||||
val conn2 = Address("akka", "", "localhost", 2553)
|
||||
|
||||
def fakeTimeGenerator(timeIntervals: Seq[Long]): () ⇒ Long = {
|
||||
def fakeTimeGenerator(timeIntervals: immutable.Seq[Long]): () ⇒ Long = {
|
||||
var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c))
|
||||
def timeGenerator(): Long = {
|
||||
val currentTime = times.head
|
||||
|
|
@ -73,7 +73,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
|||
|
||||
"return realistic phi values" in {
|
||||
val fd = createFailureDetector()
|
||||
val test = TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3)
|
||||
val test = immutable.TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3)
|
||||
for ((timeDiff, expectedPhi) ← test) {
|
||||
fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,6 +6,8 @@
|
|||
package akka.cluster
|
||||
|
||||
import scala.language.postfixOps
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import scala.util.{ Success, Try, Failure }
|
||||
|
|
@ -122,7 +124,7 @@ trait MetricsCollectorFactory { this: AkkaSpec ⇒
|
|||
|
||||
def createMetricsCollector: MetricsCollector =
|
||||
Try(new SigarMetricsCollector(selfAddress, defaultDecayFactor,
|
||||
extendedActorSystem.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty).get)) match {
|
||||
extendedActorSystem.dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Nil).get)) match {
|
||||
case Success(sigarCollector) ⇒ sigarCollector
|
||||
case Failure(e) ⇒
|
||||
log.debug("Metrics will be retreived from MBeans, Sigar failed to load. Reason: " + e)
|
||||
|
|
|
|||
|
|
@ -29,7 +29,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
"allocate weighted refs" in {
|
||||
val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10)
|
||||
val refs = IndexedSeq(refA, refB, refC)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
|
||||
weighted(1) must be(refA)
|
||||
|
|
@ -41,7 +41,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
"check boundaries" in {
|
||||
val empty = new WeightedRoutees(Vector(), a1, Map.empty)
|
||||
empty.total must be(0)
|
||||
val weighted = new WeightedRoutees(IndexedSeq(refA, refB, refC), a1, Map.empty)
|
||||
val weighted = new WeightedRoutees(Vector(refA, refB, refC), a1, Map.empty)
|
||||
weighted.total must be(3)
|
||||
intercept[IllegalArgumentException] {
|
||||
weighted(0)
|
||||
|
|
@ -53,7 +53,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
"allocate refs for undefined weight" in {
|
||||
val weights = Map(a1 -> 1, b1 -> 7)
|
||||
val refs = IndexedSeq(refA, refB, refC)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
|
||||
weighted(1) must be(refA)
|
||||
|
|
@ -65,7 +65,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
"allocate weighted local refs" in {
|
||||
val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10)
|
||||
val refs = IndexedSeq(testActor, refB, refC)
|
||||
val refs = Vector(testActor, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
|
||||
1 to 2 foreach { weighted(_) must be(testActor) }
|
||||
|
|
@ -74,7 +74,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
"not allocate ref with weight zero" in {
|
||||
val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10)
|
||||
val refs = IndexedSeq(refA, refB, refC)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
|
||||
1 to weighted.total foreach { weighted(_) must not be (refA) }
|
||||
|
|
|
|||
|
|
@ -36,7 +36,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp
|
|||
}
|
||||
|
||||
"A throttler" must {
|
||||
|
||||
def println(a: Any) = ()
|
||||
"must pass the ScalaDoc class documentation example program" in {
|
||||
//#demo-code
|
||||
// A simple actor that prints whatever it receives
|
||||
|
|
|
|||
|
|
@ -72,7 +72,7 @@ Here is an example Build.scala file that uses the MultiJvm plugin:
|
|||
"test" cross CrossVersion.full,
|
||||
"com.typesafe.akka" %% "akka-remote-tests-experimental" % "@version@" %
|
||||
"test" cross CrossVersion.full,
|
||||
"org.scalatest" %% "scalatest" % "1.8" % "test" cross CrossVersion.full,
|
||||
"org.scalatest" %% "scalatest" % "1.8-B2" % "test" cross CrossVersion.full,
|
||||
"junit" % "junit" % "4.5" % "test"
|
||||
)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -108,8 +108,8 @@ Actor Best Practices
|
|||
#. Top-level actors are the innermost part of your Error Kernel, so create them
|
||||
sparingly and prefer truly hierarchical systems. This has benefits wrt.
|
||||
fault-handling (both considering the granularity of configuration and the
|
||||
performance) and it also reduces the number of blocking calls made, since
|
||||
the creation of top-level actors involves synchronous messaging.
|
||||
performance) and it also reduces the strain on the guardian actor, which is
|
||||
a single point of contention if over-used.
|
||||
|
||||
Blocking Needs Careful Management
|
||||
---------------------------------
|
||||
|
|
|
|||
|
|
@ -27,10 +27,11 @@ import akka.testkit.ErrorFilter;
|
|||
import akka.testkit.EventFilter;
|
||||
import akka.testkit.TestEvent;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
import static akka.japi.Util.immutableSeq;
|
||||
import akka.japi.Function;
|
||||
import scala.Option;
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.collection.Seq;
|
||||
import scala.collection.immutable.Seq;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.junit.BeforeClass;
|
||||
|
|
@ -219,8 +220,7 @@ public class FaultHandlingTestBase {
|
|||
|
||||
//#testkit
|
||||
public <A> Seq<A> seq(A... args) {
|
||||
return JavaConverters.collectionAsScalaIterableConverter(
|
||||
java.util.Arrays.asList(args)).asScala().toSeq();
|
||||
return immutableSeq(args);
|
||||
}
|
||||
//#testkit
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import static docs.jrouting.CustomRouterDocTestBase.Message.RepublicanVote;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.junit.After;
|
||||
|
|
@ -69,7 +70,7 @@ public class CustomRouterDocTestBase {
|
|||
//#supervision
|
||||
final SupervisorStrategy strategy =
|
||||
new OneForOneStrategy(5, Duration.create("1 minute"),
|
||||
new Class<?>[] { Exception.class });
|
||||
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
||||
final ActorRef router = system.actorOf(new Props(MyActor.class)
|
||||
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
||||
//#supervision
|
||||
|
|
@ -179,16 +180,14 @@ public class CustomRouterDocTestBase {
|
|||
//#crRoutingLogic
|
||||
return new CustomRoute() {
|
||||
@Override
|
||||
public Iterable<Destination> destinationsFor(ActorRef sender, Object msg) {
|
||||
public scala.collection.immutable.Seq<Destination> destinationsFor(ActorRef sender, Object msg) {
|
||||
switch ((Message) msg) {
|
||||
case DemocratVote:
|
||||
case DemocratCountResult:
|
||||
return Arrays.asList(
|
||||
new Destination[] { new Destination(sender, democratActor) });
|
||||
return akka.japi.Util.immutableSingletonSeq(new Destination(sender, democratActor));
|
||||
case RepublicanVote:
|
||||
case RepublicanCountResult:
|
||||
return Arrays.asList(
|
||||
new Destination[] { new Destination(sender, republicanActor) });
|
||||
return akka.japi.Util.immutableSingletonSeq(new Destination(sender, republicanActor));
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown message: " + msg);
|
||||
}
|
||||
|
|
|
|||
26
akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst
Normal file
26
akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst
Normal 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.
|
||||
|
|
@ -8,3 +8,4 @@ Migration Guides
|
|||
|
||||
migration-guide-1.3.x-2.0.x
|
||||
migration-guide-2.0.x-2.1.x
|
||||
migration-guide-2.1.x-2.2.x
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import language.postfixOps
|
|||
import akka.testkit.{ AkkaSpec ⇒ MyFavoriteTestFrameWorkPlusAkkaTestKit }
|
||||
//#test-code
|
||||
import akka.actor.Props
|
||||
import scala.collection.immutable
|
||||
|
||||
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||
|
||||
|
|
@ -24,7 +25,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
|||
case object Flush
|
||||
|
||||
// sent events
|
||||
case class Batch(obj: Seq[Any])
|
||||
case class Batch(obj: immutable.Seq[Any])
|
||||
//#simple-events
|
||||
//#simple-state
|
||||
// states
|
||||
|
|
@ -34,7 +35,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
|||
|
||||
sealed trait Data
|
||||
case object Uninitialized extends Data
|
||||
case class Todo(target: ActorRef, queue: Seq[Any]) extends Data
|
||||
case class Todo(target: ActorRef, queue: immutable.Seq[Any]) extends Data
|
||||
//#simple-state
|
||||
//#simple-fsm
|
||||
class Buncher extends Actor with FSM[State, Data] {
|
||||
|
|
@ -193,12 +194,12 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
|||
buncher ! SetTarget(testActor)
|
||||
buncher ! Queue(42)
|
||||
buncher ! Queue(43)
|
||||
expectMsg(Batch(Seq(42, 43)))
|
||||
expectMsg(Batch(immutable.Seq(42, 43)))
|
||||
buncher ! Queue(44)
|
||||
buncher ! Flush
|
||||
buncher ! Queue(45)
|
||||
expectMsg(Batch(Seq(44)))
|
||||
expectMsg(Batch(Seq(45)))
|
||||
expectMsg(Batch(immutable.Seq(44)))
|
||||
expectMsg(Batch(immutable.Seq(45)))
|
||||
}
|
||||
|
||||
"batch not if uninitialized" in {
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import akka.testkit.DefaultTimeout
|
|||
import akka.testkit.ImplicitSender
|
||||
import akka.testkit.TestKit
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* a Test to show some TestKit examples
|
||||
|
|
@ -38,8 +39,8 @@ class TestKitUsageSpec
|
|||
val filterRef = system.actorOf(Props(new FilteringActor(testActor)))
|
||||
val randomHead = Random.nextInt(6)
|
||||
val randomTail = Random.nextInt(10)
|
||||
val headList = Seq().padTo(randomHead, "0")
|
||||
val tailList = Seq().padTo(randomTail, "1")
|
||||
val headList = immutable.Seq().padTo(randomHead, "0")
|
||||
val tailList = immutable.Seq().padTo(randomTail, "1")
|
||||
val seqRef =
|
||||
system.actorOf(Props(new SequencingActor(testActor, headList, tailList)))
|
||||
|
||||
|
|
@ -145,7 +146,7 @@ object TestKitUsageSpec {
|
|||
* like to test that the interesting value is received and that you cant
|
||||
* be bothered with the rest
|
||||
*/
|
||||
class SequencingActor(next: ActorRef, head: Seq[String], tail: Seq[String])
|
||||
class SequencingActor(next: ActorRef, head: immutable.Seq[String], tail: immutable.Seq[String])
|
||||
extends Actor {
|
||||
def receive = {
|
||||
case msg ⇒ {
|
||||
|
|
|
|||
|
|
@ -5,13 +5,13 @@ package docs.zeromq
|
|||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.actor.{ Actor, Props }
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.actor.{ Actor, Props }
|
||||
import akka.testkit._
|
||||
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension }
|
||||
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.Date
|
||||
import akka.zeromq.{ SocketType, Bind }
|
||||
|
||||
object ZeromqDocSpec {
|
||||
|
||||
|
|
@ -52,12 +52,12 @@ object ZeromqDocSpec {
|
|||
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed,
|
||||
currentHeap.getMax)).get
|
||||
// the first frame is the topic, second is the message
|
||||
pubSocket ! ZMQMessage(Seq(Frame("health.heap"), Frame(heapPayload)))
|
||||
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.heap"), Frame(heapPayload)))
|
||||
|
||||
// use akka SerializationExtension to convert to bytes
|
||||
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get
|
||||
// the first frame is the topic, second is the message
|
||||
pubSocket ! ZMQMessage(Seq(Frame("health.load"), Frame(loadPayload)))
|
||||
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.load"), Frame(loadPayload)))
|
||||
}
|
||||
}
|
||||
//#health
|
||||
|
|
@ -146,7 +146,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
|
|||
|
||||
val payload = Array.empty[Byte]
|
||||
//#pub-topic
|
||||
pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload)))
|
||||
pubSocket ! ZMQMessage(Frame("foo.bar"), Frame(payload))
|
||||
//#pub-topic
|
||||
|
||||
system.stop(subSocket)
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import java.io.File
|
|||
import java.lang.Boolean.getBoolean
|
||||
import java.net.URLClassLoader
|
||||
import java.util.jar.JarFile
|
||||
import scala.collection.immutable
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
|
|
@ -77,8 +78,8 @@ object Main {
|
|||
|
||||
Thread.currentThread.setContextClassLoader(classLoader)
|
||||
|
||||
val bootClasses: Seq[String] = args.toSeq
|
||||
val bootables: Seq[Bootable] = bootClasses map { c ⇒ classLoader.loadClass(c).newInstance.asInstanceOf[Bootable] }
|
||||
val bootClasses: immutable.Seq[String] = args.to[immutable.Seq]
|
||||
val bootables: immutable.Seq[Bootable] = bootClasses map { c ⇒ classLoader.loadClass(c).newInstance.asInstanceOf[Bootable] }
|
||||
|
||||
for (bootable ← bootables) {
|
||||
log("Starting up " + bootable.getClass.getName)
|
||||
|
|
@ -122,7 +123,7 @@ object Main {
|
|||
new URLClassLoader(urls, Thread.currentThread.getContextClassLoader)
|
||||
}
|
||||
|
||||
private def addShutdownHook(bootables: Seq[Bootable]): Unit = {
|
||||
private def addShutdownHook(bootables: immutable.Seq[Bootable]): Unit = {
|
||||
Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {
|
||||
def run = {
|
||||
log("")
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ class SimpleNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT
|
|||
|
||||
import NamespaceHandlerTest._
|
||||
|
||||
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(
|
||||
val testBundles = buildTestBundles(List(
|
||||
AKKA_OSGI_BLUEPRINT,
|
||||
bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("simple.xml"))))
|
||||
|
||||
|
|
@ -62,7 +62,7 @@ class ConfigNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT
|
|||
|
||||
import NamespaceHandlerTest._
|
||||
|
||||
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(
|
||||
val testBundles = buildTestBundles(List(
|
||||
AKKA_OSGI_BLUEPRINT,
|
||||
bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("config.xml"))))
|
||||
|
||||
|
|
@ -94,7 +94,7 @@ class DependencyInjectionNamespaceHandlerTest extends WordSpec with MustMatchers
|
|||
|
||||
import NamespaceHandlerTest._
|
||||
|
||||
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(
|
||||
val testBundles = buildTestBundles(List(
|
||||
AKKA_OSGI_BLUEPRINT,
|
||||
bundle(TEST_BUNDLE_NAME).withBlueprintFile(getClass.getResource("injection.xml"))))
|
||||
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.actor.ActorSystem
|
|||
import akka.pattern.ask
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.util.Timeout
|
||||
import de.kalpatec.pojosr.framework.launch.BundleDescriptor
|
||||
import test.{ RuntimeNameActorSystemActivator, TestActivators, PingPongActorSystemActivator }
|
||||
|
|
@ -32,7 +33,7 @@ class PingPongActorSystemActivatorTest extends WordSpec with MustMatchers with P
|
|||
|
||||
import ActorSystemActivatorTest._
|
||||
|
||||
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(
|
||||
val testBundles: immutable.Seq[BundleDescriptor] = buildTestBundles(List(
|
||||
bundle(TEST_BUNDLE_NAME).withActivator(classOf[PingPongActorSystemActivator])))
|
||||
|
||||
"PingPongActorSystemActivator" must {
|
||||
|
|
@ -65,7 +66,8 @@ class RuntimeNameActorSystemActivatorTest extends WordSpec with MustMatchers wit
|
|||
|
||||
import ActorSystemActivatorTest._
|
||||
|
||||
val testBundles: Seq[BundleDescriptor] = buildTestBundles(Seq(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator])))
|
||||
val testBundles: immutable.Seq[BundleDescriptor] =
|
||||
buildTestBundles(List(bundle(TEST_BUNDLE_NAME).withActivator(classOf[RuntimeNameActorSystemActivator])))
|
||||
|
||||
"RuntimeNameActorSystemActivator" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -6,7 +6,6 @@ package akka.osgi
|
|||
import de.kalpatec.pojosr.framework.launch.{ BundleDescriptor, PojoServiceRegistryFactory, ClasspathScanner }
|
||||
|
||||
import scala.collection.JavaConversions.seqAsJavaList
|
||||
import scala.collection.JavaConversions.collectionAsScalaIterable
|
||||
import org.apache.commons.io.IOUtils.copy
|
||||
|
||||
import org.osgi.framework._
|
||||
|
|
@ -17,7 +16,7 @@ import java.io._
|
|||
import org.scalatest.{ BeforeAndAfterAll, Suite }
|
||||
import java.util.{ UUID, Date, ServiceLoader, HashMap }
|
||||
import scala.reflect.ClassTag
|
||||
import scala.Some
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Trait that provides support for building akka-osgi tests using PojoSR
|
||||
|
|
@ -31,7 +30,7 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
|
|||
* All bundles being found on the test classpath are automatically installed and started in the PojoSR runtime.
|
||||
* Implement this to define the extra bundles that should be available for testing.
|
||||
*/
|
||||
def testBundles: Seq[BundleDescriptor]
|
||||
def testBundles: immutable.Seq[BundleDescriptor]
|
||||
|
||||
val bufferedLoadingErrors = new ByteArrayOutputStream()
|
||||
|
||||
|
|
@ -82,15 +81,11 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
|
|||
}
|
||||
}
|
||||
|
||||
protected def buildTestBundles(builders: Seq[BundleDescriptorBuilder]): Seq[BundleDescriptor] = builders map (_.build)
|
||||
protected def buildTestBundles(builders: immutable.Seq[BundleDescriptorBuilder]): immutable.Seq[BundleDescriptor] =
|
||||
builders map (_.build)
|
||||
|
||||
def filterErrors()(block: ⇒ Unit): Unit = {
|
||||
try {
|
||||
block
|
||||
} catch {
|
||||
case e: Throwable ⇒ System.err.write(bufferedLoadingErrors.toByteArray); throw e
|
||||
}
|
||||
}
|
||||
def filterErrors()(block: ⇒ Unit): Unit =
|
||||
try block catch { case e: Throwable ⇒ System.err.write(bufferedLoadingErrors.toByteArray); throw e }
|
||||
}
|
||||
|
||||
object PojoSRTestSupport {
|
||||
|
|
@ -142,12 +137,12 @@ class BundleDescriptorBuilder(name: String) {
|
|||
}
|
||||
|
||||
def extractHeaders(file: File): HashMap[String, String] = {
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
val headers = new HashMap[String, String]()
|
||||
|
||||
val jis = new JarInputStream(new FileInputStream(file))
|
||||
try {
|
||||
for (entry ← jis.getManifest().getMainAttributes().entrySet())
|
||||
headers.put(entry.getKey().toString(), entry.getValue().toString())
|
||||
for (entry ← jis.getManifest.getMainAttributes.entrySet.asScala)
|
||||
headers.put(entry.getKey.toString, entry.getValue.toString)
|
||||
} finally jis.close()
|
||||
|
||||
headers
|
||||
|
|
|
|||
|
|
@ -6,13 +6,13 @@ package akka.remote.testconductor
|
|||
import language.postfixOps
|
||||
import java.net.InetSocketAddress
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.Queue
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import org.jboss.netty.buffer.ChannelBuffer
|
||||
import org.jboss.netty.channel.{ SimpleChannelHandler, MessageEvent, Channels, ChannelStateEvent, ChannelHandlerContext, ChannelFutureListener, ChannelFuture }
|
||||
import akka.actor.{ Props, LoggingFSM, Address, ActorSystem, ActorRef, ActorLogging, Actor, FSM }
|
||||
import akka.event.Logging
|
||||
import akka.remote.netty.ChannelAddress
|
||||
import scala.concurrent.duration._
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
|
|
@ -230,7 +230,7 @@ private[akka] object ThrottleActor {
|
|||
case object Throttle extends State
|
||||
case object Blackhole extends State
|
||||
|
||||
case class Data(lastSent: Long, rateMBit: Float, queue: Queue[Send])
|
||||
case class Data(lastSent: Long, rateMBit: Float, queue: immutable.Queue[Send])
|
||||
|
||||
case class Send(ctx: ChannelHandlerContext, direction: Direction, future: Option[ChannelFuture], msg: AnyRef)
|
||||
case class SetRate(rateMBit: Float)
|
||||
|
|
@ -248,7 +248,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext)
|
|||
|
||||
private val packetSplitThreshold = TestConductor(context.system).Settings.PacketSplitThreshold
|
||||
|
||||
startWith(PassThrough, Data(0, -1, Queue()))
|
||||
startWith(PassThrough, Data(0, -1, immutable.Queue()))
|
||||
|
||||
when(PassThrough) {
|
||||
case Event(s @ Send(_, _, _, msg), _) ⇒
|
||||
|
|
@ -258,8 +258,8 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext)
|
|||
}
|
||||
|
||||
when(Throttle) {
|
||||
case Event(s: Send, data @ Data(_, _, Queue())) ⇒
|
||||
stay using sendThrottled(data.copy(lastSent = System.nanoTime, queue = Queue(s)))
|
||||
case Event(s: Send, data @ Data(_, _, immutable.Queue())) ⇒
|
||||
stay using sendThrottled(data.copy(lastSent = System.nanoTime, queue = immutable.Queue(s)))
|
||||
case Event(s: Send, data) ⇒
|
||||
stay using sendThrottled(data.copy(queue = data.queue.enqueue(s)))
|
||||
case Event(Tick, data) ⇒
|
||||
|
|
@ -286,7 +286,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext)
|
|||
whenUnhandled {
|
||||
case Event(SetRate(rate), d) ⇒
|
||||
if (rate > 0) {
|
||||
goto(Throttle) using d.copy(lastSent = System.nanoTime, rateMBit = rate, queue = Queue())
|
||||
goto(Throttle) using d.copy(lastSent = System.nanoTime, rateMBit = rate, queue = immutable.Queue())
|
||||
} else if (rate == 0) {
|
||||
goto(Blackhole)
|
||||
} else {
|
||||
|
|
@ -328,7 +328,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext)
|
|||
*/
|
||||
private def schedule(d: Data): (Data, Seq[Send], Option[FiniteDuration]) = {
|
||||
val now = System.nanoTime
|
||||
@tailrec def rec(d: Data, toSend: Seq[Send]): (Data, Seq[Send], Option[FiniteDuration]) = {
|
||||
@tailrec def rec(d: Data, toSend: immutable.Seq[Send]): (Data, immutable.Seq[Send], Option[FiniteDuration]) = {
|
||||
if (d.queue.isEmpty) (d, toSend, None)
|
||||
else {
|
||||
val timeForPacket = d.lastSent + (1000 * size(d.queue.head.msg) / d.rateMBit).toLong
|
||||
|
|
@ -344,7 +344,7 @@ private[akka] class ThrottleActor(channelContext: ChannelHandlerContext)
|
|||
}
|
||||
}
|
||||
}
|
||||
rec(d, Seq())
|
||||
rec(d, Nil)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -4,20 +4,20 @@
|
|||
package akka.remote.testconductor
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.actor.{ Actor, ActorRef, ActorSystem, LoggingFSM, Props, PoisonPill, Status, Address, Scheduler }
|
||||
import RemoteConnection.getAddrString
|
||||
import akka.remote.testconductor.RemoteConnection.getAddrString
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ ExecutionContext, Await, Future }
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
import scala.reflect.classTag
|
||||
import akka.util.Timeout
|
||||
import org.jboss.netty.channel.{ Channel, SimpleChannelUpstreamHandler, ChannelHandlerContext, ChannelStateEvent, MessageEvent, WriteCompletionEvent, ExceptionEvent }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.pattern.{ ask, pipe, AskTimeoutException }
|
||||
import scala.util.control.NoStackTrace
|
||||
import akka.event.{ LoggingAdapter, Logging }
|
||||
import java.net.{ InetSocketAddress, ConnectException }
|
||||
import scala.reflect.classTag
|
||||
import concurrent.{ ExecutionContext, Await, Future }
|
||||
|
||||
/**
|
||||
* The Player is the client component of the
|
||||
|
|
@ -67,15 +67,13 @@ trait Player { this: TestConductorExt ⇒
|
|||
* Enter the named barriers, one after the other, in the order given. Will
|
||||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(name: String*) {
|
||||
enter(Settings.BarrierTimeout, name)
|
||||
}
|
||||
def enter(name: String*): Unit = enter(Settings.BarrierTimeout, name.to[immutable.Seq])
|
||||
|
||||
/**
|
||||
* Enter the named barriers, one after the other, in the order given. Will
|
||||
* throw an exception in case of timeouts or other errors.
|
||||
*/
|
||||
def enter(timeout: Timeout, name: Seq[String]) {
|
||||
def enter(timeout: Timeout, name: immutable.Seq[String]) {
|
||||
system.log.debug("entering barriers " + name.mkString("(", ", ", ")"))
|
||||
val stop = Deadline.now + timeout.duration
|
||||
name foreach { b ⇒
|
||||
|
|
|
|||
|
|
@ -7,16 +7,17 @@ import language.implicitConversions
|
|||
import language.postfixOps
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.TimeoutException
|
||||
import com.typesafe.config.{ ConfigObject, ConfigFactory, Config }
|
||||
import scala.concurrent.{ Await, Awaitable }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.collection.immutable
|
||||
import akka.actor._
|
||||
import akka.util.Timeout
|
||||
import akka.remote.testconductor.{ TestConductorExt, TestConductor, RoleName }
|
||||
import akka.remote.RemoteActorRefProvider
|
||||
import akka.testkit._
|
||||
import scala.concurrent.{ Await, Awaitable }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.remote.testconductor.RoleName
|
||||
import akka.remote.testconductor.TestConductorTransport
|
||||
import akka.actor.RootActorPath
|
||||
|
|
@ -30,7 +31,7 @@ abstract class MultiNodeConfig {
|
|||
private var _commonConf: Option[Config] = None
|
||||
private var _nodeConf = Map[RoleName, Config]()
|
||||
private var _roles = Vector[RoleName]()
|
||||
private var _deployments = Map[RoleName, Seq[String]]()
|
||||
private var _deployments = Map[RoleName, immutable.Seq[String]]()
|
||||
private var _allDeploy = Vector[String]()
|
||||
private var _testTransport = false
|
||||
|
||||
|
|
@ -106,9 +107,9 @@ abstract class MultiNodeConfig {
|
|||
configs reduce (_ withFallback _)
|
||||
}
|
||||
|
||||
private[testkit] def deployments(node: RoleName): Seq[String] = (_deployments get node getOrElse Nil) ++ _allDeploy
|
||||
private[testkit] def deployments(node: RoleName): immutable.Seq[String] = (_deployments get node getOrElse Nil) ++ _allDeploy
|
||||
|
||||
private[testkit] def roles: Seq[RoleName] = _roles
|
||||
private[testkit] def roles: immutable.Seq[RoleName] = _roles
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -234,7 +235,7 @@ object MultiNodeSpec {
|
|||
* `AskTimeoutException: sending to terminated ref breaks promises`. Using lazy
|
||||
* val is fine.
|
||||
*/
|
||||
abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: Seq[RoleName], deployments: RoleName ⇒ Seq[String])
|
||||
abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles: immutable.Seq[RoleName], deployments: RoleName ⇒ Seq[String])
|
||||
extends TestKit(_system) with MultiNodeSpecCallbacks {
|
||||
|
||||
import MultiNodeSpec._
|
||||
|
|
@ -294,7 +295,7 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
/**
|
||||
* All registered roles
|
||||
*/
|
||||
def roles: Seq[RoleName] = _roles
|
||||
def roles: immutable.Seq[RoleName] = _roles
|
||||
|
||||
/**
|
||||
* TO BE DEFINED BY USER: Defines the number of participants required for starting the test. This
|
||||
|
|
@ -335,9 +336,10 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
* Enter the named barriers in the order given. Use the remaining duration from
|
||||
* the innermost enclosing `within` block or the default `BarrierTimeout`
|
||||
*/
|
||||
def enterBarrier(name: String*) {
|
||||
testConductor.enter(Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)), name)
|
||||
}
|
||||
def enterBarrier(name: String*): Unit =
|
||||
testConductor.enter(
|
||||
Timeout.durationToTimeout(remainingOr(testConductor.Settings.BarrierTimeout.duration)),
|
||||
name.to[immutable.Seq])
|
||||
|
||||
/**
|
||||
* Query the controller for the transport address of the given node (by role name) and
|
||||
|
|
@ -400,10 +402,8 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
|
|||
}
|
||||
import scala.collection.JavaConverters._
|
||||
ConfigFactory.parseString(deployString).root.asScala foreach {
|
||||
case (key, value: ConfigObject) ⇒
|
||||
deployer.parseConfig(key, value.toConfig) foreach deployer.deploy
|
||||
case (key, x) ⇒
|
||||
throw new IllegalArgumentException("key " + key + " must map to deployment section, not simple value " + x)
|
||||
case (key, value: ConfigObject) ⇒ deployer.parseConfig(key, value.toConfig) foreach deployer.deploy
|
||||
case (key, x) ⇒ throw new IllegalArgumentException(s"key $key must map to deployment section, not simple value $x")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -71,7 +71,7 @@ class RemoteActorRefProvider(
|
|||
|
||||
_transport = {
|
||||
val fqn = remoteSettings.RemoteTransport
|
||||
val args = Seq(
|
||||
val args = List(
|
||||
classOf[ExtendedActorSystem] -> system,
|
||||
classOf[RemoteActorRefProvider] -> this)
|
||||
|
||||
|
|
|
|||
|
|
@ -6,8 +6,9 @@ package akka.remote
|
|||
import akka.actor._
|
||||
import akka.routing._
|
||||
import akka.remote.routing._
|
||||
import com.typesafe.config._
|
||||
import akka.ConfigurationException
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config._
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
case class RemoteScope(node: Address) extends Scope {
|
||||
|
|
@ -22,9 +23,9 @@ private[akka] class RemoteDeployer(_settings: ActorSystem.Settings, _pm: Dynamic
|
|||
case d @ Some(deploy) ⇒
|
||||
deploy.config.getString("remote") match {
|
||||
case AddressFromURIString(r) ⇒ Some(deploy.copy(scope = RemoteScope(r)))
|
||||
case str ⇒
|
||||
if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str)
|
||||
val nodes = deploy.config.getStringList("target.nodes").asScala.toIndexedSeq map (AddressFromURIString(_))
|
||||
case str if !str.isEmpty ⇒ throw new ConfigurationException("unparseable remote node name " + str)
|
||||
case _ ⇒
|
||||
val nodes = immutableSeq(deploy.config.getStringList("target.nodes")).map(AddressFromURIString(_))
|
||||
if (nodes.isEmpty || deploy.routerConfig == NoRouter) d
|
||||
else Some(deploy.copy(routerConfig = RemoteRouterConfig(deploy.routerConfig, nodes)))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,7 +8,9 @@ import java.net.InetSocketAddress
|
|||
import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean }
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||
import java.util.concurrent.Executors
|
||||
import scala.collection.mutable.HashMap
|
||||
import scala.collection.mutable
|
||||
import scala.collection.immutable
|
||||
import scala.util.control.NonFatal
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel, DefaultChannelPipeline, ChannelHandler, ChannelPipelineFactory, ChannelLocal }
|
||||
|
|
@ -20,7 +22,6 @@ import org.jboss.netty.util.{ DefaultObjectSizeEstimator, HashedWheelTimer }
|
|||
import akka.event.Logging
|
||||
import akka.remote.RemoteProtocol.AkkaRemoteProtocol
|
||||
import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteActorRefProvider, RemoteActorRef, RemoteServerStarted }
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.{ ExtendedActorSystem, Address, ActorRef }
|
||||
import com.google.protobuf.MessageLite
|
||||
|
||||
|
|
@ -40,13 +41,9 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
// TODO replace by system.scheduler
|
||||
val timer: HashedWheelTimer = new HashedWheelTimer(system.threadFactory)
|
||||
|
||||
val clientChannelFactory = settings.UseDispatcherForIO match {
|
||||
case Some(id) ⇒
|
||||
val d = system.dispatchers.lookup(id)
|
||||
new NioClientSocketChannelFactory(d, d)
|
||||
case None ⇒
|
||||
new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
|
||||
settings.ClientSocketWorkerPoolSize)
|
||||
val clientChannelFactory = {
|
||||
val boss, worker = settings.UseDispatcherForIO.map(system.dispatchers.lookup) getOrElse Executors.newCachedThreadPool()
|
||||
new NioClientSocketChannelFactory(boss, worker, settings.ClientSocketWorkerPoolSize)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -57,7 +54,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
* Construct a DefaultChannelPipeline from a sequence of handlers; to be used
|
||||
* in implementations of ChannelPipelineFactory.
|
||||
*/
|
||||
def apply(handlers: Seq[ChannelHandler]): DefaultChannelPipeline =
|
||||
def apply(handlers: immutable.Seq[ChannelHandler]): DefaultChannelPipeline =
|
||||
(new DefaultChannelPipeline /: handlers) { (p, h) ⇒ p.addLast(Logging.simpleName(h.getClass), h); p }
|
||||
|
||||
/**
|
||||
|
|
@ -73,7 +70,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
* Construct a default protocol stack, excluding the “head” handler (i.e. the one which
|
||||
* actually dispatches the received messages to the local target actors).
|
||||
*/
|
||||
def defaultStack(withTimeout: Boolean, isClient: Boolean): Seq[ChannelHandler] =
|
||||
def defaultStack(withTimeout: Boolean, isClient: Boolean): immutable.Seq[ChannelHandler] =
|
||||
(if (settings.EnableSSL) List(NettySSLSupport(settings, NettyRemoteTransport.this.log, isClient)) else Nil) :::
|
||||
(if (withTimeout) List(timeout) else Nil) :::
|
||||
msgFormat :::
|
||||
|
|
@ -142,7 +139,7 @@ private[akka] class NettyRemoteTransport(_system: ExtendedActorSystem, _provider
|
|||
def createPipeline(endpoint: ⇒ ChannelHandler, withTimeout: Boolean, isClient: Boolean): ChannelPipelineFactory =
|
||||
PipelineFactory(Seq(endpoint), withTimeout, isClient)
|
||||
|
||||
private val remoteClients = new HashMap[Address, RemoteClient]
|
||||
private val remoteClients = new mutable.HashMap[Address, RemoteClient]
|
||||
private val clientsLock = new ReentrantReadWriteLock
|
||||
|
||||
override protected def useUntrustedMode = remoteSettings.UntrustedMode
|
||||
|
|
|
|||
|
|
@ -23,14 +23,9 @@ private[akka] class NettyRemoteServer(val netty: NettyRemoteTransport) {
|
|||
|
||||
val ip = InetAddress.getByName(settings.Hostname)
|
||||
|
||||
private val factory =
|
||||
settings.UseDispatcherForIO match {
|
||||
case Some(id) ⇒
|
||||
val d = netty.system.dispatchers.lookup(id)
|
||||
new NioServerSocketChannelFactory(d, d)
|
||||
case None ⇒
|
||||
new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool(),
|
||||
settings.ServerSocketWorkerPoolSize)
|
||||
private val factory = {
|
||||
val boss, worker = settings.UseDispatcherForIO.map(netty.system.dispatchers.lookup) getOrElse Executors.newCachedThreadPool()
|
||||
new NioServerSocketChannelFactory(boss, worker, settings.ServerSocketWorkerPoolSize)
|
||||
}
|
||||
|
||||
// group of open channels, used for clean-up
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.concurrent.duration.Duration
|
|||
import java.util.concurrent.TimeUnit._
|
||||
import java.net.InetAddress
|
||||
import akka.ConfigurationException
|
||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.dispatch.ThreadPoolConfig
|
||||
|
||||
|
|
@ -89,42 +89,19 @@ private[akka] class NettySettings(config: Config, val systemName: String) {
|
|||
case sz ⇒ sz
|
||||
}
|
||||
|
||||
val SSLKeyStore = getString("ssl.key-store") match {
|
||||
case "" ⇒ None
|
||||
case keyStore ⇒ Some(keyStore)
|
||||
}
|
||||
val SSLKeyStore = Option(getString("ssl.key-store")).filter(_.length > 0)
|
||||
val SSLTrustStore = Option(getString("ssl.trust-store")).filter(_.length > 0)
|
||||
val SSLKeyStorePassword = Option(getString("ssl.key-store-password")).filter(_.length > 0)
|
||||
|
||||
val SSLTrustStore = getString("ssl.trust-store") match {
|
||||
case "" ⇒ None
|
||||
case trustStore ⇒ Some(trustStore)
|
||||
}
|
||||
val SSLTrustStorePassword = Option(getString("ssl.trust-store-password")).filter(_.length > 0)
|
||||
|
||||
val SSLKeyStorePassword = getString("ssl.key-store-password") match {
|
||||
case "" ⇒ None
|
||||
case password ⇒ Some(password)
|
||||
}
|
||||
val SSLEnabledAlgorithms = immutableSeq(getStringList("ssl.enabled-algorithms")).to[Set]
|
||||
|
||||
val SSLTrustStorePassword = getString("ssl.trust-store-password") match {
|
||||
case "" ⇒ None
|
||||
case password ⇒ Some(password)
|
||||
}
|
||||
val SSLProtocol = Option(getString("ssl.protocol")).filter(_.length > 0)
|
||||
|
||||
val SSLEnabledAlgorithms = iterableAsScalaIterableConverter(getStringList("ssl.enabled-algorithms")).asScala.toSet[String]
|
||||
val SSLRandomSource = Option(getString("ssl.sha1prng-random-source")).filter(_.length > 0)
|
||||
|
||||
val SSLProtocol = getString("ssl.protocol") match {
|
||||
case "" ⇒ None
|
||||
case protocol ⇒ Some(protocol)
|
||||
}
|
||||
|
||||
val SSLRandomSource = getString("ssl.sha1prng-random-source") match {
|
||||
case "" ⇒ None
|
||||
case path ⇒ Some(path)
|
||||
}
|
||||
|
||||
val SSLRandomNumberGenerator = getString("ssl.random-number-generator") match {
|
||||
case "" ⇒ None
|
||||
case rng ⇒ Some(rng)
|
||||
}
|
||||
val SSLRandomNumberGenerator = Option(getString("ssl.random-number-generator")).filter(_.length > 0)
|
||||
|
||||
val EnableSSL = {
|
||||
val enableSSL = getBoolean("ssl.enable")
|
||||
|
|
|
|||
|
|
@ -6,19 +6,17 @@ package akka.remote.routing
|
|||
import akka.routing.{ Route, Router, RouterConfig, RouteeProvider, Resizer }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Deploy
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor.Props
|
||||
import akka.ConfigurationException
|
||||
import akka.remote.RemoteScope
|
||||
import akka.actor.AddressFromURIString
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.Address
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.actor.ActorCell
|
||||
import akka.ConfigurationException
|
||||
import akka.remote.RemoteScope
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.collection.immutable
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.lang.IllegalStateException
|
||||
import akka.actor.ActorCell
|
||||
|
||||
/**
|
||||
* [[akka.routing.RouterConfig]] implementation for remote deployment on defined
|
||||
|
|
@ -29,7 +27,7 @@ import akka.actor.ActorCell
|
|||
@SerialVersionUID(1L)
|
||||
final case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address]) extends RouterConfig {
|
||||
|
||||
def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, nodes.asScala)
|
||||
def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, immutableSeq(nodes))
|
||||
def this(local: RouterConfig, nodes: Array[Address]) = this(local, nodes: Iterable[Address])
|
||||
|
||||
override def createRouteeProvider(context: ActorContext, routeeProps: Props) =
|
||||
|
|
@ -64,20 +62,20 @@ final case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address
|
|||
final class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _routeeProps: Props, _resizer: Option[Resizer])
|
||||
extends RouteeProvider(_context, _routeeProps, _resizer) {
|
||||
|
||||
if (nodes.isEmpty) throw new ConfigurationException("Must specify list of remote target.nodes for [%s]"
|
||||
format context.self.path.toString)
|
||||
if (nodes.isEmpty)
|
||||
throw new ConfigurationException("Must specify list of remote target.nodes for [%s]" format context.self.path.toString)
|
||||
|
||||
// need this iterator as instance variable since Resizer may call createRoutees several times
|
||||
private val nodeAddressIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator
|
||||
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||
private val childNameCounter = new AtomicInteger
|
||||
|
||||
override def registerRouteesFor(paths: Iterable[String]): Unit =
|
||||
override def registerRouteesFor(paths: immutable.Iterable[String]): Unit =
|
||||
throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]"
|
||||
format context.self.path.toString)
|
||||
|
||||
override def createRoutees(nrOfInstances: Int): Unit = {
|
||||
val refs = IndexedSeq.fill(nrOfInstances) {
|
||||
val refs = immutable.IndexedSeq.fill(nrOfInstances) {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
||||
scope = RemoteScope(nodeAddressIter.next))
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
package akka.remote.security.provider
|
||||
|
||||
import org.uncommons.maths.random.{ SeedGenerator, SeedException, SecureRandomSeedGenerator, RandomDotOrgSeedGenerator, DevRandomSeedGenerator }
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
|
|
@ -33,8 +34,8 @@ object InternetSeedGenerator {
|
|||
/**Singleton instance. */
|
||||
private final val Instance: InternetSeedGenerator = new InternetSeedGenerator
|
||||
/**Delegate generators. */
|
||||
private final val Generators: Seq[SeedGenerator] =
|
||||
Seq(new RandomDotOrgSeedGenerator, // first try the Internet seed generator
|
||||
private final val Generators: immutable.Seq[SeedGenerator] =
|
||||
List(new RandomDotOrgSeedGenerator, // first try the Internet seed generator
|
||||
new SecureRandomSeedGenerator) // this is last because it always works
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -184,31 +184,31 @@ public class JavaTestKit {
|
|||
}
|
||||
|
||||
public Object expectMsgAnyOf(Object... msgs) {
|
||||
return p.expectMsgAnyOf(Util.arrayToSeq(msgs));
|
||||
return p.expectMsgAnyOf(Util.immutableSeq(msgs));
|
||||
}
|
||||
|
||||
public Object expectMsgAnyOf(FiniteDuration max, Object... msgs) {
|
||||
return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs));
|
||||
return p.expectMsgAnyOf(max, Util.immutableSeq(msgs));
|
||||
}
|
||||
|
||||
public Object[] expectMsgAllOf(Object... msgs) {
|
||||
return (Object[]) p.expectMsgAllOf(Util.arrayToSeq(msgs)).toArray(
|
||||
return (Object[]) p.expectMsgAllOf(Util.immutableSeq(msgs)).toArray(
|
||||
Util.classTag(Object.class));
|
||||
}
|
||||
|
||||
public Object[] expectMsgAllOf(FiniteDuration max, Object... msgs) {
|
||||
return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray(
|
||||
return (Object[]) p.expectMsgAllOf(max, Util.immutableSeq(msgs)).toArray(
|
||||
Util.classTag(Object.class));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public <T> T expectMsgAnyClassOf(Class<? extends T>... classes) {
|
||||
final Object result = p.expectMsgAnyClassOf(Util.arrayToSeq(classes));
|
||||
final Object result = p.expectMsgAnyClassOf(Util.immutableSeq(classes));
|
||||
return (T) result;
|
||||
}
|
||||
|
||||
public Object expectMsgAnyClassOf(FiniteDuration max, Class<?>... classes) {
|
||||
return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes));
|
||||
return p.expectMsgAnyClassOf(max, Util.immutableSeq(classes));
|
||||
}
|
||||
|
||||
public void expectNoMsg() {
|
||||
|
|
|
|||
|
|
@ -138,7 +138,7 @@ object TestActorRef {
|
|||
def apply[T <: Actor](implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName)
|
||||
|
||||
def apply[T <: Actor](name: String)(implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({
|
||||
system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](t.runtimeClass, Seq()).recover({
|
||||
system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](t.runtimeClass, Nil).recover({
|
||||
case exception ⇒ throw ActorInitializationException(null,
|
||||
"Could not instantiate Actor" +
|
||||
"\nMake sure Actor is NOT defined inside a class/trait," +
|
||||
|
|
|
|||
|
|
@ -6,15 +6,16 @@ package akka.testkit
|
|||
import language.existentials
|
||||
|
||||
import scala.util.matching.Regex
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.reflect.ClassTag
|
||||
import akka.actor.{ DeadLetter, ActorSystem, Terminated, UnhandledMessage }
|
||||
import akka.dispatch.{ SystemMessage, Terminate }
|
||||
import akka.event.Logging.{ Warning, LogEvent, InitializeLogger, Info, Error, Debug, LoggerInitialized }
|
||||
import akka.event.Logging
|
||||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
import scala.collection.JavaConverters
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.reflect.ClassTag
|
||||
import akka.actor.NoSerializationVerificationNeeded
|
||||
import akka.japi.Util.immutableSeq
|
||||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
|
||||
/**
|
||||
* Implementation helpers of the EventFilter facilities: send `Mute`
|
||||
|
|
@ -38,22 +39,22 @@ sealed trait TestEvent
|
|||
*/
|
||||
object TestEvent {
|
||||
object Mute {
|
||||
def apply(filter: EventFilter, filters: EventFilter*): Mute = new Mute(filter +: filters.toSeq)
|
||||
def apply(filter: EventFilter, filters: EventFilter*): Mute = new Mute(filter +: filters.to[immutable.Seq])
|
||||
}
|
||||
case class Mute(filters: Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded {
|
||||
case class Mute(filters: immutable.Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded {
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.toSeq)
|
||||
def this(filters: JIterable[EventFilter]) = this(immutableSeq(filters))
|
||||
}
|
||||
object UnMute {
|
||||
def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.toSeq)
|
||||
def apply(filter: EventFilter, filters: EventFilter*): UnMute = new UnMute(filter +: filters.to[immutable.Seq])
|
||||
}
|
||||
case class UnMute(filters: Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded {
|
||||
case class UnMute(filters: immutable.Seq[EventFilter]) extends TestEvent with NoSerializationVerificationNeeded {
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(filters: JIterable[EventFilter]) = this(JavaConverters.iterableAsScalaIterableConverter(filters).asScala.toSeq)
|
||||
def this(filters: JIterable[EventFilter]) = this(immutableSeq(filters))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,15 +5,15 @@ package akka.testkit
|
|||
|
||||
import language.postfixOps
|
||||
|
||||
import scala.annotation.{ varargs, tailrec }
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.reflect.ClassTag
|
||||
import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic }
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor._
|
||||
import akka.actor.Actor._
|
||||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic }
|
||||
import atomic.AtomicInteger
|
||||
import scala.annotation.tailrec
|
||||
import akka.util.{ Timeout, BoxedType }
|
||||
import scala.annotation.varargs
|
||||
import scala.reflect.ClassTag
|
||||
|
||||
object TestActor {
|
||||
type Ignore = Option[PartialFunction[Any, Boolean]]
|
||||
|
|
@ -415,7 +415,7 @@ trait TestKitBase {
|
|||
/**
|
||||
* Same as `expectMsgAllOf(remaining, obj...)`, but correctly treating the timeFactor.
|
||||
*/
|
||||
def expectMsgAllOf[T](obj: T*): Seq[T] = expectMsgAllOf_internal(remaining, obj: _*)
|
||||
def expectMsgAllOf[T](obj: T*): immutable.Seq[T] = expectMsgAllOf_internal(remaining, obj: _*)
|
||||
|
||||
/**
|
||||
* Receive a number of messages from the test actor matching the given
|
||||
|
|
@ -430,19 +430,19 @@ trait TestKitBase {
|
|||
* expectMsgAllOf(1 second, Result1(), Result2())
|
||||
* </pre>
|
||||
*/
|
||||
def expectMsgAllOf[T](max: FiniteDuration, obj: T*): Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*)
|
||||
def expectMsgAllOf[T](max: FiniteDuration, obj: T*): immutable.Seq[T] = expectMsgAllOf_internal(max.dilated, obj: _*)
|
||||
|
||||
private def expectMsgAllOf_internal[T](max: FiniteDuration, obj: T*): Seq[T] = {
|
||||
private def expectMsgAllOf_internal[T](max: FiniteDuration, obj: T*): immutable.Seq[T] = {
|
||||
val recv = receiveN_internal(obj.size, max)
|
||||
obj foreach (x ⇒ assert(recv exists (x == _), "not found " + x))
|
||||
recv foreach (x ⇒ assert(obj exists (x == _), "found unexpected " + x))
|
||||
recv.asInstanceOf[Seq[T]]
|
||||
recv.asInstanceOf[immutable.Seq[T]]
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as `expectMsgAllClassOf(remaining, obj...)`, but correctly treating the timeFactor.
|
||||
*/
|
||||
def expectMsgAllClassOf[T](obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(remaining, obj: _*)
|
||||
def expectMsgAllClassOf[T](obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllClassOf(remaining, obj: _*)
|
||||
|
||||
/**
|
||||
* Receive a number of messages from the test actor matching the given
|
||||
|
|
@ -452,19 +452,19 @@ trait TestKitBase {
|
|||
* Wait time is bounded by the given duration, with an AssertionFailure
|
||||
* being thrown in case of timeout.
|
||||
*/
|
||||
def expectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*)
|
||||
def expectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllClassOf(max.dilated, obj: _*)
|
||||
|
||||
private def internalExpectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = {
|
||||
private def internalExpectMsgAllClassOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = {
|
||||
val recv = receiveN_internal(obj.size, max)
|
||||
obj foreach (x ⇒ assert(recv exists (_.getClass eq BoxedType(x)), "not found " + x))
|
||||
recv foreach (x ⇒ assert(obj exists (c ⇒ BoxedType(c) eq x.getClass), "found non-matching object " + x))
|
||||
recv.asInstanceOf[Seq[T]]
|
||||
recv.asInstanceOf[immutable.Seq[T]]
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as `expectMsgAllConformingOf(remaining, obj...)`, but correctly treating the timeFactor.
|
||||
*/
|
||||
def expectMsgAllConformingOf[T](obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(remaining, obj: _*)
|
||||
def expectMsgAllConformingOf[T](obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllConformingOf(remaining, obj: _*)
|
||||
|
||||
/**
|
||||
* Receive a number of messages from the test actor matching the given
|
||||
|
|
@ -477,13 +477,13 @@ trait TestKitBase {
|
|||
* Beware that one object may satisfy all given class constraints, which
|
||||
* may be counter-intuitive.
|
||||
*/
|
||||
def expectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*)
|
||||
def expectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = internalExpectMsgAllConformingOf(max.dilated, obj: _*)
|
||||
|
||||
private def internalExpectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): Seq[T] = {
|
||||
private def internalExpectMsgAllConformingOf[T](max: FiniteDuration, obj: Class[_ <: T]*): immutable.Seq[T] = {
|
||||
val recv = receiveN_internal(obj.size, max)
|
||||
obj foreach (x ⇒ assert(recv exists (BoxedType(x) isInstance _), "not found " + x))
|
||||
recv foreach (x ⇒ assert(obj exists (c ⇒ BoxedType(c) isInstance x), "found non-matching object " + x))
|
||||
recv.asInstanceOf[Seq[T]]
|
||||
recv.asInstanceOf[immutable.Seq[T]]
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -520,7 +520,7 @@ trait TestKitBase {
|
|||
* assert(series == (1 to 7).toList)
|
||||
* </pre>
|
||||
*/
|
||||
def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): Seq[T] = {
|
||||
def receiveWhile[T](max: Duration = Duration.Undefined, idle: Duration = Duration.Inf, messages: Int = Int.MaxValue)(f: PartialFunction[AnyRef, T]): immutable.Seq[T] = {
|
||||
val stop = now + remainingOrDilated(max)
|
||||
var msg: Message = NullMessage
|
||||
|
||||
|
|
@ -553,14 +553,14 @@ trait TestKitBase {
|
|||
* Same as `receiveN(n, remaining)` but correctly taking into account
|
||||
* Duration.timeFactor.
|
||||
*/
|
||||
def receiveN(n: Int): Seq[AnyRef] = receiveN_internal(n, remaining)
|
||||
def receiveN(n: Int): immutable.Seq[AnyRef] = receiveN_internal(n, remaining)
|
||||
|
||||
/**
|
||||
* Receive N messages in a row before the given deadline.
|
||||
*/
|
||||
def receiveN(n: Int, max: FiniteDuration): Seq[AnyRef] = receiveN_internal(n, max.dilated)
|
||||
def receiveN(n: Int, max: FiniteDuration): immutable.Seq[AnyRef] = receiveN_internal(n, max.dilated)
|
||||
|
||||
private def receiveN_internal(n: Int, max: Duration): Seq[AnyRef] = {
|
||||
private def receiveN_internal(n: Int, max: Duration): immutable.Seq[AnyRef] = {
|
||||
val stop = max + now
|
||||
for { x ← 1 to n } yield {
|
||||
val timeout = stop - now
|
||||
|
|
|
|||
|
|
@ -4,14 +4,16 @@ import language.implicitConversions
|
|||
|
||||
import akka.actor.ActorSystem
|
||||
import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import scala.reflect.ClassTag
|
||||
import scala.collection.immutable
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
|
||||
package object testkit {
|
||||
def filterEvents[T](eventFilters: Iterable[EventFilter])(block: ⇒ T)(implicit system: ActorSystem): T = {
|
||||
def now = System.currentTimeMillis
|
||||
|
||||
system.eventStream.publish(TestEvent.Mute(eventFilters.toSeq))
|
||||
system.eventStream.publish(TestEvent.Mute(eventFilters.to[immutable.Seq]))
|
||||
|
||||
try {
|
||||
val result = block
|
||||
|
||||
|
|
@ -23,7 +25,7 @@ package object testkit {
|
|||
|
||||
result
|
||||
} finally {
|
||||
system.eventStream.publish(TestEvent.UnMute(eventFilters.toSeq))
|
||||
system.eventStream.publish(TestEvent.UnMute(eventFilters.to[immutable.Seq]))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -25,14 +25,14 @@ import akka.testkit.ErrorFilter;
|
|||
import akka.testkit.TestEvent;
|
||||
import akka.util.Timeout;
|
||||
|
||||
import java.util.Arrays;
|
||||
import static akka.japi.Util.immutableSeq;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.collection.Seq;
|
||||
import scala.collection.immutable.Seq;
|
||||
|
||||
public class UntypedCoordinatedIncrementTest {
|
||||
private static ActorSystem system;
|
||||
|
|
@ -110,6 +110,6 @@ public class UntypedCoordinatedIncrementTest {
|
|||
}
|
||||
|
||||
public <A> Seq<A> seq(A... args) {
|
||||
return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toSeq();
|
||||
return immutableSeq(args);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -25,14 +25,14 @@ import akka.testkit.ErrorFilter;
|
|||
import akka.testkit.TestEvent;
|
||||
import akka.util.Timeout;
|
||||
|
||||
import java.util.Arrays;
|
||||
import static akka.japi.Util.immutableSeq;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.collection.Seq;
|
||||
import scala.collection.immutable.Seq;
|
||||
|
||||
public class UntypedTransactorTest {
|
||||
|
||||
|
|
@ -118,8 +118,6 @@ public class UntypedTransactorTest {
|
|||
}
|
||||
|
||||
public <A> Seq<A> seq(A... args) {
|
||||
return JavaConverters
|
||||
.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala()
|
||||
.toSeq();
|
||||
return immutableSeq(args);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue