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:
Patrik Nordwall 2012-11-15 12:33:11 +01:00
commit 1914be7069
110 changed files with 888 additions and 798 deletions

View file

@ -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);
}

View file

@ -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

View file

@ -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))
}
}
}

View file

@ -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

View file

@ -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()

View file

@ -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

View file

@ -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

View file

@ -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
}

View file

@ -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)
}

View file

@ -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 {

View file

@ -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);
}
}

View file

@ -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)

View file

@ -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 descendants 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)
}

View file

@ -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.

View file

@ -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 descendants 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)

View file

@ -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

View file

@ -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], " +

View file

@ -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] = {

View file

@ -98,5 +98,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext
def this(clazz: Class[T]) = this()(ClassTag(clazz))
override def lookup(): ExtensionId[T] = this
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, Seq(classOf[ExtendedActorSystem] -> system)).get
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] -> system)).get
}

View file

@ -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

View file

@ -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))

View file

@ -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())
}
}

View file

@ -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,

View file

@ -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 {

View file

@ -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)

View file

@ -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 {

View file

@ -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({

View file

@ -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(

View file

@ -95,7 +95,7 @@ object Futures {
*/
def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = {
implicit val ec = executor
Future.find[T](futures.asScala)(predicate.apply(_))(executor).map(JOption.fromScalaOption(_))
Future.find[T](futures.asScala)(predicate.apply(_))(executor) map JOption.fromScalaOption
}
/**

View file

@ -10,6 +10,7 @@ import java.util.concurrent.ConcurrentSkipListSet
import java.util.Comparator
import 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

View file

@ -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

View file

@ -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
}

View file

@ -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)
}

View file

@ -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

View file

@ -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 {

View file

@ -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]]
}

View file

@ -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 systems [[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

View file

@ -0,0 +1,54 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import scala.collection.immutable
import scala.annotation.tailrec
/**
* INTERNAL API
*/
private[akka] object Collections {
case object EmptyImmutableSeq extends immutable.Seq[Nothing] {
override final def iterator = Iterator.empty
override final def apply(idx: Int): Nothing = throw new java.lang.IndexOutOfBoundsException(idx.toString)
override final def length: Int = 0
}
abstract class PartialImmutableValuesIterable[From, To] extends immutable.Iterable[To] {
def isDefinedAt(from: From): Boolean
def apply(from: From): To
def valuesIterator: Iterator[From]
final def iterator: Iterator[To] = {
val superIterator = valuesIterator
new Iterator[To] {
private[this] var _next: To = _
private[this] var _hasNext = false
@tailrec override final def hasNext: Boolean =
if (!_hasNext && superIterator.hasNext) { // If we need and are able to look for the next value
val potentiallyNext = superIterator.next()
if (isDefinedAt(potentiallyNext)) {
_next = apply(potentiallyNext)
_hasNext = true
true
} else hasNext //Attempt to find the next
} else _hasNext // Return if we found one
override final def next(): To = if (hasNext) {
val ret = _next
_next = null.asInstanceOf[To] // Mark as consumed (nice to the GC, don't leak the last returned value)
_hasNext = false // Mark as consumed (we need to look for the next value)
ret
} else throw new java.util.NoSuchElementException("next")
}
}
override lazy val size: Int = iterator.size
override def foreach[C](f: To C) = iterator foreach f
}
}

View file

@ -1,45 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
//FIXME DOCS!
object Convert {
def intToBytes(value: Int): Array[Byte] = {
val bytes = Array.fill[Byte](4)(0)
bytes(0) = (value >>> 24).asInstanceOf[Byte]
bytes(1) = (value >>> 16).asInstanceOf[Byte]
bytes(2) = (value >>> 8).asInstanceOf[Byte]
bytes(3) = value.asInstanceOf[Byte]
bytes
}
def bytesToInt(bytes: Array[Byte], offset: Int): Int = {
(0 until 4).foldLeft(0)((value, index) value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
}
def longToBytes(value: Long): Array[Byte] = {
val writeBuffer = Array.fill[Byte](8)(0)
writeBuffer(0) = (value >>> 56).asInstanceOf[Byte]
writeBuffer(1) = (value >>> 48).asInstanceOf[Byte]
writeBuffer(2) = (value >>> 40).asInstanceOf[Byte]
writeBuffer(3) = (value >>> 32).asInstanceOf[Byte]
writeBuffer(4) = (value >>> 24).asInstanceOf[Byte]
writeBuffer(5) = (value >>> 16).asInstanceOf[Byte]
writeBuffer(6) = (value >>> 8).asInstanceOf[Byte]
writeBuffer(7) = (value >>> 0).asInstanceOf[Byte]
writeBuffer
}
def bytesToLong(buf: Array[Byte]): Long = {
((buf(0) & 0xFFL) << 56) |
((buf(1) & 0xFFL) << 48) |
((buf(2) & 0xFFL) << 40) |
((buf(3) & 0xFFL) << 32) |
((buf(4) & 0xFFL) << 24) |
((buf(5) & 0xFFL) << 16) |
((buf(6) & 0xFFL) << 8) |
((buf(7) & 0xFFL) << 0)
}
}

View file

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

View file

@ -3,6 +3,8 @@
*/
package akka.util
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]
}

View file

@ -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;
}

View file

@ -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)))
}

View file

@ -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)

View file

@ -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) {

View file

@ -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)
/**

View file

@ -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], " +

View file

@ -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

View file

@ -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
}

View file

@ -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

View file

@ -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
}

View file

@ -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.

View file

@ -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)

View file

@ -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] {

View file

@ -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._

View file

@ -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

View file

@ -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)
}
}

View file

@ -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")

View file

@ -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")

View file

@ -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 {

View file

@ -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")

View file

@ -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 {

View file

@ -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
}

View file

@ -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)
}

View file

@ -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 {

View file

@ -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

View file

@ -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))
}

View file

@ -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)

View file

@ -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) }

View file

@ -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

View file

@ -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"
)
}

View file

@ -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
---------------------------------

View file

@ -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
}

View file

@ -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);
}

View file

@ -0,0 +1,26 @@
.. _migration-2.2:
################################
Migration Guide 2.1.x to 2.2.x
################################
The 2.2 release contains several structural changes that require some
simple, mechanical source-level changes in client code.
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
migrating `1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_ and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
Immutable everywhere
====================
Akka has in 2.2 been refactored to require ``scala.collection.immutable`` data structures as much as possible,
this leads to fewer bugs and more opportunity for sharing data safely.
==================================== ====================================
Search Replace with
==================================== ====================================
``akka.japi.Util.arrayToSeq`` ``akka.japi.Util.immutableSeq``
==================================== ====================================
If you need to convert from Java to ``scala.collection.immutable.Seq`` or ``scala.collection.immutable.Iterable`` you should use ``akka.japi.Util.immutableSeq(…)``,
and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.<collection-type>]`` method.

View file

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

View file

@ -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 {

View file

@ -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 {

View file

@ -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)

View file

@ -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("")

View file

@ -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"))))

View file

@ -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 {

View file

@ -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

View file

@ -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)
}
/**

View file

@ -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

View file

@ -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")
}
}

View file

@ -71,7 +71,7 @@ class RemoteActorRefProvider(
_transport = {
val fqn = remoteSettings.RemoteTransport
val args = Seq(
val args = List(
classOf[ExtendedActorSystem] -> system,
classOf[RemoteActorRefProvider] -> this)

View file

@ -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)))
}

View file

@ -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

View file

@ -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

View file

@ -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")

View file

@ -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))

View file

@ -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
}

View file

@ -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() {

View file

@ -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," +

View file

@ -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))
}
}

View file

@ -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

View file

@ -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]))
}
}

View file

@ -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);
}
}

View file

@ -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