Merge branch 'master' into wip-2053d-actorbased-remote-drewhk
This commit is contained in:
commit
0f0c5cb17a
160 changed files with 2071 additions and 1089 deletions
|
|
@ -1,5 +1,7 @@
|
||||||
package akka.japi;
|
package akka.japi;
|
||||||
|
|
||||||
|
import akka.event.LoggingAdapter;
|
||||||
|
import akka.event.NoLogging;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
|
|
@ -46,4 +48,10 @@ public class JavaAPITestBase {
|
||||||
public void shouldBeSingleton() {
|
public void shouldBeSingleton() {
|
||||||
assertSame(Option.none(), Option.none());
|
assertSame(Option.none(), Option.none());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void mustBeAbleToGetNoLogging() {
|
||||||
|
LoggingAdapter a = NoLogging.getInstance();
|
||||||
|
assertNotNull(a);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -15,7 +15,8 @@ public class CustomRouteTest {
|
||||||
// only to test compilability
|
// only to test compilability
|
||||||
public void testRoute() {
|
public void testRoute() {
|
||||||
final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1)));
|
final ActorRef ref = system.actorOf(new Props().withRouter(new RoundRobinRouter(1)));
|
||||||
final scala.Function1<scala.Tuple2<ActorRef, Object>, scala.collection.Iterable<Destination>> route = ExtractRoute.apply(ref);
|
final scala.Function1<scala.Tuple2<ActorRef, Object>,
|
||||||
|
scala.collection.immutable.Iterable<Destination>> route = ExtractRoute.apply(ref);
|
||||||
route.apply(null);
|
route.apply(null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -103,6 +103,32 @@ class ActorDSLSpec extends AkkaSpec {
|
||||||
i.receive() must be("hi")
|
i.receive() must be("hi")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"support becomeStacked" in {
|
||||||
|
//#becomeStacked
|
||||||
|
val a = actor(new Act {
|
||||||
|
become { // this will replace the initial (empty) behavior
|
||||||
|
case "info" ⇒ sender ! "A"
|
||||||
|
case "switch" ⇒
|
||||||
|
becomeStacked { // this will stack upon the "A" behavior
|
||||||
|
case "info" ⇒ sender ! "B"
|
||||||
|
case "switch" ⇒ unbecome() // return to the "A" behavior
|
||||||
|
}
|
||||||
|
case "lobotomize" ⇒ unbecome() // OH NOES: Actor.emptyBehavior
|
||||||
|
}
|
||||||
|
})
|
||||||
|
//#becomeStacked
|
||||||
|
|
||||||
|
implicit def sender = testActor
|
||||||
|
a ! "info"
|
||||||
|
expectMsg("A")
|
||||||
|
a ! "switch"
|
||||||
|
a ! "info"
|
||||||
|
expectMsg("B")
|
||||||
|
a ! "switch"
|
||||||
|
a ! "info"
|
||||||
|
expectMsg("A")
|
||||||
|
}
|
||||||
|
|
||||||
"support setup/teardown" in {
|
"support setup/teardown" in {
|
||||||
//#simple-start-stop
|
//#simple-start-stop
|
||||||
val a = actor(new Act {
|
val a = actor(new Act {
|
||||||
|
|
@ -188,7 +214,7 @@ class ActorDSLSpec extends AkkaSpec {
|
||||||
become {
|
become {
|
||||||
case 1 ⇒ stash()
|
case 1 ⇒ stash()
|
||||||
case 2 ⇒
|
case 2 ⇒
|
||||||
testActor ! 2; unstashAll(); become {
|
testActor ! 2; unstashAll(); becomeStacked {
|
||||||
case 1 ⇒ testActor ! 1; unbecome()
|
case 1 ⇒ testActor ! 1; unbecome()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,14 +4,15 @@
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.collection.JavaConverters
|
import java.util.concurrent.{ RejectedExecutionException, ConcurrentLinkedQueue }
|
||||||
import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue }
|
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
|
||||||
|
|
@ -102,8 +103,6 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
|
||||||
}
|
}
|
||||||
|
|
||||||
"run termination callbacks in order" in {
|
"run termination callbacks in order" in {
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
|
|
||||||
val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf)
|
val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf)
|
||||||
val result = new ConcurrentLinkedQueue[Int]
|
val result = new ConcurrentLinkedQueue[Int]
|
||||||
val count = 10
|
val count = 10
|
||||||
|
|
@ -121,13 +120,11 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
|
||||||
Await.ready(latch, 5 seconds)
|
Await.ready(latch, 5 seconds)
|
||||||
|
|
||||||
val expected = (for (i ← 1 to count) yield i).reverse
|
val expected = (for (i ← 1 to count) yield i).reverse
|
||||||
result.asScala.toSeq must be(expected)
|
|
||||||
|
|
||||||
|
immutableSeq(result) must be(expected)
|
||||||
}
|
}
|
||||||
|
|
||||||
"awaitTermination after termination callbacks" in {
|
"awaitTermination after termination callbacks" in {
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
|
|
||||||
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
|
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
|
||||||
@volatile
|
@volatile
|
||||||
var callbackWasRun = false
|
var callbackWasRun = false
|
||||||
|
|
|
||||||
|
|
@ -14,7 +14,7 @@ import akka.dispatch.MessageDispatcher
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress }
|
import java.net.{ Socket, InetSocketAddress, InetAddress, SocketAddress }
|
||||||
import scala.util.Failure
|
import scala.util.Failure
|
||||||
import annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
||||||
object IOActorSpec {
|
object IOActorSpec {
|
||||||
|
|
||||||
|
|
@ -55,6 +55,8 @@ object IOActorSpec {
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
|
|
||||||
|
case _: IO.Connected ⇒ //don't care
|
||||||
|
|
||||||
case bytes: ByteString ⇒
|
case bytes: ByteString ⇒
|
||||||
val source = sender
|
val source = sender
|
||||||
socket write bytes
|
socket write bytes
|
||||||
|
|
@ -65,9 +67,9 @@ object IOActorSpec {
|
||||||
|
|
||||||
case IO.Closed(`socket`, cause) ⇒
|
case IO.Closed(`socket`, cause) ⇒
|
||||||
state(cause)
|
state(cause)
|
||||||
throw cause match {
|
cause match {
|
||||||
case IO.Error(e) ⇒ e
|
case IO.Error(e) ⇒ throw e
|
||||||
case _ ⇒ new RuntimeException("Socket closed")
|
case _ ⇒ throw new RuntimeException("Socket closed")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -154,6 +156,8 @@ object IOActorSpec {
|
||||||
case IO.Read(socket, bytes) ⇒
|
case IO.Read(socket, bytes) ⇒
|
||||||
state(socket)(IO Chunk bytes)
|
state(socket)(IO Chunk bytes)
|
||||||
|
|
||||||
|
case _: IO.Connected ⇒ //don't care
|
||||||
|
|
||||||
case IO.Closed(socket, cause) ⇒
|
case IO.Closed(socket, cause) ⇒
|
||||||
state -= socket
|
state -= socket
|
||||||
|
|
||||||
|
|
@ -181,6 +185,8 @@ object IOActorSpec {
|
||||||
readResult map (source !)
|
readResult map (source !)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case _: IO.Connected ⇒ //don't care
|
||||||
|
|
||||||
case IO.Read(`socket`, bytes) ⇒
|
case IO.Read(`socket`, bytes) ⇒
|
||||||
state(IO Chunk bytes)
|
state(IO Chunk bytes)
|
||||||
|
|
||||||
|
|
@ -276,7 +282,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout {
|
||||||
}
|
}
|
||||||
|
|
||||||
"an IO Actor" must {
|
"an IO Actor" must {
|
||||||
implicit val ec = system.dispatcher
|
import system.dispatcher
|
||||||
"run echo server" in {
|
"run echo server" in {
|
||||||
filterException[java.net.ConnectException] {
|
filterException[java.net.ConnectException] {
|
||||||
val addressPromise = Promise[SocketAddress]()
|
val addressPromise = Promise[SocketAddress]()
|
||||||
|
|
|
||||||
|
|
@ -43,9 +43,10 @@ class LocalActorRefProviderSpec extends AkkaSpec(LocalActorRefProviderSpec.confi
|
||||||
val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552"
|
val childName = "akka%3A%2F%2FClusterSystem%40127.0.0.1%3A2552"
|
||||||
val a = system.actorOf(Props(new Actor {
|
val a = system.actorOf(Props(new Actor {
|
||||||
val child = context.actorOf(Props.empty, name = childName)
|
val child = context.actorOf(Props.empty, name = childName)
|
||||||
assert(childName == child.path.name)
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case "lookup" ⇒ sender ! context.actorFor(childName)
|
case "lookup" ⇒
|
||||||
|
if (childName == child.path.name) sender ! context.actorFor(childName)
|
||||||
|
else sender ! s"$childName is not ${child.path.name}!"
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
a.tell("lookup", testActor)
|
a.tell("lookup", testActor)
|
||||||
|
|
|
||||||
|
|
@ -6,24 +6,22 @@ package akka.actor
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
import java.net.URLEncoder
|
import java.net.URLEncoder
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
class RelativeActorPathSpec extends WordSpec with MustMatchers {
|
class RelativeActorPathSpec extends WordSpec with MustMatchers {
|
||||||
|
|
||||||
def elements(path: String): Seq[String] = path match {
|
def elements(path: String): immutable.Seq[String] = RelativeActorPath.unapply(path).getOrElse(Nil)
|
||||||
case RelativeActorPath(elem) ⇒ elem.toSeq
|
|
||||||
case _ ⇒ Nil
|
|
||||||
}
|
|
||||||
|
|
||||||
"RelativeActorPath" must {
|
"RelativeActorPath" must {
|
||||||
"match single name" in {
|
"match single name" in {
|
||||||
elements("foo") must be(Seq("foo"))
|
elements("foo") must be(List("foo"))
|
||||||
}
|
}
|
||||||
"match path separated names" in {
|
"match path separated names" in {
|
||||||
elements("foo/bar/baz") must be(Seq("foo", "bar", "baz"))
|
elements("foo/bar/baz") must be(List("foo", "bar", "baz"))
|
||||||
}
|
}
|
||||||
"match url encoded name" in {
|
"match url encoded name" in {
|
||||||
val name = URLEncoder.encode("akka://ClusterSystem@127.0.0.1:2552", "UTF-8")
|
val name = URLEncoder.encode("akka://ClusterSystem@127.0.0.1:2552", "UTF-8")
|
||||||
elements(name) must be(Seq(name))
|
elements(name) must be(List(name))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -194,7 +194,7 @@ object SupervisorHierarchySpec {
|
||||||
case x ⇒ (x, x)
|
case x ⇒ (x, x)
|
||||||
}
|
}
|
||||||
override val supervisorStrategy = OneForOneStrategy()(unwrap andThen {
|
override val supervisorStrategy = OneForOneStrategy()(unwrap andThen {
|
||||||
case _: Failure if pongsToGo > 0 ⇒
|
case (_: Failure, _) if pongsToGo > 0 ⇒
|
||||||
log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this))
|
log :+= Event("pongOfDeath resuming " + sender, identityHashCode(this))
|
||||||
Resume
|
Resume
|
||||||
case (f: Failure, orig) ⇒
|
case (f: Failure, orig) ⇒
|
||||||
|
|
@ -393,7 +393,7 @@ object SupervisorHierarchySpec {
|
||||||
override val supervisorStrategy = OneForOneStrategy() {
|
override val supervisorStrategy = OneForOneStrategy() {
|
||||||
case f: Failure ⇒ f.directive
|
case f: Failure ⇒ f.directive
|
||||||
case OriginalRestartException(f: Failure) ⇒ f.directive
|
case OriginalRestartException(f: Failure) ⇒ f.directive
|
||||||
case ActorInitializationException(f: Failure) ⇒ f.directive
|
case ActorInitializationException(_, _, f: Failure) ⇒ f.directive
|
||||||
case _ ⇒ Stop
|
case _ ⇒ Stop
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,18 +5,19 @@ package akka.actor
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
||||||
import akka.util.Timeout
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.concurrent.{ Await, Future, Promise }
|
import scala.concurrent.{ Await, Future, Promise }
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
|
||||||
import annotation.tailrec
|
|
||||||
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
|
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }
|
||||||
|
import akka.util.Timeout
|
||||||
import akka.japi.{ Option ⇒ JOption }
|
import akka.japi.{ Option ⇒ JOption }
|
||||||
import akka.testkit.DefaultTimeout
|
import akka.testkit.DefaultTimeout
|
||||||
import akka.dispatch.{ Dispatchers }
|
import akka.dispatch.Dispatchers
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import akka.serialization.JavaSerializer
|
import akka.serialization.JavaSerializer
|
||||||
import akka.actor.TypedActor._
|
import akka.actor.TypedActor._
|
||||||
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
import java.lang.IllegalStateException
|
import java.lang.IllegalStateException
|
||||||
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
|
import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch }
|
||||||
|
|
||||||
|
|
@ -35,9 +36,9 @@ object TypedActorSpec {
|
||||||
}
|
}
|
||||||
"""
|
"""
|
||||||
|
|
||||||
class CyclicIterator[T](val items: Seq[T]) extends Iterator[T] {
|
class CyclicIterator[T](val items: immutable.Seq[T]) extends Iterator[T] {
|
||||||
|
|
||||||
private[this] val current: AtomicReference[Seq[T]] = new AtomicReference(items)
|
private[this] val current = new AtomicReference(items)
|
||||||
|
|
||||||
def hasNext = items != Nil
|
def hasNext = items != Nil
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -45,6 +45,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
|
||||||
|
|
||||||
getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10)
|
getInt("akka.actor.deployment.default.virtual-nodes-factor") must be(10)
|
||||||
settings.DefaultVirtualNodesFactor must be(10)
|
settings.DefaultVirtualNodesFactor must be(10)
|
||||||
|
|
||||||
|
getMilliseconds("akka.actor.unstarted-push-timeout") must be(10.seconds.toMillis)
|
||||||
|
settings.UnstartedPushTimeout.duration must be(10.seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
|
|
|
||||||
|
|
@ -6,10 +6,9 @@ import org.junit.runner.RunWith
|
||||||
import org.scalatest.junit.JUnitRunner
|
import org.scalatest.junit.JUnitRunner
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor }
|
import akka.actor.{ Props, ActorSystem, Actor }
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import akka.testkit.{ DefaultTimeout, AkkaSpec }
|
import akka.testkit.{ DefaultTimeout, AkkaSpec }
|
||||||
import scala.concurrent.Await
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
object PriorityDispatcherSpec {
|
object PriorityDispatcherSpec {
|
||||||
|
|
@ -50,24 +49,23 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
|
||||||
}
|
}
|
||||||
|
|
||||||
def testOrdering(dispatcherKey: String) {
|
def testOrdering(dispatcherKey: String) {
|
||||||
|
val msgs = (1 to 100) toList
|
||||||
|
|
||||||
val actor = system.actorOf(Props(new Actor {
|
val actor = system.actorOf(Props(new Actor {
|
||||||
var acc: List[Int] = Nil
|
|
||||||
|
val acc = scala.collection.mutable.ListBuffer[Int]()
|
||||||
|
|
||||||
|
scala.util.Random.shuffle(msgs) foreach { m ⇒ self ! m }
|
||||||
|
|
||||||
|
self.tell('Result, testActor)
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case i: Int ⇒ acc = i :: acc
|
case i: Int ⇒ acc += i
|
||||||
case 'Result ⇒ sender ! acc
|
case 'Result ⇒ sender ! acc.toList
|
||||||
}
|
}
|
||||||
}).withDispatcher(dispatcherKey)).asInstanceOf[InternalActorRef]
|
}).withDispatcher(dispatcherKey))
|
||||||
|
|
||||||
actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages
|
expectMsgType[List[_]] must be === msgs
|
||||||
|
|
||||||
val msgs = (1 to 100).toList
|
|
||||||
for (m ← msgs) actor ! m
|
|
||||||
|
|
||||||
actor.resume(causedByFailure = null) //Signal the actor to start treating it's message backlog
|
|
||||||
|
|
||||||
Await.result(actor.?('Result).mapTo[List[Int]], timeout.duration) must be === msgs.reverse
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -12,17 +12,18 @@ import java.io.PrintWriter
|
||||||
import java.text.SimpleDateFormat
|
import java.text.SimpleDateFormat
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
import scala.collection.mutable.{ Map ⇒ MutableMap }
|
import scala.collection.mutable.{ Map ⇒ MutableMap }
|
||||||
|
import scala.collection.immutable
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
|
|
||||||
trait BenchResultRepository {
|
trait BenchResultRepository {
|
||||||
def add(stats: Stats)
|
def add(stats: Stats)
|
||||||
|
|
||||||
def get(name: String): Seq[Stats]
|
def get(name: String): immutable.Seq[Stats]
|
||||||
|
|
||||||
def get(name: String, load: Int): Option[Stats]
|
def get(name: String, load: Int): Option[Stats]
|
||||||
|
|
||||||
def getWithHistorical(name: String, load: Int): Seq[Stats]
|
def getWithHistorical(name: String, load: Int): immutable.Seq[Stats]
|
||||||
|
|
||||||
def isBaseline(stats: Stats): Boolean
|
def isBaseline(stats: Stats): Boolean
|
||||||
|
|
||||||
|
|
@ -38,9 +39,9 @@ object BenchResultRepository {
|
||||||
}
|
}
|
||||||
|
|
||||||
class FileBenchResultRepository extends BenchResultRepository {
|
class FileBenchResultRepository extends BenchResultRepository {
|
||||||
private val statsByName = MutableMap[String, Seq[Stats]]()
|
private val statsByName = MutableMap[String, immutable.Seq[Stats]]()
|
||||||
private val baselineStats = MutableMap[Key, Stats]()
|
private val baselineStats = MutableMap[Key, Stats]()
|
||||||
private val historicalStats = MutableMap[Key, Seq[Stats]]()
|
private val historicalStats = MutableMap[Key, immutable.Seq[Stats]]()
|
||||||
private def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir")
|
private def resultDir = BenchmarkConfig.config.getString("benchmark.resultDir")
|
||||||
private val serDir = resultDir + "/ser"
|
private val serDir = resultDir + "/ser"
|
||||||
private def serDirExists: Boolean = new File(serDir).exists
|
private def serDirExists: Boolean = new File(serDir).exists
|
||||||
|
|
@ -51,13 +52,13 @@ class FileBenchResultRepository extends BenchResultRepository {
|
||||||
case class Key(name: String, load: Int)
|
case class Key(name: String, load: Int)
|
||||||
|
|
||||||
def add(stats: Stats): Unit = synchronized {
|
def add(stats: Stats): Unit = synchronized {
|
||||||
val values = statsByName.getOrElseUpdate(stats.name, IndexedSeq.empty)
|
val values = statsByName.getOrElseUpdate(stats.name, Vector.empty)
|
||||||
statsByName(stats.name) = values :+ stats
|
statsByName(stats.name) = values :+ stats
|
||||||
save(stats)
|
save(stats)
|
||||||
}
|
}
|
||||||
|
|
||||||
def get(name: String): Seq[Stats] = synchronized {
|
def get(name: String): immutable.Seq[Stats] = synchronized {
|
||||||
statsByName.getOrElse(name, IndexedSeq.empty)
|
statsByName.getOrElse(name, Vector.empty)
|
||||||
}
|
}
|
||||||
|
|
||||||
def get(name: String, load: Int): Option[Stats] = synchronized {
|
def get(name: String, load: Int): Option[Stats] = synchronized {
|
||||||
|
|
@ -68,13 +69,13 @@ class FileBenchResultRepository extends BenchResultRepository {
|
||||||
baselineStats.get(Key(stats.name, stats.load)) == Some(stats)
|
baselineStats.get(Key(stats.name, stats.load)) == Some(stats)
|
||||||
}
|
}
|
||||||
|
|
||||||
def getWithHistorical(name: String, load: Int): Seq[Stats] = synchronized {
|
def getWithHistorical(name: String, load: Int): immutable.Seq[Stats] = synchronized {
|
||||||
val key = Key(name, load)
|
val key = Key(name, load)
|
||||||
val historical = historicalStats.getOrElse(key, IndexedSeq.empty)
|
val historical = historicalStats.getOrElse(key, Vector.empty)
|
||||||
val baseline = baselineStats.get(key)
|
val baseline = baselineStats.get(key)
|
||||||
val current = get(name, load)
|
val current = get(name, load)
|
||||||
|
|
||||||
val limited = (IndexedSeq.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical)
|
val limited = (Vector.empty ++ historical ++ baseline ++ current).takeRight(maxHistorical)
|
||||||
limited.sortBy(_.timestamp)
|
limited.sortBy(_.timestamp)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -94,7 +95,7 @@ class FileBenchResultRepository extends BenchResultRepository {
|
||||||
}
|
}
|
||||||
val historical = load(historicalFiles)
|
val historical = load(historicalFiles)
|
||||||
for (h ← historical) {
|
for (h ← historical) {
|
||||||
val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), IndexedSeq.empty)
|
val values = historicalStats.getOrElseUpdate(Key(h.name, h.load), Vector.empty)
|
||||||
historicalStats(Key(h.name, h.load)) = values :+ h
|
historicalStats(Key(h.name, h.load)) = values :+ h
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -120,7 +121,7 @@ class FileBenchResultRepository extends BenchResultRepository {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def load(files: Iterable[File]): Seq[Stats] = {
|
private def load(files: Iterable[File]): immutable.Seq[Stats] = {
|
||||||
val result =
|
val result =
|
||||||
for (f ← files) yield {
|
for (f ← files) yield {
|
||||||
var in: ObjectInputStream = null
|
var in: ObjectInputStream = null
|
||||||
|
|
@ -132,11 +133,11 @@ class FileBenchResultRepository extends BenchResultRepository {
|
||||||
case e: Throwable ⇒
|
case e: Throwable ⇒
|
||||||
None
|
None
|
||||||
} finally {
|
} finally {
|
||||||
if (in ne null) try { in.close() } catch { case ignore: Exception ⇒ }
|
if (in ne null) try in.close() catch { case ignore: Exception ⇒ }
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
result.flatten.toSeq.sortBy(_.timestamp)
|
result.flatten.toVector.sortBy(_.timestamp)
|
||||||
}
|
}
|
||||||
|
|
||||||
loadFiles()
|
loadFiles()
|
||||||
|
|
|
||||||
|
|
@ -3,7 +3,7 @@ package akka.performance.workbench
|
||||||
import java.io.UnsupportedEncodingException
|
import java.io.UnsupportedEncodingException
|
||||||
import java.net.URLEncoder
|
import java.net.URLEncoder
|
||||||
|
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Generates URLs to Google Chart API http://code.google.com/apis/chart/
|
* Generates URLs to Google Chart API http://code.google.com/apis/chart/
|
||||||
|
|
@ -16,7 +16,7 @@ object GoogleChartBuilder {
|
||||||
/**
|
/**
|
||||||
* Builds a bar chart for tps in the statistics.
|
* Builds a bar chart for tps in the statistics.
|
||||||
*/
|
*/
|
||||||
def tpsChartUrl(statsByTimestamp: TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = {
|
def tpsChartUrl(statsByTimestamp: immutable.TreeMap[Long, Seq[Stats]], title: String, legend: Stats ⇒ String): String = {
|
||||||
if (statsByTimestamp.isEmpty) ""
|
if (statsByTimestamp.isEmpty) ""
|
||||||
else {
|
else {
|
||||||
val loads = statsByTimestamp.values.head.map(_.load)
|
val loads = statsByTimestamp.values.head.map(_.load)
|
||||||
|
|
@ -46,7 +46,7 @@ object GoogleChartBuilder {
|
||||||
//sb.append("&")
|
//sb.append("&")
|
||||||
|
|
||||||
// legend
|
// legend
|
||||||
val legendStats = statsByTimestamp.values.map(_.head).toSeq
|
val legendStats = statsByTimestamp.values.toVector.map(_.head)
|
||||||
appendLegend(legendStats, sb, legend)
|
appendLegend(legendStats, sb, legend)
|
||||||
sb.append("&")
|
sb.append("&")
|
||||||
// bar spacing
|
// bar spacing
|
||||||
|
|
@ -60,10 +60,7 @@ object GoogleChartBuilder {
|
||||||
val loadStr = loads.mkString(",")
|
val loadStr = loads.mkString(",")
|
||||||
sb.append("chd=t:")
|
sb.append("chd=t:")
|
||||||
val maxValue = allStats.map(_.tps).max
|
val maxValue = allStats.map(_.tps).max
|
||||||
val tpsSeries: Iterable[String] =
|
val tpsSeries: Iterable[String] = for (statsSeq ← statsByTimestamp.values) yield statsSeq.map(_.tps).mkString(",")
|
||||||
for (statsSeq ← statsByTimestamp.values) yield {
|
|
||||||
statsSeq.map(_.tps).mkString(",")
|
|
||||||
}
|
|
||||||
sb.append(tpsSeries.mkString("|"))
|
sb.append(tpsSeries.mkString("|"))
|
||||||
|
|
||||||
// y range
|
// y range
|
||||||
|
|
@ -83,7 +80,7 @@ object GoogleChartBuilder {
|
||||||
/**
|
/**
|
||||||
* Builds a bar chart for all percentiles and the mean in the statistics.
|
* Builds a bar chart for all percentiles and the mean in the statistics.
|
||||||
*/
|
*/
|
||||||
def percentilesAndMeanChartUrl(statistics: Seq[Stats], title: String, legend: Stats ⇒ String): String = {
|
def percentilesAndMeanChartUrl(statistics: immutable.Seq[Stats], title: String, legend: Stats ⇒ String): String = {
|
||||||
if (statistics.isEmpty) ""
|
if (statistics.isEmpty) ""
|
||||||
else {
|
else {
|
||||||
val current = statistics.last
|
val current = statistics.last
|
||||||
|
|
@ -146,13 +143,13 @@ object GoogleChartBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def percentileLabels(percentiles: TreeMap[Int, Long], sb: StringBuilder) {
|
private def percentileLabels(percentiles: immutable.TreeMap[Int, Long], sb: StringBuilder) {
|
||||||
sb.append("chxl=1:|")
|
sb.append("chxl=1:|")
|
||||||
val s = percentiles.keys.toList.map(_ + "%").mkString("|")
|
val s = percentiles.keys.toList.map(_ + "%").mkString("|")
|
||||||
sb.append(s)
|
sb.append(s)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def appendLegend(statistics: Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) {
|
private def appendLegend(statistics: immutable.Seq[Stats], sb: StringBuilder, legend: Stats ⇒ String) {
|
||||||
val legends = statistics.map(legend(_))
|
val legends = statistics.map(legend(_))
|
||||||
sb.append("chdl=")
|
sb.append("chdl=")
|
||||||
val s = legends.map(urlEncode(_)).mkString("|")
|
val s = legends.map(urlEncode(_)).mkString("|")
|
||||||
|
|
@ -166,7 +163,7 @@ object GoogleChartBuilder {
|
||||||
sb.append(s)
|
sb.append(s)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def dataSeries(allPercentiles: Seq[TreeMap[Int, Long]], meanValues: Seq[Double], sb: StringBuilder) {
|
private def dataSeries(allPercentiles: immutable.Seq[immutable.TreeMap[Int, Long]], meanValues: immutable.Seq[Double], sb: StringBuilder) {
|
||||||
val percentileSeries =
|
val percentileSeries =
|
||||||
for {
|
for {
|
||||||
percentiles ← allPercentiles
|
percentiles ← allPercentiles
|
||||||
|
|
@ -181,7 +178,7 @@ object GoogleChartBuilder {
|
||||||
sb.append(series.mkString("|"))
|
sb.append(series.mkString("|"))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def dataSeries(values: Seq[Double], sb: StringBuilder) {
|
private def dataSeries(values: immutable.Seq[Double], sb: StringBuilder) {
|
||||||
val series = values.map(formatDouble(_))
|
val series = values.map(formatDouble(_))
|
||||||
sb.append(series.mkString("|"))
|
sb.append(series.mkString("|"))
|
||||||
}
|
}
|
||||||
|
|
@ -198,7 +195,7 @@ object GoogleChartBuilder {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def latencyAndThroughputChartUrl(statistics: Seq[Stats], title: String): String = {
|
def latencyAndThroughputChartUrl(statistics: immutable.Seq[Stats], title: String): String = {
|
||||||
if (statistics.isEmpty) ""
|
if (statistics.isEmpty) ""
|
||||||
else {
|
else {
|
||||||
val sb = new StringBuilder
|
val sb = new StringBuilder
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@ import java.text.SimpleDateFormat
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable
|
||||||
|
|
||||||
class Report(
|
class Report(
|
||||||
system: ActorSystem,
|
system: ActorSystem,
|
||||||
|
|
@ -19,7 +19,7 @@ class Report(
|
||||||
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
|
val legendTimeFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm")
|
||||||
val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss")
|
val fileTimestampFormat = new SimpleDateFormat("yyyyMMddHHmmss")
|
||||||
|
|
||||||
def html(statistics: Seq[Stats]) {
|
def html(statistics: immutable.Seq[Stats]) {
|
||||||
|
|
||||||
val current = statistics.last
|
val current = statistics.last
|
||||||
val sb = new StringBuilder
|
val sb = new StringBuilder
|
||||||
|
|
@ -80,13 +80,13 @@ class Report(
|
||||||
chartUrl
|
chartUrl
|
||||||
}
|
}
|
||||||
|
|
||||||
def comparePercentilesAndMeanChart(stats: Stats): Seq[String] = {
|
def comparePercentilesAndMeanChart(stats: Stats): immutable.Seq[String] = {
|
||||||
for {
|
for {
|
||||||
compareName ← compareResultWith.toSeq
|
compareName ← compareResultWith.to[immutable.Seq]
|
||||||
compareStats ← resultRepository.get(compareName, stats.load)
|
compareStats ← resultRepository.get(compareName, stats.load)
|
||||||
} yield {
|
} yield {
|
||||||
val chartTitle = stats.name + " vs. " + compareName + ", " + stats.load + " clients" + ", Percentiles and Mean (microseconds)"
|
val chartTitle = stats.name + " vs. " + compareName + ", " + stats.load + " clients" + ", Percentiles and Mean (microseconds)"
|
||||||
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(Seq(compareStats, stats), chartTitle, _.name)
|
val chartUrl = GoogleChartBuilder.percentilesAndMeanChartUrl(List(compareStats, stats), chartTitle, _.name)
|
||||||
chartUrl
|
chartUrl
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -102,17 +102,17 @@ class Report(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def compareWithHistoricalTpsChart(statistics: Seq[Stats]): Option[String] = {
|
def compareWithHistoricalTpsChart(statistics: immutable.Seq[Stats]): Option[String] = {
|
||||||
|
|
||||||
if (statistics.isEmpty) {
|
if (statistics.isEmpty) {
|
||||||
None
|
None
|
||||||
} else {
|
} else {
|
||||||
val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp)
|
val histTimestamps = resultRepository.getWithHistorical(statistics.head.name, statistics.head.load).map(_.timestamp)
|
||||||
val statsByTimestamp = TreeMap[Long, Seq[Stats]]() ++
|
val statsByTimestamp = immutable.TreeMap[Long, Seq[Stats]]() ++
|
||||||
(for (ts ← histTimestamps) yield {
|
(for (ts ← histTimestamps) yield {
|
||||||
val seq =
|
val seq =
|
||||||
for (stats ← statistics) yield {
|
for (stats ← statistics) yield {
|
||||||
val withHistorical: Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load)
|
val withHistorical: immutable.Seq[Stats] = resultRepository.getWithHistorical(stats.name, stats.load)
|
||||||
val cell = withHistorical.find(_.timestamp == ts)
|
val cell = withHistorical.find(_.timestamp == ts)
|
||||||
cell.getOrElse(Stats(stats.name, stats.load, ts))
|
cell.getOrElse(Stats(stats.name, stats.load, ts))
|
||||||
}
|
}
|
||||||
|
|
@ -131,7 +131,7 @@ class Report(
|
||||||
chartUrl
|
chartUrl
|
||||||
}
|
}
|
||||||
|
|
||||||
def formatResultsTable(statsSeq: Seq[Stats]): String = {
|
def formatResultsTable(statsSeq: immutable.Seq[Stats]): String = {
|
||||||
|
|
||||||
val name = statsSeq.head.name
|
val name = statsSeq.head.name
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,7 @@ class CustomRouteSpec extends AkkaSpec {
|
||||||
provider.createRoutees(1)
|
provider.createRoutees(1)
|
||||||
|
|
||||||
{
|
{
|
||||||
case (sender, message: String) ⇒ Seq(Destination(sender, target))
|
case (sender, message: String) ⇒ List(Destination(sender, target))
|
||||||
case (sender, message) ⇒ toAll(sender, provider.routees)
|
case (sender, message) ⇒ toAll(sender, provider.routees)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -43,8 +43,8 @@ class CustomRouteSpec extends AkkaSpec {
|
||||||
val r = Await.result(router.ask(CurrentRoutees)(1 second).
|
val r = Await.result(router.ask(CurrentRoutees)(1 second).
|
||||||
mapTo[RouterRoutees], 1 second)
|
mapTo[RouterRoutees], 1 second)
|
||||||
r.routees.size must be(1)
|
r.routees.size must be(1)
|
||||||
route(testActor -> "hallo") must be(Seq(Destination(testActor, target)))
|
route(testActor -> "hallo") must be(List(Destination(testActor, target)))
|
||||||
route(testActor -> 12) must be(Seq(Destination(testActor, r.routees.head)))
|
route(testActor -> 12) must be(List(Destination(testActor, r.routees.head)))
|
||||||
//#test-route
|
//#test-route
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,10 +10,9 @@ import akka.testkit.TestEvent._
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
import java.util.concurrent.TimeoutException
|
|
||||||
import scala.util.Try
|
import scala.util.Try
|
||||||
|
|
||||||
object ResizerSpec {
|
object ResizerSpec {
|
||||||
|
|
@ -61,10 +60,10 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
||||||
lowerBound = 2,
|
lowerBound = 2,
|
||||||
upperBound = 3)
|
upperBound = 3)
|
||||||
|
|
||||||
val c1 = resizer.capacity(IndexedSeq.empty[ActorRef])
|
val c1 = resizer.capacity(immutable.IndexedSeq.empty[ActorRef])
|
||||||
c1 must be(2)
|
c1 must be(2)
|
||||||
|
|
||||||
val current = IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor]))
|
val current = immutable.IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor]))
|
||||||
val c2 = resizer.capacity(current)
|
val c2 = resizer.capacity(current)
|
||||||
c2 must be(0)
|
c2 must be(0)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,9 +5,8 @@ package akka.routing
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import scala.collection.mutable.LinkedList
|
import scala.collection.immutable
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
|
|
@ -17,7 +16,9 @@ import akka.pattern.{ ask, pipe }
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
|
import akka.util.Collections.EmptyImmutableSeq
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
object RoutingSpec {
|
object RoutingSpec {
|
||||||
|
|
||||||
|
|
@ -54,11 +55,10 @@ object RoutingSpec {
|
||||||
class MyRouter(config: Config) extends RouterConfig {
|
class MyRouter(config: Config) extends RouterConfig {
|
||||||
val foo = config.getString("foo")
|
val foo = config.getString("foo")
|
||||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||||
val routees = IndexedSeq(routeeProvider.context.actorOf(Props[Echo]))
|
routeeProvider.registerRoutees(List(routeeProvider.context.actorOf(Props[Echo])))
|
||||||
routeeProvider.registerRoutees(routees)
|
|
||||||
|
|
||||||
{
|
{
|
||||||
case (sender, message) ⇒ Nil
|
case (sender, message) ⇒ EmptyImmutableSeq
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||||
|
|
@ -101,33 +101,34 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to send their routees" in {
|
"be able to send their routees" in {
|
||||||
class TheActor extends Actor {
|
case class TestRun(id: String, names: immutable.Iterable[String], actors: Int)
|
||||||
val routee1 = context.actorOf(Props[TestActor], "routee1")
|
val actor = system.actorOf(Props(new Actor {
|
||||||
val routee2 = context.actorOf(Props[TestActor], "routee2")
|
|
||||||
val routee3 = context.actorOf(Props[TestActor], "routee3")
|
|
||||||
val router = context.actorOf(Props[TestActor].withRouter(
|
|
||||||
ScatterGatherFirstCompletedRouter(
|
|
||||||
routees = List(routee1, routee2, routee3),
|
|
||||||
within = 5 seconds)))
|
|
||||||
|
|
||||||
def receive = {
|
def receive = {
|
||||||
case "doIt" ⇒ router ! CurrentRoutees
|
case TestRun(id, names, actors) ⇒
|
||||||
case routees: RouterRoutees ⇒ testActor forward routees
|
val routerProps = Props[TestActor].withRouter(
|
||||||
}
|
ScatterGatherFirstCompletedRouter(
|
||||||
}
|
routees = names map { context.actorOf(Props(new TestActor), _) },
|
||||||
|
within = 5 seconds))
|
||||||
|
|
||||||
val theActor = system.actorOf(Props(new TheActor), "theActor")
|
1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) }
|
||||||
theActor ! "doIt"
|
|
||||||
val routees = expectMsgPF() {
|
|
||||||
case RouterRoutees(routees) ⇒ routees.toSet
|
|
||||||
}
|
}
|
||||||
|
}))
|
||||||
|
|
||||||
routees.map(_.path.name) must be(Set("routee1", "routee2", "routee3"))
|
val actors = 15
|
||||||
|
val names = 1 to 20 map { "routee" + _ } toList
|
||||||
|
|
||||||
|
actor ! TestRun("test", names, actors)
|
||||||
|
1 to actors foreach { _ ⇒
|
||||||
|
val routees = expectMsgType[RouterRoutees].routees
|
||||||
|
routees.map(_.path.name) must be === names
|
||||||
|
}
|
||||||
|
expectNoMsg(500.millis)
|
||||||
}
|
}
|
||||||
|
|
||||||
"use configured nr-of-instances when FromConfig" in {
|
"use configured nr-of-instances when FromConfig" in {
|
||||||
val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1")
|
val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1")
|
||||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3)
|
router ! CurrentRoutees
|
||||||
|
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||||
watch(router)
|
watch(router)
|
||||||
system.stop(router)
|
system.stop(router)
|
||||||
expectMsgType[Terminated]
|
expectMsgType[Terminated]
|
||||||
|
|
@ -135,7 +136,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
|
|
||||||
"use configured nr-of-instances when router is specified" in {
|
"use configured nr-of-instances when router is specified" in {
|
||||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2")
|
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2")
|
||||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3)
|
router ! CurrentRoutees
|
||||||
|
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||||
system.stop(router)
|
system.stop(router)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -150,7 +152,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
}
|
}
|
||||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3")
|
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3")
|
||||||
Await.ready(latch, remaining)
|
Await.ready(latch, remaining)
|
||||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size must be(3)
|
router ! CurrentRoutees
|
||||||
|
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||||
system.stop(router)
|
system.stop(router)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -251,15 +254,15 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
val doneLatch = new TestLatch(connectionCount)
|
val doneLatch = new TestLatch(connectionCount)
|
||||||
|
|
||||||
//lets create some connections.
|
//lets create some connections.
|
||||||
var actors = new LinkedList[ActorRef]
|
@volatile var actors = immutable.IndexedSeq[ActorRef]()
|
||||||
var counters = new LinkedList[AtomicInteger]
|
@volatile var counters = immutable.IndexedSeq[AtomicInteger]()
|
||||||
for (i ← 0 until connectionCount) {
|
for (i ← 0 until connectionCount) {
|
||||||
counters = counters :+ new AtomicInteger()
|
counters = counters :+ new AtomicInteger()
|
||||||
|
|
||||||
val actor = system.actorOf(Props(new Actor {
|
val actor = system.actorOf(Props(new Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case "end" ⇒ doneLatch.countDown()
|
case "end" ⇒ doneLatch.countDown()
|
||||||
case msg: Int ⇒ counters.get(i).get.addAndGet(msg)
|
case msg: Int ⇒ counters(i).addAndGet(msg)
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
actors = actors :+ actor
|
actors = actors :+ actor
|
||||||
|
|
@ -278,10 +281,8 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
//now wait some and do validations.
|
//now wait some and do validations.
|
||||||
Await.ready(doneLatch, remaining)
|
Await.ready(doneLatch, remaining)
|
||||||
|
|
||||||
for (i ← 0 until connectionCount) {
|
for (i ← 0 until connectionCount)
|
||||||
val counter = counters.get(i).get
|
counters(i).get must be((iterationCount * (i + 1)))
|
||||||
counter.get must be((iterationCount * (i + 1)))
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"deliver a broadcast message using the !" in {
|
"deliver a broadcast message using the !" in {
|
||||||
|
|
|
||||||
|
|
@ -8,10 +8,31 @@ import language.postfixOps
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.concurrent.Await
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit._
|
import java.util.concurrent.TimeUnit._
|
||||||
|
import akka.testkit.AkkaSpec
|
||||||
|
import akka.testkit.TestLatch
|
||||||
|
import java.util.concurrent.TimeoutException
|
||||||
|
import akka.testkit.LongRunningTest
|
||||||
|
|
||||||
class DurationSpec extends WordSpec with MustMatchers {
|
class DurationSpec extends AkkaSpec {
|
||||||
|
|
||||||
|
"A HashedWheelTimer" must {
|
||||||
|
|
||||||
|
"not mess up long timeouts" taggedAs LongRunningTest in {
|
||||||
|
val longish = Long.MaxValue.nanos
|
||||||
|
val barrier = TestLatch()
|
||||||
|
import system.dispatcher
|
||||||
|
val job = system.scheduler.scheduleOnce(longish)(barrier.countDown())
|
||||||
|
intercept[TimeoutException] {
|
||||||
|
// this used to fire after 46 seconds due to wrap-around
|
||||||
|
Await.ready(barrier, 90 seconds)
|
||||||
|
}
|
||||||
|
job.cancel()
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
"Duration" must {
|
"Duration" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@ import scala.collection.Seq;
|
||||||
public class JAPI {
|
public class JAPI {
|
||||||
|
|
||||||
public static <T> Seq<T> seq(T... ts) {
|
public static <T> Seq<T> seq(T... ts) {
|
||||||
return Util.arrayToSeq(ts);
|
return Util.immutableSeq(ts);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -263,8 +263,11 @@ public class HashedWheelTimer implements Timer {
|
||||||
|
|
||||||
void scheduleTimeout(HashedWheelTimeout timeout, long delay) {
|
void scheduleTimeout(HashedWheelTimeout timeout, long delay) {
|
||||||
// Prepare the required parameters to schedule the timeout object.
|
// Prepare the required parameters to schedule the timeout object.
|
||||||
final long relativeIndex = Math.max(1, (delay + tickDuration - 1) / tickDuration); // If relative index < 1 then it should be 1
|
long relativeIndex = (delay + tickDuration - 1) / tickDuration;
|
||||||
|
// if the previous line had an overflow going on, then we’ll just schedule this timeout
|
||||||
|
// one tick early; that shouldn’t matter since we’re talking 270 years here
|
||||||
|
if (relativeIndex < 0) relativeIndex = delay / tickDuration;
|
||||||
|
if (relativeIndex == 0) relativeIndex = 1;
|
||||||
final long remainingRounds = relativeIndex / wheel.length;
|
final long remainingRounds = relativeIndex / wheel.length;
|
||||||
|
|
||||||
// Add the timeout to the wheel.
|
// Add the timeout to the wheel.
|
||||||
|
|
@ -304,7 +307,7 @@ public class HashedWheelTimer implements Timer {
|
||||||
|
|
||||||
while (!shutdown()) {
|
while (!shutdown()) {
|
||||||
final long deadline = waitForNextTick();
|
final long deadline = waitForNextTick();
|
||||||
if (deadline > 0)
|
if (deadline > Long.MIN_VALUE)
|
||||||
notifyExpiredTimeouts(fetchExpiredTimeouts(deadline));
|
notifyExpiredTimeouts(fetchExpiredTimeouts(deadline));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -332,7 +335,7 @@ public class HashedWheelTimer implements Timer {
|
||||||
HashedWheelTimeout timeout = i.next();
|
HashedWheelTimeout timeout = i.next();
|
||||||
if (timeout.remainingRounds <= 0) {
|
if (timeout.remainingRounds <= 0) {
|
||||||
i.remove();
|
i.remove();
|
||||||
if (timeout.deadline <= deadline) {
|
if (timeout.deadline - deadline <= 0) {
|
||||||
expiredTimeouts.add(timeout);
|
expiredTimeouts.add(timeout);
|
||||||
} else {
|
} else {
|
||||||
// Handle the case where the timeout is put into a wrong
|
// Handle the case where the timeout is put into a wrong
|
||||||
|
|
@ -368,6 +371,12 @@ public class HashedWheelTimer implements Timer {
|
||||||
expiredTimeouts.clear();
|
expiredTimeouts.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* calculate goal nanoTime from startTime and current tick number,
|
||||||
|
* then wait until that goal has been reached.
|
||||||
|
*
|
||||||
|
* @return Long.MIN_VALUE if received a shutdown request, current time otherwise (with Long.MIN_VALUE changed by +1)
|
||||||
|
*/
|
||||||
private long waitForNextTick() {
|
private long waitForNextTick() {
|
||||||
long deadline = startTime + tickDuration * tick;
|
long deadline = startTime + tickDuration * tick;
|
||||||
|
|
||||||
|
|
@ -378,7 +387,8 @@ public class HashedWheelTimer implements Timer {
|
||||||
|
|
||||||
if (sleepTimeMs <= 0) {
|
if (sleepTimeMs <= 0) {
|
||||||
tick += 1;
|
tick += 1;
|
||||||
return currentTime;
|
if (currentTime == Long.MIN_VALUE) return -Long.MAX_VALUE;
|
||||||
|
else return currentTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Check if we run on windows, as if thats the case we will need
|
// Check if we run on windows, as if thats the case we will need
|
||||||
|
|
@ -394,7 +404,7 @@ public class HashedWheelTimer implements Timer {
|
||||||
Thread.sleep(sleepTimeMs);
|
Thread.sleep(sleepTimeMs);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
if (shutdown()) {
|
if (shutdown()) {
|
||||||
return -1;
|
return Long.MIN_VALUE;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,7 @@ package akka.actor
|
||||||
|
|
||||||
import java.io.{ ObjectOutputStream, NotSerializableException }
|
import java.io.{ ObjectOutputStream, NotSerializableException }
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.immutable.TreeSet
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import akka.actor.dungeon.ChildrenContainer
|
import akka.actor.dungeon.ChildrenContainer
|
||||||
|
|
@ -76,8 +76,14 @@ trait ActorContext extends ActorRefFactory {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler.
|
* Changes the Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler.
|
||||||
* Puts the behavior on top of the hotswap stack.
|
* This method acts upon the behavior stack as follows:
|
||||||
* If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack
|
*
|
||||||
|
* - if `discardOld = true` it will replace the top element (i.e. the current behavior)
|
||||||
|
* - if `discardOld = false` it will keep the current behavior and push the given one atop
|
||||||
|
*
|
||||||
|
* The default of replacing the current behavior has been chosen to avoid memory leaks in
|
||||||
|
* case client code is written without consulting this documentation first (i.e. always pushing
|
||||||
|
* new closures and never issuing an `unbecome()`)
|
||||||
*/
|
*/
|
||||||
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit
|
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit
|
||||||
|
|
||||||
|
|
@ -102,7 +108,7 @@ trait ActorContext extends ActorRefFactory {
|
||||||
* val goodLookup = context.actorFor("kid")
|
* val goodLookup = context.actorFor("kid")
|
||||||
* }}}
|
* }}}
|
||||||
*/
|
*/
|
||||||
def children: Iterable[ActorRef]
|
def children: immutable.Iterable[ActorRef]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the child with the given name if it exists.
|
* Get the child with the given name if it exists.
|
||||||
|
|
@ -167,14 +173,20 @@ trait UntypedActorContext extends ActorContext {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Changes the Actor's behavior to become the new 'Procedure' handler.
|
* Changes the Actor's behavior to become the new 'Procedure' handler.
|
||||||
* Puts the behavior on top of the hotswap stack.
|
* Replaces the current behavior at the top of the hotswap stack.
|
||||||
*/
|
*/
|
||||||
def become(behavior: Procedure[Any]): Unit
|
def become(behavior: Procedure[Any]): Unit
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Changes the Actor's behavior to become the new 'Procedure' handler.
|
* Changes the Actor's behavior to become the new 'Procedure' handler.
|
||||||
* Puts the behavior on top of the hotswap stack.
|
* This method acts upon the behavior stack as follows:
|
||||||
* If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack
|
*
|
||||||
|
* - if `discardOld = true` it will replace the top element (i.e. the current behavior)
|
||||||
|
* - if `discardOld = false` it will keep the current behavior and push the given one atop
|
||||||
|
*
|
||||||
|
* The default of replacing the current behavior has been chosen to avoid memory leaks in
|
||||||
|
* case client code is written without consulting this documentation first (i.e. always pushing
|
||||||
|
* new closures and never issuing an `unbecome()`)
|
||||||
*/
|
*/
|
||||||
def become(behavior: Procedure[Any], discardOld: Boolean): Unit
|
def become(behavior: Procedure[Any], discardOld: Boolean): Unit
|
||||||
|
|
||||||
|
|
@ -247,12 +259,12 @@ private[akka] trait Cell {
|
||||||
*/
|
*/
|
||||||
def isLocal: Boolean
|
def isLocal: Boolean
|
||||||
/**
|
/**
|
||||||
* If the actor isLocal, returns whether messages are currently queued,
|
* If the actor isLocal, returns whether "user messages" are currently queued,
|
||||||
* “false” otherwise.
|
* “false” otherwise.
|
||||||
*/
|
*/
|
||||||
def hasMessages: Boolean
|
def hasMessages: Boolean
|
||||||
/**
|
/**
|
||||||
* If the actor isLocal, returns the number of messages currently queued,
|
* If the actor isLocal, returns the number of "user messages" currently queued,
|
||||||
* which may be a costly operation, 0 otherwise.
|
* which may be a costly operation, 0 otherwise.
|
||||||
*/
|
*/
|
||||||
def numberOfMessages: Int
|
def numberOfMessages: Int
|
||||||
|
|
@ -275,7 +287,7 @@ private[akka] object ActorCell {
|
||||||
|
|
||||||
final val emptyBehaviorStack: List[Actor.Receive] = Nil
|
final val emptyBehaviorStack: List[Actor.Receive] = Nil
|
||||||
|
|
||||||
final val emptyActorRefSet: Set[ActorRef] = TreeSet.empty
|
final val emptyActorRefSet: Set[ActorRef] = immutable.TreeSet.empty
|
||||||
}
|
}
|
||||||
|
|
||||||
//ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit)
|
//ACTORCELL IS 64bytes and should stay that way unless very good reason not to (machine sympathy, cache line fit)
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,8 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor
|
package akka.actor
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
import java.net.MalformedURLException
|
import java.net.MalformedURLException
|
||||||
|
|
||||||
object ActorPath {
|
object ActorPath {
|
||||||
|
|
@ -20,6 +22,8 @@ object ActorPath {
|
||||||
* http://www.ietf.org/rfc/rfc2396.txt
|
* http://www.ietf.org/rfc/rfc2396.txt
|
||||||
*/
|
*/
|
||||||
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
|
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
|
||||||
|
|
||||||
|
private[akka] final val emptyActorPath: immutable.Iterable[String] = List("")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -68,23 +72,18 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
|
||||||
/**
|
/**
|
||||||
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
||||||
*/
|
*/
|
||||||
def descendant(names: java.lang.Iterable[String]): ActorPath = {
|
def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
/(names.asScala)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
* Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
||||||
*/
|
*/
|
||||||
def elements: Iterable[String]
|
def elements: immutable.Iterable[String]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* ''Java API'': Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
* ''Java API'': Sequence of names for this path from root to this. Performance implication: has to allocate a list.
|
||||||
*/
|
*/
|
||||||
def getElements: java.lang.Iterable[String] = {
|
def getElements: java.lang.Iterable[String] =
|
||||||
import scala.collection.JavaConverters._
|
scala.collection.JavaConverters.asJavaIterableConverter(elements).asJava
|
||||||
elements.asJava
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Walk up the tree to obtain and return the RootActorPath.
|
* Walk up the tree to obtain and return the RootActorPath.
|
||||||
|
|
@ -112,7 +111,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
||||||
|
|
||||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||||
|
|
||||||
override val elements: Iterable[String] = List("")
|
override def elements: immutable.Iterable[String] = ActorPath.emptyActorPath
|
||||||
|
|
||||||
override val toString: String = address + name
|
override val toString: String = address + name
|
||||||
|
|
||||||
|
|
@ -121,7 +120,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
||||||
else addr + name
|
else addr + name
|
||||||
|
|
||||||
override def compareTo(other: ActorPath): Int = other match {
|
override def compareTo(other: ActorPath): Int = other match {
|
||||||
case r: RootActorPath ⇒ toString compareTo r.toString
|
case r: RootActorPath ⇒ toString compareTo r.toString // FIXME make this cheaper by comparing address and name in isolation
|
||||||
case c: ChildActorPath ⇒ 1
|
case c: ChildActorPath ⇒ 1
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -134,9 +133,9 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
||||||
|
|
||||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||||
|
|
||||||
override def elements: Iterable[String] = {
|
override def elements: immutable.Iterable[String] = {
|
||||||
@tailrec
|
@tailrec
|
||||||
def rec(p: ActorPath, acc: List[String]): Iterable[String] = p match {
|
def rec(p: ActorPath, acc: List[String]): immutable.Iterable[String] = p match {
|
||||||
case r: RootActorPath ⇒ acc
|
case r: RootActorPath ⇒ acc
|
||||||
case _ ⇒ rec(p.parent, p.name :: acc)
|
case _ ⇒ rec(p.parent, p.name :: acc)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -419,7 +419,10 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
|
||||||
* to the ActorSystem's EventStream
|
* to the ActorSystem's EventStream
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
|
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) {
|
||||||
|
require(sender ne null, "DeadLetter sender may not be null")
|
||||||
|
require(recipient ne null, "DeadLetter recipient may not be null")
|
||||||
|
}
|
||||||
|
|
||||||
private[akka] object DeadLetterActorRef {
|
private[akka] object DeadLetterActorRef {
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
|
|
@ -446,8 +449,11 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
|
||||||
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
|
override def sendSystemMessage(message: SystemMessage): Unit = specialHandle(message)
|
||||||
|
|
||||||
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match {
|
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = message match {
|
||||||
case d: DeadLetter ⇒ specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
|
case d: DeadLetter ⇒
|
||||||
case _ ⇒ if (!specialHandle(message)) eventStream.publish(DeadLetter(message, sender, this))
|
specialHandle(d.message) // do NOT form endless loops, since deadLetters will resend!
|
||||||
|
case _ if !specialHandle(message) ⇒
|
||||||
|
eventStream.publish(DeadLetter(message, if (sender eq Actor.noSender) provider.deadLetters else sender, this))
|
||||||
|
case _ ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
protected def specialHandle(msg: Any): Boolean = msg match {
|
protected def specialHandle(msg: Any): Boolean = msg match {
|
||||||
|
|
@ -530,7 +536,7 @@ private[akka] class VirtualPathContainer(
|
||||||
|
|
||||||
def hasChildren: Boolean = !children.isEmpty
|
def hasChildren: Boolean = !children.isEmpty
|
||||||
|
|
||||||
def foreachChild(f: ActorRef ⇒ Unit) = {
|
def foreachChild(f: ActorRef ⇒ Unit): Unit = {
|
||||||
val iter = children.values.iterator
|
val iter = children.values.iterator
|
||||||
while (iter.hasNext) f(iter.next)
|
while (iter.hasNext) f(iter.next)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,8 +8,9 @@ import akka.dispatch._
|
||||||
import akka.routing._
|
import akka.routing._
|
||||||
import akka.event._
|
import akka.event._
|
||||||
import akka.util.{ Switch, Helpers }
|
import akka.util.{ Switch, Helpers }
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
|
import akka.util.Collections.EmptyImmutableSeq
|
||||||
import scala.util.{ Success, Failure }
|
import scala.util.{ Success, Failure }
|
||||||
import scala.util.control.NonFatal
|
|
||||||
import scala.concurrent.{ Future, Promise }
|
import scala.concurrent.{ Future, Promise }
|
||||||
import java.util.concurrent.atomic.AtomicLong
|
import java.util.concurrent.atomic.AtomicLong
|
||||||
|
|
||||||
|
|
@ -271,10 +272,7 @@ trait ActorRefFactory {
|
||||||
*
|
*
|
||||||
* For maximum performance use a collection with efficient head & tail operations.
|
* For maximum performance use a collection with efficient head & tail operations.
|
||||||
*/
|
*/
|
||||||
def actorFor(path: java.lang.Iterable[String]): ActorRef = {
|
def actorFor(path: java.lang.Iterable[String]): ActorRef = provider.actorFor(lookupRoot, immutableSeq(path))
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
provider.actorFor(lookupRoot, path.asScala)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Construct an [[akka.actor.ActorSelection]] from the given path, which is
|
* Construct an [[akka.actor.ActorSelection]] from the given path, which is
|
||||||
|
|
@ -480,7 +478,7 @@ class LocalActorRefProvider(
|
||||||
def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras
|
def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras
|
||||||
|
|
||||||
private def guardianSupervisorStrategyConfigurator =
|
private def guardianSupervisorStrategyConfigurator =
|
||||||
dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, Seq()).get
|
dynamicAccess.createInstanceFor[SupervisorStrategyConfigurator](settings.SupervisorStrategyClass, EmptyImmutableSeq).get
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Overridable supervision strategy to be used by the “/user” guardian.
|
* Overridable supervision strategy to be used by the “/user” guardian.
|
||||||
|
|
|
||||||
|
|
@ -6,20 +6,20 @@ package akka.actor
|
||||||
|
|
||||||
import akka.event._
|
import akka.event._
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import akka.pattern.ask
|
import akka.japi.Util.immutableSeq
|
||||||
import com.typesafe.config.{ Config, ConfigFactory }
|
import com.typesafe.config.{ Config, ConfigFactory }
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.concurrent.duration.Duration
|
import scala.collection.immutable
|
||||||
import java.io.Closeable
|
import scala.concurrent.duration.{ FiniteDuration, Duration }
|
||||||
import scala.concurrent.{ Await, Awaitable, CanAwait, Future }
|
import scala.concurrent.{ Await, Awaitable, CanAwait, Future }
|
||||||
|
import scala.util.{ Failure, Success }
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import akka.util._
|
import akka.util._
|
||||||
|
import java.io.Closeable
|
||||||
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
|
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
|
||||||
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
|
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
import akka.actor.dungeon.ChildrenContainer
|
import akka.actor.dungeon.ChildrenContainer
|
||||||
import scala.concurrent.duration.FiniteDuration
|
|
||||||
import util.{ Failure, Success }
|
|
||||||
|
|
||||||
object ActorSystem {
|
object ActorSystem {
|
||||||
|
|
||||||
|
|
@ -144,7 +144,7 @@ object ActorSystem {
|
||||||
|
|
||||||
final val LogLevel: String = getString("akka.loglevel")
|
final val LogLevel: String = getString("akka.loglevel")
|
||||||
final val StdoutLogLevel: String = getString("akka.stdout-loglevel")
|
final val StdoutLogLevel: String = getString("akka.stdout-loglevel")
|
||||||
final val EventHandlers: Seq[String] = getStringList("akka.event-handlers").asScala
|
final val EventHandlers: immutable.Seq[String] = immutableSeq(getStringList("akka.event-handlers"))
|
||||||
final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
|
final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
|
||||||
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")
|
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")
|
||||||
|
|
||||||
|
|
@ -273,10 +273,7 @@ abstract class ActorSystem extends ActorRefFactory {
|
||||||
/**
|
/**
|
||||||
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
* ''Java API'': Recursively create a descendant’s path by appending all child names.
|
||||||
*/
|
*/
|
||||||
def descendant(names: java.lang.Iterable[String]): ActorPath = {
|
def descendant(names: java.lang.Iterable[String]): ActorPath = /(immutableSeq(names))
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
/(names.asScala)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Start-up time in milliseconds since the epoch.
|
* Start-up time in milliseconds since the epoch.
|
||||||
|
|
@ -536,7 +533,7 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
||||||
val scheduler: Scheduler = createScheduler()
|
val scheduler: Scheduler = createScheduler()
|
||||||
|
|
||||||
val provider: ActorRefProvider = {
|
val provider: ActorRefProvider = {
|
||||||
val arguments = Seq(
|
val arguments = Vector(
|
||||||
classOf[String] -> name,
|
classOf[String] -> name,
|
||||||
classOf[Settings] -> settings,
|
classOf[Settings] -> settings,
|
||||||
classOf[EventStream] -> eventStream,
|
classOf[EventStream] -> eventStream,
|
||||||
|
|
@ -676,9 +673,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
||||||
def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null
|
def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null
|
||||||
|
|
||||||
private def loadExtensions() {
|
private def loadExtensions() {
|
||||||
import scala.collection.JavaConverters.collectionAsScalaIterableConverter
|
immutableSeq(settings.config.getStringList("akka.extensions")) foreach { fqcn ⇒
|
||||||
settings.config.getStringList("akka.extensions").asScala foreach { fqcn ⇒
|
dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Nil) } match {
|
||||||
dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()) } match {
|
|
||||||
case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup())
|
case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup())
|
||||||
case Success(p: ExtensionId[_]) ⇒ registerExtension(p)
|
case Success(p: ExtensionId[_]) ⇒ registerExtension(p)
|
||||||
case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
case Success(other) ⇒ log.error("[{}] is not an 'ExtensionIdProvider' or 'ExtensionId', skipping...", fqcn)
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,8 @@ package akka.actor
|
||||||
import java.net.URI
|
import java.net.URI
|
||||||
import java.net.URISyntaxException
|
import java.net.URISyntaxException
|
||||||
import java.net.MalformedURLException
|
import java.net.MalformedURLException
|
||||||
import annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The address specifies the physical location under which an Actor can be
|
* The address specifies the physical location under which an Actor can be
|
||||||
|
|
@ -71,7 +72,7 @@ private[akka] trait PathUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
object RelativeActorPath extends PathUtils {
|
object RelativeActorPath extends PathUtils {
|
||||||
def unapply(addr: String): Option[Iterable[String]] = {
|
def unapply(addr: String): Option[immutable.Seq[String]] = {
|
||||||
try {
|
try {
|
||||||
val uri = new URI(addr)
|
val uri = new URI(addr)
|
||||||
if (uri.isAbsolute) None
|
if (uri.isAbsolute) None
|
||||||
|
|
@ -119,13 +120,12 @@ object AddressFromURIString {
|
||||||
* Given an ActorPath it returns the Address and the path elements if the path is well-formed
|
* Given an ActorPath it returns the Address and the path elements if the path is well-formed
|
||||||
*/
|
*/
|
||||||
object ActorPathExtractor extends PathUtils {
|
object ActorPathExtractor extends PathUtils {
|
||||||
def unapply(addr: String): Option[(Address, Iterable[String])] =
|
def unapply(addr: String): Option[(Address, immutable.Iterable[String])] =
|
||||||
try {
|
try {
|
||||||
val uri = new URI(addr)
|
val uri = new URI(addr)
|
||||||
if (uri.getRawPath == null) None
|
uri.getRawPath match {
|
||||||
else AddressFromURIString.unapply(uri) match {
|
case null ⇒ None
|
||||||
case None ⇒ None
|
case path ⇒ AddressFromURIString.unapply(uri).map((_, split(path).drop(1)))
|
||||||
case Some(addr) ⇒ Some((addr, split(uri.getRawPath).drop(1)))
|
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case _: URISyntaxException ⇒ None
|
case _: URISyntaxException ⇒ None
|
||||||
|
|
|
||||||
|
|
@ -7,10 +7,11 @@ package akka.actor
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import com.typesafe.config._
|
import com.typesafe.config._
|
||||||
import akka.routing._
|
import akka.routing._
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
import java.util.concurrent.{ TimeUnit }
|
import java.util.concurrent.{ TimeUnit }
|
||||||
import akka.util.WildcardTree
|
import akka.util.WildcardTree
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
import annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class represents deployment configuration for a given actor path. It is
|
* This class represents deployment configuration for a given actor path. It is
|
||||||
|
|
@ -141,7 +142,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
||||||
|
|
||||||
val deployment = config.withFallback(default)
|
val deployment = config.withFallback(default)
|
||||||
|
|
||||||
val routees = Vector() ++ deployment.getStringList("routees.paths").asScala
|
val routees = immutableSeq(deployment.getStringList("routees.paths"))
|
||||||
|
|
||||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||||
|
|
||||||
|
|
@ -160,7 +161,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
||||||
val vnodes = deployment.getInt("virtual-nodes-factor")
|
val vnodes = deployment.getInt("virtual-nodes-factor")
|
||||||
ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes)
|
ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes)
|
||||||
case fqn ⇒
|
case fqn ⇒
|
||||||
val args = Seq(classOf[Config] -> deployment)
|
val args = List(classOf[Config] -> deployment)
|
||||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({
|
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({
|
||||||
case exception ⇒ throw new IllegalArgumentException(
|
case exception ⇒ throw new IllegalArgumentException(
|
||||||
("Cannot instantiate router [%s], defined in [%s], " +
|
("Cannot instantiate router [%s], defined in [%s], " +
|
||||||
|
|
|
||||||
|
|
@ -3,7 +3,7 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
import scala.util.control.NonFatal
|
import scala.collection.immutable
|
||||||
import java.lang.reflect.InvocationTargetException
|
import java.lang.reflect.InvocationTargetException
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import scala.util.Try
|
import scala.util.Try
|
||||||
|
|
@ -25,7 +25,7 @@ abstract class DynamicAccess {
|
||||||
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
|
* val obj = DynamicAccess.createInstanceFor(clazz, Seq(classOf[Config] -> config, classOf[String] -> name))
|
||||||
* }}}
|
* }}}
|
||||||
*/
|
*/
|
||||||
def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T]
|
def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Obtain a `Class[_]` object loaded with the right class loader (i.e. the one
|
* Obtain a `Class[_]` object loaded with the right class loader (i.e. the one
|
||||||
|
|
@ -40,7 +40,7 @@ abstract class DynamicAccess {
|
||||||
* `args` argument. The exact usage of args depends on which type is requested,
|
* `args` argument. The exact usage of args depends on which type is requested,
|
||||||
* see the relevant requesting code for details.
|
* see the relevant requesting code for details.
|
||||||
*/
|
*/
|
||||||
def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T]
|
def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Obtain the Scala “object” instance for the given fully-qualified class name, if there is one.
|
* Obtain the Scala “object” instance for the given fully-qualified class name, if there is one.
|
||||||
|
|
@ -70,7 +70,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces
|
||||||
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c)
|
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c)
|
||||||
})
|
})
|
||||||
|
|
||||||
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: Seq[(Class[_], AnyRef)]): Try[T] =
|
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
|
||||||
Try {
|
Try {
|
||||||
val types = args.map(_._1).toArray
|
val types = args.map(_._1).toArray
|
||||||
val values = args.map(_._2).toArray
|
val values = args.map(_._2).toArray
|
||||||
|
|
@ -81,7 +81,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces
|
||||||
if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t)
|
if (t.isInstance(obj)) obj.asInstanceOf[T] else throw new ClassCastException(clazz.getName + " is not a subtype of " + t)
|
||||||
} recover { case i: InvocationTargetException if i.getTargetException ne null ⇒ throw i.getTargetException }
|
} recover { case i: InvocationTargetException if i.getTargetException ne null ⇒ throw i.getTargetException }
|
||||||
|
|
||||||
override def createInstanceFor[T: ClassTag](fqcn: String, args: Seq[(Class[_], AnyRef)]): Try[T] =
|
override def createInstanceFor[T: ClassTag](fqcn: String, args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =
|
||||||
getClassFor(fqcn) flatMap { c ⇒ createInstanceFor(c, args) }
|
getClassFor(fqcn) flatMap { c ⇒ createInstanceFor(c, args) }
|
||||||
|
|
||||||
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {
|
override def getObjectFor[T: ClassTag](fqcn: String): Try[T] = {
|
||||||
|
|
|
||||||
|
|
@ -98,5 +98,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext
|
||||||
def this(clazz: Class[T]) = this()(ClassTag(clazz))
|
def this(clazz: Class[T]) = this()(ClassTag(clazz))
|
||||||
|
|
||||||
override def lookup(): ExtensionId[T] = this
|
override def lookup(): ExtensionId[T] = this
|
||||||
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, Seq(classOf[ExtendedActorSystem] -> system)).get
|
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] -> system)).get
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -521,7 +521,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
||||||
* Main actor receive() method
|
* Main actor receive() method
|
||||||
* *******************************************
|
* *******************************************
|
||||||
*/
|
*/
|
||||||
override final def receive: Receive = {
|
override def receive: Receive = {
|
||||||
case TimeoutMarker(gen) ⇒
|
case TimeoutMarker(gen) ⇒
|
||||||
if (generation == gen) {
|
if (generation == gen) {
|
||||||
processMsg(StateTimeout, "state timeout")
|
processMsg(StateTimeout, "state timeout")
|
||||||
|
|
|
||||||
|
|
@ -5,10 +5,13 @@ package akka.actor
|
||||||
|
|
||||||
import language.implicitConversions
|
import language.implicitConversions
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit
|
|
||||||
import scala.collection.mutable.ArrayBuffer
|
|
||||||
import java.lang.{ Iterable ⇒ JIterable }
|
import java.lang.{ Iterable ⇒ JIterable }
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
|
|
@ -170,7 +173,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
||||||
* Implicit conversion from `Seq` of Throwables to a `Decider`.
|
* Implicit conversion from `Seq` of Throwables to a `Decider`.
|
||||||
* This maps the given Throwables to restarts, otherwise escalates.
|
* This maps the given Throwables to restarts, otherwise escalates.
|
||||||
*/
|
*/
|
||||||
implicit def seqThrowable2Decider(trapExit: Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit)
|
implicit def seqThrowable2Decider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit)
|
||||||
|
|
||||||
type Decider = PartialFunction[Throwable, Directive]
|
type Decider = PartialFunction[Throwable, Directive]
|
||||||
type JDecider = akka.japi.Function[Throwable, Directive]
|
type JDecider = akka.japi.Function[Throwable, Directive]
|
||||||
|
|
@ -180,25 +183,16 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
||||||
* Decider builder which just checks whether one of
|
* Decider builder which just checks whether one of
|
||||||
* the given Throwables matches the cause and restarts, otherwise escalates.
|
* the given Throwables matches the cause and restarts, otherwise escalates.
|
||||||
*/
|
*/
|
||||||
def makeDecider(trapExit: Array[Class[_]]): Decider =
|
def makeDecider(trapExit: immutable.Seq[Class[_ <: Throwable]]): Decider = {
|
||||||
{ case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate }
|
case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate
|
||||||
|
|
||||||
/**
|
|
||||||
* Decider builder which just checks whether one of
|
|
||||||
* the given Throwables matches the cause and restarts, otherwise escalates.
|
|
||||||
*/
|
|
||||||
def makeDecider(trapExit: Seq[Class[_ <: Throwable]]): Decider =
|
|
||||||
{ case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate }
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Decider builder which just checks whether one of
|
|
||||||
* the given Throwables matches the cause and restarts, otherwise escalates.
|
|
||||||
*/
|
|
||||||
def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = {
|
|
||||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
|
||||||
makeDecider(trapExit.asScala.toSeq)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decider builder which just checks whether one of
|
||||||
|
* the given Throwables matches the cause and restarts, otherwise escalates.
|
||||||
|
*/
|
||||||
|
def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(immutableSeq(trapExit))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Decider builder for Iterables of cause-directive pairs, e.g. a map obtained
|
* Decider builder for Iterables of cause-directive pairs, e.g. a map obtained
|
||||||
* from configuration; will sort the pairs so that the most specific type is
|
* from configuration; will sort the pairs so that the most specific type is
|
||||||
|
|
@ -222,14 +216,14 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
||||||
*
|
*
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
private[akka] def sort(in: Iterable[CauseDirective]): immutable.Seq[CauseDirective] =
|
||||||
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒
|
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒
|
||||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||||
case -1 ⇒ buf append ca
|
case -1 ⇒ buf append ca
|
||||||
case x ⇒ buf insert (x, ca)
|
case x ⇒ buf insert (x, ca)
|
||||||
}
|
}
|
||||||
buf
|
buf
|
||||||
}
|
}.to[immutable.IndexedSeq]
|
||||||
|
|
||||||
private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] =
|
private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] =
|
||||||
if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None
|
if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None
|
||||||
|
|
@ -338,10 +332,6 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
|
||||||
|
|
||||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
|
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
|
||||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||||
|
|
||||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) =
|
|
||||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* this is a performance optimization to avoid re-allocating the pairs upon
|
* this is a performance optimization to avoid re-allocating the pairs upon
|
||||||
* every call to requestRestartPermission, assuming that strategies are shared
|
* every call to requestRestartPermission, assuming that strategies are shared
|
||||||
|
|
@ -380,9 +370,6 @@ case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
|
||||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
|
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) =
|
||||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
||||||
|
|
||||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_]]) =
|
|
||||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit))
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* this is a performance optimization to avoid re-allocating the pairs upon
|
* this is a performance optimization to avoid re-allocating the pairs upon
|
||||||
* every call to requestRestartPermission, assuming that strategies are shared
|
* every call to requestRestartPermission, assuming that strategies are shared
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ package akka.actor
|
||||||
import language.higherKinds
|
import language.higherKinds
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.concurrent.{ ExecutionContext, Future }
|
import scala.concurrent.{ ExecutionContext, Future }
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
|
@ -122,7 +123,7 @@ object IO {
|
||||||
* @return a new SocketHandle that can be used to perform actions on the
|
* @return a new SocketHandle that can be used to perform actions on the
|
||||||
* new connection's SocketChannel.
|
* new connection's SocketChannel.
|
||||||
*/
|
*/
|
||||||
def accept(options: Seq[SocketOption] = Seq.empty)(implicit socketOwner: ActorRef): SocketHandle = {
|
def accept(options: immutable.Seq[SocketOption] = Nil)(implicit socketOwner: ActorRef): SocketHandle = {
|
||||||
val socket = SocketHandle(socketOwner, ioManager)
|
val socket = SocketHandle(socketOwner, ioManager)
|
||||||
ioManager ! Accept(socket, this, options)
|
ioManager ! Accept(socket, this, options)
|
||||||
socket
|
socket
|
||||||
|
|
@ -250,7 +251,7 @@ object IO {
|
||||||
*
|
*
|
||||||
* Normally sent using IOManager.listen()
|
* Normally sent using IOManager.listen()
|
||||||
*/
|
*/
|
||||||
case class Listen(server: ServerHandle, address: SocketAddress, options: Seq[ServerSocketOption] = Seq.empty) extends IOMessage
|
case class Listen(server: ServerHandle, address: SocketAddress, options: immutable.Seq[ServerSocketOption] = Nil) extends IOMessage
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is
|
* Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is
|
||||||
|
|
@ -272,7 +273,7 @@ object IO {
|
||||||
*
|
*
|
||||||
* Normally sent using [[akka.actor.IO.ServerHandle]].accept()
|
* Normally sent using [[akka.actor.IO.ServerHandle]].accept()
|
||||||
*/
|
*/
|
||||||
case class Accept(socket: SocketHandle, server: ServerHandle, options: Seq[SocketOption] = Seq.empty) extends IOMessage
|
case class Accept(socket: SocketHandle, server: ServerHandle, options: immutable.Seq[SocketOption] = Nil) extends IOMessage
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Message to an [[akka.actor.IOManager]] to create a SocketChannel connected
|
* Message to an [[akka.actor.IOManager]] to create a SocketChannel connected
|
||||||
|
|
@ -280,7 +281,7 @@ object IO {
|
||||||
*
|
*
|
||||||
* Normally sent using IOManager.connect()
|
* Normally sent using IOManager.connect()
|
||||||
*/
|
*/
|
||||||
case class Connect(socket: SocketHandle, address: SocketAddress, options: Seq[SocketOption] = Seq.empty) extends IOMessage
|
case class Connect(socket: SocketHandle, address: SocketAddress, options: immutable.Seq[SocketOption] = Nil) extends IOMessage
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Message from an [[akka.actor.IOManager]] that the SocketChannel has
|
* Message from an [[akka.actor.IOManager]] that the SocketChannel has
|
||||||
|
|
@ -832,7 +833,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
||||||
* @param option Seq of [[akka.actor.IO.ServerSocketOptions]] to setup on socket
|
* @param option Seq of [[akka.actor.IO.ServerSocketOptions]] to setup on socket
|
||||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||||
*/
|
*/
|
||||||
def listen(address: SocketAddress, options: Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = {
|
def listen(address: SocketAddress, options: immutable.Seq[IO.ServerSocketOption])(implicit owner: ActorRef): IO.ServerHandle = {
|
||||||
val server = IO.ServerHandle(owner, actor)
|
val server = IO.ServerHandle(owner, actor)
|
||||||
actor ! IO.Listen(server, address, options)
|
actor ! IO.Listen(server, address, options)
|
||||||
server
|
server
|
||||||
|
|
@ -847,7 +848,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
||||||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||||
*/
|
*/
|
||||||
def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Seq.empty)
|
def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = listen(address, Nil)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a ServerSocketChannel listening on a host and port. Messages will
|
* Create a ServerSocketChannel listening on a host and port. Messages will
|
||||||
|
|
@ -860,7 +861,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
||||||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||||
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
* @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket
|
||||||
*/
|
*/
|
||||||
def listen(host: String, port: Int, options: Seq[IO.ServerSocketOption] = Seq.empty)(implicit owner: ActorRef): IO.ServerHandle =
|
def listen(host: String, port: Int, options: immutable.Seq[IO.ServerSocketOption] = Nil)(implicit owner: ActorRef): IO.ServerHandle =
|
||||||
listen(new InetSocketAddress(host, port), options)(owner)
|
listen(new InetSocketAddress(host, port), options)(owner)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -873,7 +874,7 @@ final class IOManager private (system: ExtendedActorSystem) extends Extension {
|
||||||
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
* @param owner the ActorRef that will receive messages from the IOManagerActor
|
||||||
* @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket
|
* @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket
|
||||||
*/
|
*/
|
||||||
def connect(address: SocketAddress, options: Seq[IO.SocketOption] = Seq.empty)(implicit owner: ActorRef): IO.SocketHandle = {
|
def connect(address: SocketAddress, options: immutable.Seq[IO.SocketOption] = Nil)(implicit owner: ActorRef): IO.SocketHandle = {
|
||||||
val socket = IO.SocketHandle(owner, actor)
|
val socket = IO.SocketHandle(owner, actor)
|
||||||
actor ! IO.Connect(socket, address, options)
|
actor ! IO.Connect(socket, address, options)
|
||||||
socket
|
socket
|
||||||
|
|
@ -991,7 +992,7 @@ final class IOManagerActor(val settings: Settings) extends Actor with ActorLoggi
|
||||||
|
|
||||||
private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) }
|
private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) }
|
||||||
|
|
||||||
private def setSocketOptions(socket: java.net.Socket, options: Seq[IO.SocketOption]) {
|
private def setSocketOptions(socket: java.net.Socket, options: immutable.Seq[IO.SocketOption]) {
|
||||||
options foreach {
|
options foreach {
|
||||||
case IO.KeepAlive(on) ⇒ forwardFailure(socket.setKeepAlive(on))
|
case IO.KeepAlive(on) ⇒ forwardFailure(socket.setKeepAlive(on))
|
||||||
case IO.OOBInline(on) ⇒ forwardFailure(socket.setOOBInline(on))
|
case IO.OOBInline(on) ⇒ forwardFailure(socket.setOOBInline(on))
|
||||||
|
|
|
||||||
|
|
@ -189,15 +189,16 @@ private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: Rep
|
||||||
def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer
|
def childrenRefs: ChildrenContainer = ChildrenContainer.EmptyChildrenContainer
|
||||||
def getChildByName(name: String): Option[ChildRestartStats] = None
|
def getChildByName(name: String): Option[ChildRestartStats] = None
|
||||||
def tell(message: Any, sender: ActorRef): Unit = {
|
def tell(message: Any, sender: ActorRef): Unit = {
|
||||||
|
val useSender = if (sender eq Actor.noSender) system.deadLetters else sender
|
||||||
if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
|
if (lock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
|
||||||
try {
|
try {
|
||||||
if (self.underlying eq this) queue enqueue Envelope(message, sender, system)
|
if (self.underlying eq this) queue enqueue Envelope(message, useSender, system)
|
||||||
else self.underlying.tell(message, sender)
|
else self.underlying.tell(message, useSender)
|
||||||
} finally {
|
} finally {
|
||||||
lock.unlock()
|
lock.unlock()
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
system.deadLetters ! DeadLetter(message, sender, self)
|
system.deadLetters ! DeadLetter(message, useSender, self)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
def sendSystemMessage(msg: SystemMessage): Unit = {
|
def sendSystemMessage(msg: SystemMessage): Unit = {
|
||||||
|
|
|
||||||
|
|
@ -203,8 +203,8 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def close(): Unit = {
|
override def close(): Unit = {
|
||||||
import scala.collection.JavaConverters._
|
val i = hashedWheelTimer.stop().iterator()
|
||||||
hashedWheelTimer.stop().asScala foreach execDirectly
|
while (i.hasNext) execDirectly(i.next())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,13 +16,13 @@ import akka.AkkaException
|
||||||
* def receive = {
|
* def receive = {
|
||||||
* case "open" ⇒
|
* case "open" ⇒
|
||||||
* unstashAll()
|
* unstashAll()
|
||||||
* context.become {
|
* context.become({
|
||||||
* case "write" ⇒ // do writing...
|
* case "write" ⇒ // do writing...
|
||||||
* case "close" ⇒
|
* case "close" ⇒
|
||||||
* unstashAll()
|
* unstashAll()
|
||||||
* context.unbecome()
|
* context.unbecome()
|
||||||
* case msg ⇒ stash()
|
* case msg ⇒ stash()
|
||||||
* }
|
* }, discardOld = false)
|
||||||
* case "done" ⇒ // done
|
* case "done" ⇒ // done
|
||||||
* case msg ⇒ stash()
|
* case msg ⇒ stash()
|
||||||
* }
|
* }
|
||||||
|
|
|
||||||
|
|
@ -4,22 +4,25 @@
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
import language.existentials
|
import language.existentials
|
||||||
import akka.japi.{ Creator, Option ⇒ JOption }
|
|
||||||
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
|
|
||||||
import akka.util.Timeout
|
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
import scala.util.{ Try, Success, Failure }
|
||||||
|
import scala.collection.immutable
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
|
import scala.reflect.ClassTag
|
||||||
import scala.concurrent.{ Await, Future }
|
import scala.concurrent.{ Await, Future }
|
||||||
|
import akka.japi.{ Creator, Option ⇒ JOption }
|
||||||
|
import akka.japi.Util.{ immutableSeq, immutableSingletonSeq }
|
||||||
|
import akka.util.Timeout
|
||||||
import akka.util.Reflect.instantiator
|
import akka.util.Reflect.instantiator
|
||||||
|
import akka.serialization.{ JavaSerializer, SerializationExtension }
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
|
import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
|
||||||
import java.util.concurrent.TimeoutException
|
import java.util.concurrent.TimeoutException
|
||||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||||
import scala.reflect.ClassTag
|
|
||||||
import akka.serialization.{ JavaSerializer, SerializationExtension }
|
|
||||||
import java.io.ObjectStreamException
|
import java.io.ObjectStreamException
|
||||||
import scala.util.{ Try, Success, Failure }
|
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
|
||||||
import scala.concurrent.duration.FiniteDuration
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A TypedActorFactory is something that can created TypedActor instances.
|
* A TypedActorFactory is something that can created TypedActor instances.
|
||||||
|
|
@ -439,8 +442,8 @@ object TypedProps {
|
||||||
* @return a sequence of interfaces that the specified class implements,
|
* @return a sequence of interfaces that the specified class implements,
|
||||||
* or a sequence containing only itself, if itself is an interface.
|
* or a sequence containing only itself, if itself is an interface.
|
||||||
*/
|
*/
|
||||||
def extractInterfaces(clazz: Class[_]): Seq[Class[_]] =
|
def extractInterfaces(clazz: Class[_]): immutable.Seq[Class[_]] =
|
||||||
if (clazz.isInterface) Seq[Class[_]](clazz) else clazz.getInterfaces.toList
|
if (clazz.isInterface) immutableSingletonSeq(clazz) else immutableSeq(clazz.getInterfaces)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Uses the supplied class as the factory for the TypedActor implementation,
|
* Uses the supplied class as the factory for the TypedActor implementation,
|
||||||
|
|
@ -489,7 +492,7 @@ object TypedProps {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class TypedProps[T <: AnyRef] protected[TypedProps] (
|
case class TypedProps[T <: AnyRef] protected[TypedProps] (
|
||||||
interfaces: Seq[Class[_]],
|
interfaces: immutable.Seq[Class[_]],
|
||||||
creator: () ⇒ T,
|
creator: () ⇒ T,
|
||||||
dispatcher: String = TypedProps.defaultDispatcherId,
|
dispatcher: String = TypedProps.defaultDispatcherId,
|
||||||
deploy: Deploy = Props.defaultDeploy,
|
deploy: Deploy = Props.defaultDeploy,
|
||||||
|
|
|
||||||
|
|
@ -29,7 +29,9 @@ trait Creators { this: ActorDSL.type ⇒
|
||||||
* for quickly trying things out in the REPL. It makes the following keywords
|
* for quickly trying things out in the REPL. It makes the following keywords
|
||||||
* available:
|
* available:
|
||||||
*
|
*
|
||||||
* - `become` mapped to `context.become(_, discardOld = false)`
|
* - `become` mapped to `context.become(_, discardOld = true)`
|
||||||
|
*
|
||||||
|
* - `becomeStacked` mapped to `context.become(_, discardOld = false)`
|
||||||
*
|
*
|
||||||
* - `unbecome` mapped to `context.unbecome`
|
* - `unbecome` mapped to `context.unbecome`
|
||||||
*
|
*
|
||||||
|
|
@ -87,7 +89,14 @@ trait Creators { this: ActorDSL.type ⇒
|
||||||
* stack is cleared upon restart. Use `unbecome()` to pop an element off
|
* stack is cleared upon restart. Use `unbecome()` to pop an element off
|
||||||
* this stack.
|
* this stack.
|
||||||
*/
|
*/
|
||||||
def become(r: Receive) = context.become(r, discardOld = false)
|
def becomeStacked(r: Receive) = context.become(r, discardOld = false)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Replace the behavior at the top of the behavior stack for this actor. The
|
||||||
|
* stack is cleared upon restart. Use `unbecome()` to pop an element off
|
||||||
|
* this stack or `becomeStacked()` to push a new element on top of it.
|
||||||
|
*/
|
||||||
|
def become(r: Receive) = context.become(r, discardOld = true)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Pop the active behavior from the behavior stack of this actor. This stack
|
* Pop the active behavior from the behavior stack of this actor. This stack
|
||||||
|
|
|
||||||
|
|
@ -5,14 +5,12 @@
|
||||||
package akka.actor.dungeon
|
package akka.actor.dungeon
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.JavaConverters.asJavaIterableConverter
|
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
|
import scala.collection.immutable
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.actor.ActorCell
|
|
||||||
import akka.actor.ActorPath.ElementRegex
|
import akka.actor.ActorPath.ElementRegex
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.util.{ Unsafe, Helpers }
|
import akka.util.{ Unsafe, Helpers }
|
||||||
import akka.actor.ChildNameReserved
|
|
||||||
|
|
||||||
private[akka] trait Children { this: ActorCell ⇒
|
private[akka] trait Children { this: ActorCell ⇒
|
||||||
|
|
||||||
|
|
@ -24,8 +22,9 @@ private[akka] trait Children { this: ActorCell ⇒
|
||||||
def childrenRefs: ChildrenContainer =
|
def childrenRefs: ChildrenContainer =
|
||||||
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer]
|
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer]
|
||||||
|
|
||||||
final def children: Iterable[ActorRef] = childrenRefs.children
|
final def children: immutable.Iterable[ActorRef] = childrenRefs.children
|
||||||
final def getChildren(): java.lang.Iterable[ActorRef] = children.asJava
|
final def getChildren(): java.lang.Iterable[ActorRef] =
|
||||||
|
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
|
||||||
|
|
||||||
final def child(name: String): Option[ActorRef] = Option(getChild(name))
|
final def child(name: String): Option[ActorRef] = Option(getChild(name))
|
||||||
final def getChild(name: String): ActorRef = childrenRefs.getByName(name) match {
|
final def getChild(name: String): ActorRef = childrenRefs.getByName(name) match {
|
||||||
|
|
@ -141,7 +140,7 @@ private[akka] trait Children { this: ActorCell ⇒
|
||||||
|
|
||||||
protected def getChildByRef(ref: ActorRef): Option[ChildRestartStats] = childrenRefs.getByRef(ref)
|
protected def getChildByRef(ref: ActorRef): Option[ChildRestartStats] = childrenRefs.getByRef(ref)
|
||||||
|
|
||||||
protected def getAllChildStats: Iterable[ChildRestartStats] = childrenRefs.stats
|
protected def getAllChildStats: immutable.Iterable[ChildRestartStats] = childrenRefs.stats
|
||||||
|
|
||||||
protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = {
|
protected def removeChildAndGetStateChange(child: ActorRef): Option[SuspendReason] = {
|
||||||
childrenRefs match {
|
childrenRefs match {
|
||||||
|
|
|
||||||
|
|
@ -4,10 +4,11 @@
|
||||||
|
|
||||||
package akka.actor.dungeon
|
package akka.actor.dungeon
|
||||||
|
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable
|
||||||
|
|
||||||
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
||||||
import akka.dispatch.SystemMessage
|
import akka.dispatch.SystemMessage
|
||||||
|
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -20,8 +21,8 @@ private[akka] trait ChildrenContainer {
|
||||||
def getByName(name: String): Option[ChildStats]
|
def getByName(name: String): Option[ChildStats]
|
||||||
def getByRef(actor: ActorRef): Option[ChildRestartStats]
|
def getByRef(actor: ActorRef): Option[ChildRestartStats]
|
||||||
|
|
||||||
def children: Iterable[ActorRef]
|
def children: immutable.Iterable[ActorRef]
|
||||||
def stats: Iterable[ChildRestartStats]
|
def stats: immutable.Iterable[ChildRestartStats]
|
||||||
|
|
||||||
def shallDie(actor: ActorRef): ChildrenContainer
|
def shallDie(actor: ActorRef): ChildrenContainer
|
||||||
|
|
||||||
|
|
@ -49,6 +50,18 @@ private[akka] object ChildrenContainer {
|
||||||
case class Creation() extends SuspendReason with WaitingForChildren
|
case class Creation() extends SuspendReason with WaitingForChildren
|
||||||
case object Termination extends SuspendReason
|
case object Termination extends SuspendReason
|
||||||
|
|
||||||
|
class ChildRestartsIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] {
|
||||||
|
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats]
|
||||||
|
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||||
|
override final def valuesIterator = stats.valuesIterator
|
||||||
|
}
|
||||||
|
|
||||||
|
class ChildrenIterable(stats: immutable.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] {
|
||||||
|
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child
|
||||||
|
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||||
|
override final def valuesIterator = stats.valuesIterator
|
||||||
|
}
|
||||||
|
|
||||||
trait WaitingForChildren {
|
trait WaitingForChildren {
|
||||||
private var todo: SystemMessage = null
|
private var todo: SystemMessage = null
|
||||||
def enqueue(message: SystemMessage) = { message.next = todo; todo = message }
|
def enqueue(message: SystemMessage) = { message.next = todo; todo = message }
|
||||||
|
|
@ -56,13 +69,13 @@ private[akka] object ChildrenContainer {
|
||||||
}
|
}
|
||||||
|
|
||||||
trait EmptyChildrenContainer extends ChildrenContainer {
|
trait EmptyChildrenContainer extends ChildrenContainer {
|
||||||
val emptyStats = TreeMap.empty[String, ChildStats]
|
val emptyStats = immutable.TreeMap.empty[String, ChildStats]
|
||||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, stats))
|
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, stats))
|
||||||
override def remove(child: ActorRef): ChildrenContainer = this
|
override def remove(child: ActorRef): ChildrenContainer = this
|
||||||
override def getByName(name: String): Option[ChildRestartStats] = None
|
override def getByName(name: String): Option[ChildRestartStats] = None
|
||||||
override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
|
override def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
|
||||||
override def children: Iterable[ActorRef] = Nil
|
override def children: immutable.Iterable[ActorRef] = EmptyImmutableSeq
|
||||||
override def stats: Iterable[ChildRestartStats] = Nil
|
override def stats: immutable.Iterable[ChildRestartStats] = EmptyImmutableSeq
|
||||||
override def shallDie(actor: ActorRef): ChildrenContainer = this
|
override def shallDie(actor: ActorRef): ChildrenContainer = this
|
||||||
override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
|
override def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
|
||||||
override def unreserve(name: String): ChildrenContainer = this
|
override def unreserve(name: String): ChildrenContainer = this
|
||||||
|
|
@ -95,7 +108,7 @@ private[akka] object ChildrenContainer {
|
||||||
* calling context.stop(child) and processing the ChildTerminated() system
|
* calling context.stop(child) and processing the ChildTerminated() system
|
||||||
* message).
|
* message).
|
||||||
*/
|
*/
|
||||||
class NormalChildrenContainer(val c: TreeMap[String, ChildStats]) extends ChildrenContainer {
|
class NormalChildrenContainer(val c: immutable.TreeMap[String, ChildStats]) extends ChildrenContainer {
|
||||||
|
|
||||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(c.updated(name, stats))
|
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = new NormalChildrenContainer(c.updated(name, stats))
|
||||||
|
|
||||||
|
|
@ -108,9 +121,11 @@ private[akka] object ChildrenContainer {
|
||||||
case _ ⇒ None
|
case _ ⇒ None
|
||||||
}
|
}
|
||||||
|
|
||||||
override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child }
|
override def children: immutable.Iterable[ActorRef] =
|
||||||
|
if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c)
|
||||||
|
|
||||||
override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c }
|
override def stats: immutable.Iterable[ChildRestartStats] =
|
||||||
|
if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c)
|
||||||
|
|
||||||
override def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
|
override def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
|
||||||
|
|
||||||
|
|
@ -130,7 +145,7 @@ private[akka] object ChildrenContainer {
|
||||||
}
|
}
|
||||||
|
|
||||||
object NormalChildrenContainer {
|
object NormalChildrenContainer {
|
||||||
def apply(c: TreeMap[String, ChildStats]): ChildrenContainer =
|
def apply(c: immutable.TreeMap[String, ChildStats]): ChildrenContainer =
|
||||||
if (c.isEmpty) EmptyChildrenContainer
|
if (c.isEmpty) EmptyChildrenContainer
|
||||||
else new NormalChildrenContainer(c)
|
else new NormalChildrenContainer(c)
|
||||||
}
|
}
|
||||||
|
|
@ -145,7 +160,7 @@ private[akka] object ChildrenContainer {
|
||||||
* type of container, depending on whether or not children are left and whether or not
|
* type of container, depending on whether or not children are left and whether or not
|
||||||
* the reason was “Terminating”.
|
* the reason was “Terminating”.
|
||||||
*/
|
*/
|
||||||
case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
|
case class TerminatingChildrenContainer(c: immutable.TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
|
||||||
extends ChildrenContainer {
|
extends ChildrenContainer {
|
||||||
|
|
||||||
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats))
|
override def add(name: String, stats: ChildRestartStats): ChildrenContainer = copy(c.updated(name, stats))
|
||||||
|
|
@ -166,9 +181,11 @@ private[akka] object ChildrenContainer {
|
||||||
case _ ⇒ None
|
case _ ⇒ None
|
||||||
}
|
}
|
||||||
|
|
||||||
override def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) ⇒ child }
|
override def children: immutable.Iterable[ActorRef] =
|
||||||
|
if (c.isEmpty) EmptyImmutableSeq else new ChildrenIterable(c)
|
||||||
|
|
||||||
override def stats: Iterable[ChildRestartStats] = c.values.view.collect { case c: ChildRestartStats ⇒ c }
|
override def stats: immutable.Iterable[ChildRestartStats] =
|
||||||
|
if (c.isEmpty) EmptyImmutableSeq else new ChildRestartsIterable(c)
|
||||||
|
|
||||||
override def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
|
override def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,7 +10,7 @@ import akka.dispatch._
|
||||||
import akka.event.Logging.{ Warning, Error, Debug }
|
import akka.event.Logging.{ Warning, Error, Debug }
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import scala.Some
|
import scala.collection.immutable
|
||||||
import akka.dispatch.ChildTerminated
|
import akka.dispatch.ChildTerminated
|
||||||
import akka.actor.PreRestartException
|
import akka.actor.PreRestartException
|
||||||
import akka.actor.Failed
|
import akka.actor.Failed
|
||||||
|
|
@ -160,7 +160,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final def handleInvokeFailure(childrenNotToSuspend: Iterable[ActorRef], t: Throwable, message: String): Unit = {
|
final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable, message: String): Unit = {
|
||||||
publish(Error(t, self.path.toString, clazz(actor), message))
|
publish(Error(t, self.path.toString, clazz(actor), message))
|
||||||
// prevent any further messages to be processed until the actor has been restarted
|
// prevent any further messages to be processed until the actor has been restarted
|
||||||
if (!isFailed) try {
|
if (!isFailed) try {
|
||||||
|
|
|
||||||
|
|
@ -420,7 +420,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
||||||
case "unbounded" ⇒ UnboundedMailbox()
|
case "unbounded" ⇒ UnboundedMailbox()
|
||||||
case "bounded" ⇒ new BoundedMailbox(prerequisites.settings, config)
|
case "bounded" ⇒ new BoundedMailbox(prerequisites.settings, config)
|
||||||
case fqcn ⇒
|
case fqcn ⇒
|
||||||
val args = Seq(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config)
|
val args = List(classOf[ActorSystem.Settings] -> prerequisites.settings, classOf[Config] -> config)
|
||||||
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({
|
prerequisites.dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({
|
||||||
case exception ⇒
|
case exception ⇒
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
|
|
@ -436,7 +436,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit
|
||||||
case null | "" | "fork-join-executor" ⇒ new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites)
|
case null | "" | "fork-join-executor" ⇒ new ForkJoinExecutorConfigurator(config.getConfig("fork-join-executor"), prerequisites)
|
||||||
case "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
|
case "thread-pool-executor" ⇒ new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
|
||||||
case fqcn ⇒
|
case fqcn ⇒
|
||||||
val args = Seq(
|
val args = List(
|
||||||
classOf[Config] -> config,
|
classOf[Config] -> config,
|
||||||
classOf[DispatcherPrerequisites] -> prerequisites)
|
classOf[DispatcherPrerequisites] -> prerequisites)
|
||||||
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({
|
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({
|
||||||
|
|
|
||||||
|
|
@ -147,7 +147,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
||||||
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
||||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||||
case fqn ⇒
|
case fqn ⇒
|
||||||
val args = Seq(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
val args = List(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites)
|
||||||
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({
|
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({
|
||||||
case exception ⇒
|
case exception ⇒
|
||||||
throw new IllegalArgumentException(
|
throw new IllegalArgumentException(
|
||||||
|
|
|
||||||
|
|
@ -95,7 +95,7 @@ object Futures {
|
||||||
*/
|
*/
|
||||||
def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = {
|
def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = {
|
||||||
implicit val ec = executor
|
implicit val ec = executor
|
||||||
Future.find[T](futures.asScala)(predicate.apply(_))(executor).map(JOption.fromScalaOption(_))
|
Future.find[T](futures.asScala)(predicate.apply(_))(executor) map JOption.fromScalaOption
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,7 @@ import java.util.concurrent.ConcurrentSkipListSet
|
||||||
import java.util.Comparator
|
import java.util.Comparator
|
||||||
import akka.util.{ Subclassification, SubclassifiedIndex }
|
import akka.util.{ Subclassification, SubclassifiedIndex }
|
||||||
import scala.collection.immutable.TreeSet
|
import scala.collection.immutable.TreeSet
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents the base type for EventBuses
|
* Represents the base type for EventBuses
|
||||||
|
|
@ -167,12 +168,12 @@ trait SubchannelClassification { this: EventBus ⇒
|
||||||
recv foreach (publish(event, _))
|
recv foreach (publish(event, _))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def removeFromCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit =
|
private def removeFromCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||||
cache = (cache /: changes) {
|
cache = (cache /: changes) {
|
||||||
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) -- cs)
|
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) -- cs)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def addToCache(changes: Seq[(Classifier, Set[Subscriber])]): Unit =
|
private def addToCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit =
|
||||||
cache = (cache /: changes) {
|
cache = (cache /: changes) {
|
||||||
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs)
|
case (m, (c, cs)) ⇒ m.updated(c, m.getOrElse(c, Set.empty[Subscriber]) ++ cs)
|
||||||
}
|
}
|
||||||
|
|
@ -265,9 +266,9 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected final def dissociate(monitored: ActorRef): Iterable[ActorRef] = {
|
protected final def dissociate(monitored: ActorRef): immutable.Iterable[ActorRef] = {
|
||||||
@tailrec
|
@tailrec
|
||||||
def dissociateAsMonitored(monitored: ActorRef): Iterable[ActorRef] = {
|
def dissociateAsMonitored(monitored: ActorRef): immutable.Iterable[ActorRef] = {
|
||||||
val current = mappings get monitored
|
val current = mappings get monitored
|
||||||
current match {
|
current match {
|
||||||
case null ⇒ empty
|
case null ⇒ empty
|
||||||
|
|
|
||||||
|
|
@ -9,12 +9,13 @@ import akka.actor._
|
||||||
import akka.{ ConfigurationException, AkkaException }
|
import akka.{ ConfigurationException, AkkaException }
|
||||||
import akka.actor.ActorSystem.Settings
|
import akka.actor.ActorSystem.Settings
|
||||||
import akka.util.{ Timeout, ReentrantGuard }
|
import akka.util.{ Timeout, ReentrantGuard }
|
||||||
import scala.concurrent.duration._
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
import scala.util.control.NoStackTrace
|
|
||||||
import java.util.concurrent.TimeoutException
|
import java.util.concurrent.TimeoutException
|
||||||
|
import scala.annotation.implicitNotFound
|
||||||
|
import scala.collection.immutable
|
||||||
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import annotation.implicitNotFound
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This trait brings log level handling to the EventStream: it reads the log
|
* This trait brings log level handling to the EventStream: it reads the log
|
||||||
|
|
@ -448,7 +449,7 @@ object Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
// these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type
|
// these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type
|
||||||
val AllLogLevels: Seq[LogLevel] = Seq(ErrorLevel, WarningLevel, InfoLevel, DebugLevel)
|
val AllLogLevels: immutable.Seq[LogLevel] = Vector(ErrorLevel, WarningLevel, InfoLevel, DebugLevel)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Obtain LoggingAdapter for the given actor system and source object. This
|
* Obtain LoggingAdapter for the given actor system and source object. This
|
||||||
|
|
@ -877,15 +878,25 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class
|
||||||
protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
|
protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
|
||||||
}
|
}
|
||||||
|
|
||||||
private[akka] object NoLogging extends LoggingAdapter {
|
/**
|
||||||
def isErrorEnabled = false
|
* NoLogging is a LoggingAdapter that does absolutely nothing – no logging at all.
|
||||||
def isWarningEnabled = false
|
*/
|
||||||
def isInfoEnabled = false
|
object NoLogging extends LoggingAdapter {
|
||||||
def isDebugEnabled = false
|
|
||||||
|
|
||||||
protected def notifyError(message: String): Unit = ()
|
/**
|
||||||
protected def notifyError(cause: Throwable, message: String): Unit = ()
|
* Java API to return the reference to NoLogging
|
||||||
protected def notifyWarning(message: String): Unit = ()
|
* @return The NoLogging instance
|
||||||
protected def notifyInfo(message: String): Unit = ()
|
*/
|
||||||
protected def notifyDebug(message: String): Unit = ()
|
def getInstance = this
|
||||||
|
|
||||||
|
final override def isErrorEnabled = false
|
||||||
|
final override def isWarningEnabled = false
|
||||||
|
final override def isInfoEnabled = false
|
||||||
|
final override def isDebugEnabled = false
|
||||||
|
|
||||||
|
final protected override def notifyError(message: String): Unit = ()
|
||||||
|
final protected override def notifyError(cause: Throwable, message: String): Unit = ()
|
||||||
|
final protected override def notifyWarning(message: String): Unit = ()
|
||||||
|
final protected override def notifyInfo(message: String): Unit = ()
|
||||||
|
final protected override def notifyDebug(message: String): Unit = ()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,10 +5,12 @@
|
||||||
package akka.japi
|
package akka.japi
|
||||||
|
|
||||||
import language.implicitConversions
|
import language.implicitConversions
|
||||||
import scala.Some
|
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
import scala.runtime.AbstractPartialFunction
|
import scala.runtime.AbstractPartialFunction
|
||||||
|
import akka.util.Collections.EmptyImmutableSeq
|
||||||
import java.util.Collections.{ emptyList, singletonList }
|
import java.util.Collections.{ emptyList, singletonList }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -174,9 +176,40 @@ object Option {
|
||||||
* This class hold common utilities for Java
|
* This class hold common utilities for Java
|
||||||
*/
|
*/
|
||||||
object Util {
|
object Util {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a ClassTag describing the provided Class.
|
||||||
|
*
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
|
def classTag[T](clazz: Class[T]): ClassTag[T] = ClassTag(clazz)
|
||||||
|
|
||||||
def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq
|
/**
|
||||||
|
* Returns an immutable.Seq representing the provided array of Classes,
|
||||||
|
* an overloading of the generic immutableSeq in Util, to accommodate for erasure.
|
||||||
|
*
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
|
def immutableSeq(arr: Array[Class[_]]): immutable.Seq[Class[_]] = immutableSeq[Class[_]](arr)
|
||||||
|
|
||||||
def arrayToSeq(classes: Array[Class[_]]): Seq[Class[_]] = classes.toSeq
|
/**
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
def immutableSeq[T](arr: Array[T]): immutable.Seq[T] = if ((arr ne null) && arr.length > 0) Vector(arr: _*) else Nil
|
||||||
|
|
||||||
|
def immutableSeq[T](iterable: java.lang.Iterable[T]): immutable.Seq[T] =
|
||||||
|
iterable match {
|
||||||
|
case imm: immutable.Seq[_] ⇒ imm.asInstanceOf[immutable.Seq[T]]
|
||||||
|
case other ⇒
|
||||||
|
val i = other.iterator()
|
||||||
|
if (i.hasNext) {
|
||||||
|
val builder = new immutable.VectorBuilder[T]
|
||||||
|
|
||||||
|
do { builder += i.next() } while (i.hasNext)
|
||||||
|
|
||||||
|
builder.result()
|
||||||
|
} else EmptyImmutableSeq
|
||||||
|
}
|
||||||
|
|
||||||
|
def immutableSingletonSeq[T](value: T): immutable.Seq[T] = value :: Nil
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
package akka.routing
|
package akka.routing
|
||||||
|
|
||||||
import scala.collection.immutable.SortedMap
|
import scala.collection.immutable
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import java.util.Arrays
|
import java.util.Arrays
|
||||||
|
|
||||||
|
|
@ -18,7 +18,7 @@ import java.util.Arrays
|
||||||
* hash, i.e. make sure it is different for different nodes.
|
* hash, i.e. make sure it is different for different nodes.
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtualNodesFactor: Int) {
|
class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], val virtualNodesFactor: Int) {
|
||||||
|
|
||||||
import ConsistentHash._
|
import ConsistentHash._
|
||||||
|
|
||||||
|
|
@ -106,7 +106,7 @@ class ConsistentHash[T: ClassTag] private (nodes: SortedMap[Int, T], val virtual
|
||||||
|
|
||||||
object ConsistentHash {
|
object ConsistentHash {
|
||||||
def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = {
|
def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = {
|
||||||
new ConsistentHash(SortedMap.empty[Int, T] ++
|
new ConsistentHash(immutable.SortedMap.empty[Int, T] ++
|
||||||
(for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)),
|
(for (node ← nodes; vnode ← 1 to virtualNodesFactor) yield (nodeHashFor(node, vnode) -> node)),
|
||||||
virtualNodesFactor)
|
virtualNodesFactor)
|
||||||
}
|
}
|
||||||
|
|
@ -120,8 +120,10 @@ object ConsistentHash {
|
||||||
apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]]))
|
apply(nodes.asScala, virtualNodesFactor)(ClassTag(classOf[Any].asInstanceOf[Class[T]]))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def nodeHashFor(node: Any, vnode: Int): Int =
|
private def nodeHashFor(node: Any, vnode: Int): Int = {
|
||||||
hashFor((node + ":" + vnode).getBytes("UTF-8"))
|
val baseStr = node.toString + ":"
|
||||||
|
hashFor(baseStr + vnode)
|
||||||
|
}
|
||||||
|
|
||||||
private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes)
|
private def hashFor(bytes: Array[Byte]): Int = MurmurHash.arrayHash(bytes)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,30 +3,29 @@
|
||||||
*/
|
*/
|
||||||
package akka.routing
|
package akka.routing
|
||||||
|
|
||||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
import scala.collection.immutable
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.SupervisorStrategy
|
import akka.actor.SupervisorStrategy
|
||||||
import akka.actor.Props
|
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
|
import akka.actor.Address
|
||||||
|
import akka.actor.ExtendedActorSystem
|
||||||
|
|
||||||
object ConsistentHashingRouter {
|
object ConsistentHashingRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new ConsistentHashingRouter, routing to the specified routees
|
* Creates a new ConsistentHashingRouter, routing to the specified routees
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef]): ConsistentHashingRouter =
|
def apply(routees: immutable.Iterable[ActorRef]): ConsistentHashingRouter =
|
||||||
new ConsistentHashingRouter(routees = routees map (_.path.toString))
|
new ConsistentHashingRouter(routees = routees map (_.path.toString))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = {
|
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees))
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
apply(routees.asScala)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If you don't define the `hashMapping` when
|
* If you don't define the `hashMapping` when
|
||||||
|
|
@ -144,7 +143,7 @@ object ConsistentHashingRouter {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class ConsistentHashingRouter(
|
case class ConsistentHashingRouter(
|
||||||
nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy,
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy,
|
||||||
val virtualNodesFactor: Int = 0,
|
val virtualNodesFactor: Int = 0,
|
||||||
|
|
@ -163,7 +162,7 @@ case class ConsistentHashingRouter(
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -225,7 +224,7 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
|
||||||
|
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
def virtualNodesFactor: Int
|
def virtualNodesFactor: Int
|
||||||
|
|
||||||
|
|
@ -238,20 +237,22 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
|
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
|
||||||
|
val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.rootPath.address
|
||||||
val vnodes =
|
val vnodes =
|
||||||
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
|
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
|
||||||
else virtualNodesFactor
|
else virtualNodesFactor
|
||||||
|
|
||||||
// tuple of routees and the ConsistentHash, updated together in updateConsistentHash
|
// tuple of routees and the ConsistentHash, updated together in updateConsistentHash
|
||||||
val consistentHashRef = new AtomicReference[(IndexedSeq[ActorRef], ConsistentHash[ActorRef])]((null, null))
|
val consistentHashRef = new AtomicReference[(IndexedSeq[ConsistentActorRef], ConsistentHash[ConsistentActorRef])]((null, null))
|
||||||
updateConsistentHash()
|
updateConsistentHash()
|
||||||
|
|
||||||
// update consistentHash when routees has changed
|
// update consistentHash when routees has changed
|
||||||
// changes to routees are rare and when no changes this is a quick operation
|
// changes to routees are rare and when no changes this is a quick operation
|
||||||
def updateConsistentHash(): ConsistentHash[ActorRef] = {
|
def updateConsistentHash(): ConsistentHash[ConsistentActorRef] = {
|
||||||
val oldConsistentHashTuple = consistentHashRef.get
|
val oldConsistentHashTuple = consistentHashRef.get
|
||||||
val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple
|
val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple
|
||||||
val currentRoutees = routeeProvider.routees
|
val currentRoutees = routeeProvider.routees map { ConsistentActorRef(_, selfAddress) }
|
||||||
|
|
||||||
if (currentRoutees ne oldConsistentHashRoutees) {
|
if (currentRoutees ne oldConsistentHashRoutees) {
|
||||||
// when other instance, same content, no need to re-hash, but try to set routees
|
// when other instance, same content, no need to re-hash, but try to set routees
|
||||||
val consistentHash =
|
val consistentHash =
|
||||||
|
|
@ -267,9 +268,9 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
|
||||||
val currentConsistenHash = updateConsistentHash()
|
val currentConsistenHash = updateConsistentHash()
|
||||||
if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters
|
if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters
|
||||||
else hashData match {
|
else hashData match {
|
||||||
case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes)
|
case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes).actorRef
|
||||||
case str: String ⇒ currentConsistenHash.nodeFor(str)
|
case str: String ⇒ currentConsistenHash.nodeFor(str).actorRef
|
||||||
case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get)
|
case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get).actorRef
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case NonFatal(e) ⇒
|
case NonFatal(e) ⇒
|
||||||
|
|
@ -295,3 +296,20 @@ trait ConsistentHashingLike { this: RouterConfig ⇒
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
* Important to use ActorRef with full address, with host and port, in the hash ring,
|
||||||
|
* so that same ring is produced on different nodes.
|
||||||
|
* The ConsistentHash uses toString of the ring nodes, and the ActorRef itself
|
||||||
|
* isn't a good representation, because LocalActorRef doesn't include the
|
||||||
|
* host and port.
|
||||||
|
*/
|
||||||
|
private[akka] case class ConsistentActorRef(actorRef: ActorRef, selfAddress: Address) {
|
||||||
|
override def toString: String = {
|
||||||
|
actorRef.path.address match {
|
||||||
|
case Address(_, _, None, None) ⇒ actorRef.path.toStringWithAddress(selfAddress)
|
||||||
|
case a ⇒ actorRef.path.toString
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -5,18 +5,20 @@ package akka.routing
|
||||||
|
|
||||||
import language.implicitConversions
|
import language.implicitConversions
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
import akka.actor._
|
|
||||||
import scala.concurrent.duration._
|
import scala.collection.immutable
|
||||||
import akka.ConfigurationException
|
|
||||||
import akka.pattern.pipe
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
import scala.collection.JavaConverters.iterableAsScalaIterableConverter
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import akka.actor._
|
||||||
|
import akka.ConfigurationException
|
||||||
|
import akka.dispatch.Dispatchers
|
||||||
|
import akka.pattern.pipe
|
||||||
|
import akka.japi.Util.immutableSeq
|
||||||
|
import com.typesafe.config.Config
|
||||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||||
import akka.dispatch.Dispatchers
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import concurrent.ExecutionContext
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
|
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
|
||||||
|
|
@ -50,7 +52,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
||||||
private val resizeCounter = new AtomicLong
|
private val resizeCounter = new AtomicLong
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
private var _routees: immutable.IndexedSeq[ActorRef] = immutable.IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
|
||||||
def routees = _routees
|
def routees = _routees
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
|
|
@ -75,14 +77,11 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
||||||
* end of construction
|
* end of construction
|
||||||
*/
|
*/
|
||||||
|
|
||||||
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
|
def applyRoute(sender: ActorRef, message: Any): immutable.Iterable[Destination] = message match {
|
||||||
case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil
|
case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil
|
||||||
case CurrentRoutees ⇒
|
case CurrentRoutees ⇒ sender ! RouterRoutees(_routees); Nil
|
||||||
sender ! RouterRoutees(_routees)
|
case msg if route.isDefinedAt(sender, msg) ⇒ route(sender, message)
|
||||||
Nil
|
case _ ⇒ Nil
|
||||||
case _ ⇒
|
|
||||||
if (route.isDefinedAt(sender, message)) route(sender, message)
|
|
||||||
else Nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -91,7 +90,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `RouterConfig.createRoute` and `Resizer.resize`
|
* `RouterConfig.createRoute` and `Resizer.resize`
|
||||||
*/
|
*/
|
||||||
private[akka] def addRoutees(newRoutees: Iterable[ActorRef]): Unit = {
|
private[akka] def addRoutees(newRoutees: immutable.Iterable[ActorRef]): Unit = {
|
||||||
_routees = _routees ++ newRoutees
|
_routees = _routees ++ newRoutees
|
||||||
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
||||||
newRoutees foreach watch
|
newRoutees foreach watch
|
||||||
|
|
@ -103,7 +102,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `Resizer.resize`
|
* `Resizer.resize`
|
||||||
*/
|
*/
|
||||||
private[akka] def removeRoutees(abandonedRoutees: Iterable[ActorRef]): Unit = {
|
private[akka] def removeRoutees(abandonedRoutees: immutable.Iterable[ActorRef]): Unit = {
|
||||||
_routees = abandonedRoutees.foldLeft(_routees) { (xs, x) ⇒ unwatch(x); xs.filterNot(_ == x) }
|
_routees = abandonedRoutees.foldLeft(_routees) { (xs, x) ⇒ unwatch(x); xs.filterNot(_ == x) }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -195,7 +194,7 @@ trait RouterConfig {
|
||||||
*/
|
*/
|
||||||
def withFallback(other: RouterConfig): RouterConfig = this
|
def withFallback(other: RouterConfig): RouterConfig = this
|
||||||
|
|
||||||
protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] =
|
protected def toAll(sender: ActorRef, routees: immutable.Iterable[ActorRef]): immutable.Iterable[Destination] =
|
||||||
routees.map(Destination(sender, _))
|
routees.map(Destination(sender, _))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -207,7 +206,7 @@ trait RouterConfig {
|
||||||
/**
|
/**
|
||||||
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
|
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
|
||||||
*/
|
*/
|
||||||
def verifyConfig(): Unit = {}
|
def verifyConfig(): Unit = ()
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -226,7 +225,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||||
*/
|
*/
|
||||||
def registerRoutees(routees: Iterable[ActorRef]): Unit = routedCell.addRoutees(routees)
|
def registerRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.addRoutees(routees)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds the routees to the router.
|
* Adds the routees to the router.
|
||||||
|
|
@ -235,7 +234,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||||
* Java API.
|
* Java API.
|
||||||
*/
|
*/
|
||||||
def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(routees.asScala)
|
def registerRoutees(routees: java.lang.Iterable[ActorRef]): Unit = registerRoutees(immutableSeq(routees))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Removes routees from the router. This method doesn't stop the routees.
|
* Removes routees from the router. This method doesn't stop the routees.
|
||||||
|
|
@ -243,7 +242,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `Resizer.resize`.
|
* `Resizer.resize`.
|
||||||
*/
|
*/
|
||||||
def unregisterRoutees(routees: Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees)
|
def unregisterRoutees(routees: immutable.Iterable[ActorRef]): Unit = routedCell.removeRoutees(routees)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Removes routees from the router. This method doesn't stop the routees.
|
* Removes routees from the router. This method doesn't stop the routees.
|
||||||
|
|
@ -252,28 +251,25 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
* `Resizer.resize`.
|
* `Resizer.resize`.
|
||||||
* JAVA API
|
* JAVA API
|
||||||
*/
|
*/
|
||||||
def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(routees.asScala)
|
def unregisterRoutees(routees: java.lang.Iterable[ActorRef]): Unit = unregisterRoutees(immutableSeq(routees))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Looks up routes with specified paths and registers them.
|
* Looks up routes with specified paths and registers them.
|
||||||
*/
|
*/
|
||||||
def registerRouteesFor(paths: Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_)))
|
def registerRouteesFor(paths: immutable.Iterable[String]): Unit = registerRoutees(paths.map(context.actorFor(_)))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Looks up routes with specified paths and registers them.
|
* Looks up routes with specified paths and registers them.
|
||||||
* JAVA API
|
* JAVA API
|
||||||
*/
|
*/
|
||||||
def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(paths.asScala)
|
def registerRouteesFor(paths: java.lang.Iterable[String]): Unit = registerRouteesFor(immutableSeq(paths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates new routees from specified `Props` and registers them.
|
* Creates new routees from specified `Props` and registers them.
|
||||||
*/
|
*/
|
||||||
def createRoutees(nrOfInstances: Int): Unit = {
|
def createRoutees(nrOfInstances: Int): Unit =
|
||||||
if (nrOfInstances <= 0) throw new IllegalArgumentException(
|
if (nrOfInstances <= 0) throw new IllegalArgumentException("Must specify nrOfInstances or routees for [%s]" format context.self.path.toString)
|
||||||
"Must specify nrOfInstances or routees for [%s]" format context.self.path.toString)
|
else registerRoutees(immutable.IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
|
||||||
else
|
|
||||||
registerRoutees(IndexedSeq.fill(nrOfInstances)(context.actorOf(routeeProps)))
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Remove specified number of routees by unregister them
|
* Remove specified number of routees by unregister them
|
||||||
|
|
@ -296,7 +292,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
* Give concurrent messages a chance to be placed in mailbox before
|
* Give concurrent messages a chance to be placed in mailbox before
|
||||||
* sending PoisonPill.
|
* sending PoisonPill.
|
||||||
*/
|
*/
|
||||||
protected def delayedStop(scheduler: Scheduler, abandon: Iterable[ActorRef], stopDelay: FiniteDuration): Unit = {
|
protected def delayedStop(scheduler: Scheduler, abandon: immutable.Iterable[ActorRef], stopDelay: FiniteDuration): Unit = {
|
||||||
if (abandon.nonEmpty) {
|
if (abandon.nonEmpty) {
|
||||||
if (stopDelay <= Duration.Zero) {
|
if (stopDelay <= Duration.Zero) {
|
||||||
abandon foreach (_ ! PoisonPill)
|
abandon foreach (_ ! PoisonPill)
|
||||||
|
|
@ -314,7 +310,7 @@ class RouteeProvider(val context: ActorContext, val routeeProps: Props, val resi
|
||||||
/**
|
/**
|
||||||
* All routees of the router
|
* All routees of the router
|
||||||
*/
|
*/
|
||||||
def routees: IndexedSeq[ActorRef] = routedCell.routees
|
def routees: immutable.IndexedSeq[ActorRef] = routedCell.routees
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* All routees of the router
|
* All routees of the router
|
||||||
|
|
@ -335,7 +331,7 @@ abstract class CustomRouterConfig extends RouterConfig {
|
||||||
val customRoute = createCustomRoute(routeeProvider)
|
val customRoute = createCustomRoute(routeeProvider)
|
||||||
|
|
||||||
{
|
{
|
||||||
case (sender, message) ⇒ customRoute.destinationsFor(sender, message).asScala
|
case (sender, message) ⇒ customRoute.destinationsFor(sender, message)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -344,7 +340,13 @@ abstract class CustomRouterConfig extends RouterConfig {
|
||||||
}
|
}
|
||||||
|
|
||||||
trait CustomRoute {
|
trait CustomRoute {
|
||||||
def destinationsFor(sender: ActorRef, message: Any): java.lang.Iterable[Destination]
|
/**
|
||||||
|
* use akka.japi.Util.immutableSeq to convert a java.lang.Iterable to the return type needed for destinationsFor,
|
||||||
|
* or if you just want to return a single Destination, use akka.japi.Util.immutableSingletonSeq
|
||||||
|
*
|
||||||
|
* Java API
|
||||||
|
*/
|
||||||
|
def destinationsFor(sender: ActorRef, message: Any): immutable.Seq[Destination]
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -366,7 +368,7 @@ trait Router extends Actor {
|
||||||
if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProvider)) finally ab.set(false)
|
if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProvider)) finally ab.set(false)
|
||||||
|
|
||||||
case Terminated(child) ⇒
|
case Terminated(child) ⇒
|
||||||
ref.removeRoutees(IndexedSeq(child))
|
ref.removeRoutees(child :: Nil)
|
||||||
if (ref.routees.isEmpty) context.stop(self)
|
if (ref.routees.isEmpty) context.stop(self)
|
||||||
|
|
||||||
}: Receive) orElse routerReceive
|
}: Receive) orElse routerReceive
|
||||||
|
|
@ -426,7 +428,7 @@ case object CurrentRoutees extends CurrentRoutees {
|
||||||
* Message used to carry information about what routees the router is currently using.
|
* Message used to carry information about what routees the router is currently using.
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class RouterRoutees(routees: Iterable[ActorRef])
|
case class RouterRoutees(routees: immutable.Iterable[ActorRef])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* For every message sent to a router, its route determines a set of destinations,
|
* For every message sent to a router, its route determines a set of destinations,
|
||||||
|
|
@ -494,16 +496,14 @@ object RoundRobinRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new RoundRobinRouter, routing to the specified routees
|
* Creates a new RoundRobinRouter, routing to the specified routees
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef]): RoundRobinRouter =
|
def apply(routees: immutable.Iterable[ActorRef]): RoundRobinRouter =
|
||||||
new RoundRobinRouter(routees = routees map (_.path.toString))
|
new RoundRobinRouter(routees = routees map (_.path.toString))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = {
|
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter =
|
||||||
import scala.collection.JavaConverters._
|
apply(immutableSeq(routees))
|
||||||
apply(routees.asScala)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
|
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
|
||||||
|
|
@ -547,7 +547,7 @@ object RoundRobinRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||||
extends RouterConfig with RoundRobinLike {
|
extends RouterConfig with RoundRobinLike {
|
||||||
|
|
@ -564,7 +564,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -602,7 +602,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
|
||||||
|
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||||
if (resizer.isEmpty) {
|
if (resizer.isEmpty) {
|
||||||
|
|
@ -622,7 +622,7 @@ trait RoundRobinLike { this: RouterConfig ⇒
|
||||||
case (sender, message) ⇒
|
case (sender, message) ⇒
|
||||||
message match {
|
message match {
|
||||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||||
case msg ⇒ List(Destination(sender, getNext()))
|
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -632,15 +632,13 @@ object RandomRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new RandomRouter, routing to the specified routees
|
* Creates a new RandomRouter, routing to the specified routees
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
def apply(routees: immutable.Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = {
|
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter =
|
||||||
import scala.collection.JavaConverters._
|
apply(immutableSeq(routees))
|
||||||
apply(routees.asScala)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* A Router that randomly selects one of the target connections to send a message to.
|
* A Router that randomly selects one of the target connections to send a message to.
|
||||||
|
|
@ -684,7 +682,7 @@ object RandomRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
case class RandomRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||||
extends RouterConfig with RandomLike {
|
extends RouterConfig with RandomLike {
|
||||||
|
|
@ -701,7 +699,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -738,7 +736,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
||||||
trait RandomLike { this: RouterConfig ⇒
|
trait RandomLike { this: RouterConfig ⇒
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||||
if (resizer.isEmpty) {
|
if (resizer.isEmpty) {
|
||||||
|
|
@ -756,7 +754,7 @@ trait RandomLike { this: RouterConfig ⇒
|
||||||
case (sender, message) ⇒
|
case (sender, message) ⇒
|
||||||
message match {
|
message match {
|
||||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||||
case msg ⇒ List(Destination(sender, getNext()))
|
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -766,16 +764,14 @@ object SmallestMailboxRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new SmallestMailboxRouter, routing to the specified routees
|
* Creates a new SmallestMailboxRouter, routing to the specified routees
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef]): SmallestMailboxRouter =
|
def apply(routees: immutable.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||||
new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter = {
|
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||||
import scala.collection.JavaConverters._
|
apply(immutableSeq(routees))
|
||||||
apply(routees.asScala)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
* A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||||
|
|
@ -828,7 +824,7 @@ object SmallestMailboxRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||||
extends RouterConfig with SmallestMailboxLike {
|
extends RouterConfig with SmallestMailboxLike {
|
||||||
|
|
@ -845,7 +841,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -882,7 +878,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
||||||
trait SmallestMailboxLike { this: RouterConfig ⇒
|
trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns true if the actor is currently processing a message.
|
* Returns true if the actor is currently processing a message.
|
||||||
|
|
@ -954,7 +950,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
// 4. An ActorRef with unknown mailbox size that isn't processing anything
|
// 4. An ActorRef with unknown mailbox size that isn't processing anything
|
||||||
// 5. An ActorRef with a known mailbox size
|
// 5. An ActorRef with a known mailbox size
|
||||||
// 6. An ActorRef without any messages
|
// 6. An ActorRef without any messages
|
||||||
@tailrec def getNext(targets: IndexedSeq[ActorRef] = routeeProvider.routees,
|
@tailrec def getNext(targets: immutable.IndexedSeq[ActorRef] = routeeProvider.routees,
|
||||||
proposedTarget: ActorRef = routeeProvider.context.system.deadLetters,
|
proposedTarget: ActorRef = routeeProvider.context.system.deadLetters,
|
||||||
currentScore: Long = Long.MaxValue,
|
currentScore: Long = Long.MaxValue,
|
||||||
at: Int = 0,
|
at: Int = 0,
|
||||||
|
|
@ -985,7 +981,7 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
case (sender, message) ⇒
|
case (sender, message) ⇒
|
||||||
message match {
|
message match {
|
||||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||||
case msg ⇒ List(Destination(sender, getNext()))
|
case msg ⇒ Destination(sender, getNext()) :: Nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -995,15 +991,13 @@ object BroadcastRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new BroadcastRouter, routing to the specified routees
|
* Creates a new BroadcastRouter, routing to the specified routees
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
def apply(routees: immutable.Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = {
|
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter =
|
||||||
import scala.collection.JavaConverters._
|
apply(immutableSeq(routees))
|
||||||
apply(routees.asScala)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* A Router that uses broadcasts a message to all its connections.
|
* A Router that uses broadcasts a message to all its connections.
|
||||||
|
|
@ -1047,7 +1041,7 @@ object BroadcastRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
case class BroadcastRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||||
extends RouterConfig with BroadcastLike {
|
extends RouterConfig with BroadcastLike {
|
||||||
|
|
@ -1064,7 +1058,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala)
|
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -1102,7 +1096,7 @@ trait BroadcastLike { this: RouterConfig ⇒
|
||||||
|
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||||
if (resizer.isEmpty) {
|
if (resizer.isEmpty) {
|
||||||
|
|
@ -1120,16 +1114,14 @@ object ScatterGatherFirstCompletedRouter {
|
||||||
/**
|
/**
|
||||||
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
|
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
|
||||||
*/
|
*/
|
||||||
def apply(routees: Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
def apply(routees: immutable.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||||
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API to create router with the supplied 'routees' actors.
|
* Java API to create router with the supplied 'routees' actors.
|
||||||
*/
|
*/
|
||||||
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter = {
|
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||||
import scala.collection.JavaConverters._
|
apply(immutableSeq(routees), within)
|
||||||
apply(routees.asScala, within)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* Simple router that broadcasts the message to all routees, and replies with the first response.
|
* Simple router that broadcasts the message to all routees, and replies with the first response.
|
||||||
|
|
@ -1175,7 +1167,7 @@ object ScatterGatherFirstCompletedRouter {
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: FiniteDuration,
|
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, within: FiniteDuration,
|
||||||
override val resizer: Option[Resizer] = None,
|
override val resizer: Option[Resizer] = None,
|
||||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
|
||||||
|
|
@ -1196,7 +1188,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
|
||||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||||
*/
|
*/
|
||||||
def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = routeePaths.asScala, within = w)
|
def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = immutableSeq(routeePaths), within = w)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor that sets the resizer to be used.
|
* Constructor that sets the resizer to be used.
|
||||||
|
|
@ -1234,7 +1226,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
|
||||||
|
|
||||||
def nrOfInstances: Int
|
def nrOfInstances: Int
|
||||||
|
|
||||||
def routees: Iterable[String]
|
def routees: immutable.Iterable[String]
|
||||||
|
|
||||||
def within: FiniteDuration
|
def within: FiniteDuration
|
||||||
|
|
||||||
|
|
@ -1394,7 +1386,7 @@ case class DefaultResizer(
|
||||||
* @param routees The current actor in the resizer
|
* @param routees The current actor in the resizer
|
||||||
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero)
|
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero)
|
||||||
*/
|
*/
|
||||||
def capacity(routees: IndexedSeq[ActorRef]): Int = {
|
def capacity(routees: immutable.IndexedSeq[ActorRef]): Int = {
|
||||||
val currentSize = routees.size
|
val currentSize = routees.size
|
||||||
val press = pressure(routees)
|
val press = pressure(routees)
|
||||||
val delta = filter(press, currentSize)
|
val delta = filter(press, currentSize)
|
||||||
|
|
@ -1422,7 +1414,7 @@ case class DefaultResizer(
|
||||||
* @param routees the current resizer of routees
|
* @param routees the current resizer of routees
|
||||||
* @return number of busy routees, between 0 and routees.size
|
* @return number of busy routees, between 0 and routees.size
|
||||||
*/
|
*/
|
||||||
def pressure(routees: IndexedSeq[ActorRef]): Int = {
|
def pressure(routees: immutable.IndexedSeq[ActorRef]): Int = {
|
||||||
routees count {
|
routees count {
|
||||||
case a: ActorRefWithCell ⇒
|
case a: ActorRefWithCell ⇒
|
||||||
a.underlying match {
|
a.underlying match {
|
||||||
|
|
|
||||||
|
|
@ -4,10 +4,12 @@
|
||||||
|
|
||||||
package akka
|
package akka
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
package object routing {
|
package object routing {
|
||||||
/**
|
/**
|
||||||
* Routing logic, partial function from (sender, message) to a
|
* Routing logic, partial function from (sender, message) to a
|
||||||
* set of destinations.
|
* set of destinations.
|
||||||
*/
|
*/
|
||||||
type Route = PartialFunction[(akka.actor.ActorRef, Any), Iterable[Destination]]
|
type Route = PartialFunction[(akka.actor.ActorRef, Any), immutable.Iterable[Destination]]
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,14 +4,14 @@
|
||||||
|
|
||||||
package akka.serialization
|
package akka.serialization
|
||||||
|
|
||||||
import akka.AkkaException
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess }
|
import akka.actor.{ Extension, ExtendedActorSystem, Address }
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
import util.{ Try, DynamicVariable }
|
import scala.util.{ Try, DynamicVariable }
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
object Serialization {
|
object Serialization {
|
||||||
|
|
||||||
|
|
@ -27,17 +27,13 @@ object Serialization {
|
||||||
val currentTransportAddress = new DynamicVariable[Address](null)
|
val currentTransportAddress = new DynamicVariable[Address](null)
|
||||||
|
|
||||||
class Settings(val config: Config) {
|
class Settings(val config: Config) {
|
||||||
|
val Serializers: Map[String, String] = configToMap("akka.actor.serializers")
|
||||||
|
val SerializationBindings: Map[String, String] = configToMap("akka.actor.serialization-bindings")
|
||||||
|
|
||||||
|
private final def configToMap(path: String): Map[String, String] = {
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import config._
|
config.getConfig(path).root.unwrapped.asScala.mapValues(_.toString).toMap
|
||||||
|
}
|
||||||
val Serializers: Map[String, String] = configToMap(getConfig("akka.actor.serializers"))
|
|
||||||
|
|
||||||
val SerializationBindings: Map[String, String] = configToMap(getConfig("akka.actor.serialization-bindings"))
|
|
||||||
|
|
||||||
private def configToMap(cfg: Config): Map[String, String] =
|
|
||||||
cfg.root.unwrapped.asScala.toMap.map { case (k, v) ⇒ (k, v.toString) }
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -62,16 +58,16 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
||||||
* using the optional type hint to the Serializer and the optional ClassLoader ot load it into.
|
* using the optional type hint to the Serializer and the optional ClassLoader ot load it into.
|
||||||
* Returns either the resulting object or an Exception if one was thrown.
|
* Returns either the resulting object or an Exception if one was thrown.
|
||||||
*/
|
*/
|
||||||
def deserialize(bytes: Array[Byte],
|
def deserialize(bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_]]): Try[AnyRef] =
|
||||||
serializerId: Int,
|
Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
|
||||||
clazz: Option[Class[_]]): Try[AnyRef] = Try(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
|
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
|
||||||
* You can specify an optional ClassLoader to load the object into.
|
* You can specify an optional ClassLoader to load the object into.
|
||||||
* Returns either the resulting object or an Exception if one was thrown.
|
* Returns either the resulting object or an Exception if one was thrown.
|
||||||
*/
|
*/
|
||||||
def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] = Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
|
def deserialize(bytes: Array[Byte], clazz: Class[_]): Try[AnyRef] =
|
||||||
|
Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
||||||
|
|
@ -95,9 +91,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
||||||
*/
|
*/
|
||||||
def serializerFor(clazz: Class[_]): Serializer =
|
def serializerFor(clazz: Class[_]): Serializer =
|
||||||
serializerMap.get(clazz) match {
|
serializerMap.get(clazz) match {
|
||||||
case null ⇒
|
case null ⇒ // bindings are ordered from most specific to least specific
|
||||||
// bindings are ordered from most specific to least specific
|
def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean =
|
||||||
def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean =
|
|
||||||
possibilities.size == 1 ||
|
possibilities.size == 1 ||
|
||||||
(possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) ||
|
(possibilities forall (_._1 isAssignableFrom possibilities(0)._1)) ||
|
||||||
(possibilities forall (_._2 == possibilities(0)._2))
|
(possibilities forall (_._2 == possibilities(0)._2))
|
||||||
|
|
@ -122,8 +117,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
||||||
* loading is performed by the system’s [[akka.actor.DynamicAccess]].
|
* loading is performed by the system’s [[akka.actor.DynamicAccess]].
|
||||||
*/
|
*/
|
||||||
def serializerOf(serializerFQN: String): Try[Serializer] =
|
def serializerOf(serializerFQN: String): Try[Serializer] =
|
||||||
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)) recoverWith {
|
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] -> system)) recoverWith {
|
||||||
case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq())
|
case _ ⇒ system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -137,21 +132,21 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
||||||
* bindings is a Seq of tuple representing the mapping from Class to Serializer.
|
* bindings is a Seq of tuple representing the mapping from Class to Serializer.
|
||||||
* It is primarily ordered by the most specific classes first, and secondly in the configured order.
|
* It is primarily ordered by the most specific classes first, and secondly in the configured order.
|
||||||
*/
|
*/
|
||||||
private[akka] val bindings: Seq[ClassSerializer] =
|
private[akka] val bindings: immutable.Seq[ClassSerializer] =
|
||||||
sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v)))
|
sort(for ((k: String, v: String) ← settings.SerializationBindings if v != "none") yield (system.dynamicAccess.getClassFor[Any](k).get, serializers(v))).to[immutable.Seq]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sort so that subtypes always precede their supertypes, but without
|
* Sort so that subtypes always precede their supertypes, but without
|
||||||
* obeying any order between unrelated subtypes (insert sort).
|
* obeying any order between unrelated subtypes (insert sort).
|
||||||
*/
|
*/
|
||||||
private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] =
|
private def sort(in: Iterable[ClassSerializer]): immutable.Seq[ClassSerializer] =
|
||||||
(new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒
|
((new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca) ⇒
|
||||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||||
case -1 ⇒ buf append ca
|
case -1 ⇒ buf append ca
|
||||||
case x ⇒ buf insert (x, ca)
|
case x ⇒ buf insert (x, ca)
|
||||||
}
|
}
|
||||||
buf
|
buf
|
||||||
}
|
}).to[immutable.Seq]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer
|
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer
|
||||||
|
|
|
||||||
54
akka-actor/src/main/scala/akka/util/Collections.scala
Normal file
54
akka-actor/src/main/scala/akka/util/Collections.scala
Normal file
|
|
@ -0,0 +1,54 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.util
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
|
import scala.annotation.tailrec
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
private[akka] object Collections {
|
||||||
|
|
||||||
|
case object EmptyImmutableSeq extends immutable.Seq[Nothing] {
|
||||||
|
override final def iterator = Iterator.empty
|
||||||
|
override final def apply(idx: Int): Nothing = throw new java.lang.IndexOutOfBoundsException(idx.toString)
|
||||||
|
override final def length: Int = 0
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract class PartialImmutableValuesIterable[From, To] extends immutable.Iterable[To] {
|
||||||
|
def isDefinedAt(from: From): Boolean
|
||||||
|
def apply(from: From): To
|
||||||
|
def valuesIterator: Iterator[From]
|
||||||
|
final def iterator: Iterator[To] = {
|
||||||
|
val superIterator = valuesIterator
|
||||||
|
new Iterator[To] {
|
||||||
|
private[this] var _next: To = _
|
||||||
|
private[this] var _hasNext = false
|
||||||
|
|
||||||
|
@tailrec override final def hasNext: Boolean =
|
||||||
|
if (!_hasNext && superIterator.hasNext) { // If we need and are able to look for the next value
|
||||||
|
val potentiallyNext = superIterator.next()
|
||||||
|
if (isDefinedAt(potentiallyNext)) {
|
||||||
|
_next = apply(potentiallyNext)
|
||||||
|
_hasNext = true
|
||||||
|
true
|
||||||
|
} else hasNext //Attempt to find the next
|
||||||
|
} else _hasNext // Return if we found one
|
||||||
|
|
||||||
|
override final def next(): To = if (hasNext) {
|
||||||
|
val ret = _next
|
||||||
|
_next = null.asInstanceOf[To] // Mark as consumed (nice to the GC, don't leak the last returned value)
|
||||||
|
_hasNext = false // Mark as consumed (we need to look for the next value)
|
||||||
|
ret
|
||||||
|
} else throw new java.util.NoSuchElementException("next")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
override lazy val size: Int = iterator.size
|
||||||
|
override def foreach[C](f: To ⇒ C) = iterator foreach f
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -1,45 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
|
||||||
*/
|
|
||||||
|
|
||||||
package akka.util
|
|
||||||
//FIXME DOCS!
|
|
||||||
object Convert {
|
|
||||||
|
|
||||||
def intToBytes(value: Int): Array[Byte] = {
|
|
||||||
val bytes = Array.fill[Byte](4)(0)
|
|
||||||
bytes(0) = (value >>> 24).asInstanceOf[Byte]
|
|
||||||
bytes(1) = (value >>> 16).asInstanceOf[Byte]
|
|
||||||
bytes(2) = (value >>> 8).asInstanceOf[Byte]
|
|
||||||
bytes(3) = value.asInstanceOf[Byte]
|
|
||||||
bytes
|
|
||||||
}
|
|
||||||
|
|
||||||
def bytesToInt(bytes: Array[Byte], offset: Int): Int = {
|
|
||||||
(0 until 4).foldLeft(0)((value, index) ⇒ value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
|
|
||||||
}
|
|
||||||
|
|
||||||
def longToBytes(value: Long): Array[Byte] = {
|
|
||||||
val writeBuffer = Array.fill[Byte](8)(0)
|
|
||||||
writeBuffer(0) = (value >>> 56).asInstanceOf[Byte]
|
|
||||||
writeBuffer(1) = (value >>> 48).asInstanceOf[Byte]
|
|
||||||
writeBuffer(2) = (value >>> 40).asInstanceOf[Byte]
|
|
||||||
writeBuffer(3) = (value >>> 32).asInstanceOf[Byte]
|
|
||||||
writeBuffer(4) = (value >>> 24).asInstanceOf[Byte]
|
|
||||||
writeBuffer(5) = (value >>> 16).asInstanceOf[Byte]
|
|
||||||
writeBuffer(6) = (value >>> 8).asInstanceOf[Byte]
|
|
||||||
writeBuffer(7) = (value >>> 0).asInstanceOf[Byte]
|
|
||||||
writeBuffer
|
|
||||||
}
|
|
||||||
|
|
||||||
def bytesToLong(buf: Array[Byte]): Long = {
|
|
||||||
((buf(0) & 0xFFL) << 56) |
|
|
||||||
((buf(1) & 0xFFL) << 48) |
|
|
||||||
((buf(2) & 0xFFL) << 40) |
|
|
||||||
((buf(3) & 0xFFL) << 32) |
|
|
||||||
((buf(4) & 0xFFL) << 24) |
|
|
||||||
((buf(5) & 0xFFL) << 16) |
|
|
||||||
((buf(6) & 0xFFL) << 8) |
|
|
||||||
((buf(7) & 0xFFL) << 0)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -75,7 +75,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
|
||||||
def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] =
|
def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] =
|
||||||
container get key match {
|
container get key match {
|
||||||
case null ⇒ None
|
case null ⇒ None
|
||||||
case set ⇒ set.iterator.asScala.find(f)
|
case set ⇒ set.iterator.asScala find f
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,8 @@
|
||||||
*/
|
*/
|
||||||
package akka.util
|
package akka.util
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
|
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
|
||||||
*/
|
*/
|
||||||
|
|
@ -55,7 +57,7 @@ private[akka] object SubclassifiedIndex {
|
||||||
}
|
}
|
||||||
|
|
||||||
private[SubclassifiedIndex] def emptyMergeMap[K, V] = internalEmptyMergeMap.asInstanceOf[Map[K, Set[V]]]
|
private[SubclassifiedIndex] def emptyMergeMap[K, V] = internalEmptyMergeMap.asInstanceOf[Map[K, Set[V]]]
|
||||||
private[this] val internalEmptyMergeMap = Map[AnyRef, Set[AnyRef]]().withDefault(_ ⇒ Set[AnyRef]())
|
private[this] val internalEmptyMergeMap = Map[AnyRef, Set[AnyRef]]().withDefaultValue(Set[AnyRef]())
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -74,7 +76,7 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V]
|
||||||
|
|
||||||
import SubclassifiedIndex._
|
import SubclassifiedIndex._
|
||||||
|
|
||||||
type Changes = Seq[(K, Set[V])]
|
type Changes = immutable.Seq[(K, Set[V])]
|
||||||
|
|
||||||
protected var subkeys = Vector.empty[Nonroot[K, V]]
|
protected var subkeys = Vector.empty[Nonroot[K, V]]
|
||||||
|
|
||||||
|
|
@ -208,5 +210,5 @@ private[akka] class SubclassifiedIndex[K, V] private (private var values: Set[V]
|
||||||
private def mergeChangesByKey(changes: Changes): Changes =
|
private def mergeChangesByKey(changes: Changes): Changes =
|
||||||
(emptyMergeMap[K, V] /: changes) {
|
(emptyMergeMap[K, V] /: changes) {
|
||||||
case (m, (k, s)) ⇒ m.updated(k, m(k) ++ s)
|
case (m, (k, s)) ⇒ m.updated(k, m(k) ++ s)
|
||||||
}.toSeq
|
}.to[immutable.Seq]
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,27 +5,9 @@
|
||||||
|
|
||||||
package akka.util;
|
package akka.util;
|
||||||
|
|
||||||
import java.lang.reflect.Field;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
public final class Unsafe {
|
public final class Unsafe {
|
||||||
public final static sun.misc.Unsafe instance;
|
public final static sun.misc.Unsafe instance = scala.concurrent.util.Unsafe.instance;
|
||||||
static {
|
|
||||||
try {
|
|
||||||
sun.misc.Unsafe found = null;
|
|
||||||
for(Field field : sun.misc.Unsafe.class.getDeclaredFields()) {
|
|
||||||
if (field.getType() == sun.misc.Unsafe.class) {
|
|
||||||
field.setAccessible(true);
|
|
||||||
found = (sun.misc.Unsafe) field.get(null);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (found == null) throw new IllegalStateException("Can't find instance of sun.misc.Unsafe");
|
|
||||||
else instance = found;
|
|
||||||
} catch(Throwable t) {
|
|
||||||
throw new ExceptionInInitializerError(t);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,16 +4,15 @@
|
||||||
|
|
||||||
package akka.camel
|
package akka.camel
|
||||||
|
|
||||||
import internal._
|
import akka.camel.internal._
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
|
import akka.ConfigurationException
|
||||||
import org.apache.camel.ProducerTemplate
|
import org.apache.camel.ProducerTemplate
|
||||||
import org.apache.camel.impl.DefaultCamelContext
|
import org.apache.camel.impl.DefaultCamelContext
|
||||||
import org.apache.camel.model.RouteDefinition
|
import org.apache.camel.model.RouteDefinition
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.ConfigurationException
|
import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||||
import scala.concurrent.duration.Duration
|
|
||||||
import java.util.concurrent.TimeUnit._
|
import java.util.concurrent.TimeUnit._
|
||||||
import scala.concurrent.duration.FiniteDuration
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Camel trait encapsulates the underlying camel machinery.
|
* Camel trait encapsulates the underlying camel machinery.
|
||||||
|
|
@ -88,8 +87,8 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess
|
||||||
final val StreamingCache: Boolean = config.getBoolean("akka.camel.streamingCache")
|
final val StreamingCache: Boolean = config.getBoolean("akka.camel.streamingCache")
|
||||||
|
|
||||||
final val Conversions: (String, RouteDefinition) ⇒ RouteDefinition = {
|
final val Conversions: (String, RouteDefinition) ⇒ RouteDefinition = {
|
||||||
import scala.collection.JavaConverters.asScalaSetConverter
|
|
||||||
val specifiedConversions = {
|
val specifiedConversions = {
|
||||||
|
import scala.collection.JavaConverters.asScalaSetConverter
|
||||||
val section = config.getConfig("akka.camel.conversions")
|
val section = config.getConfig("akka.camel.conversions")
|
||||||
section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey)))
|
section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey)))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,13 +1,18 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
package akka.camel
|
package akka.camel
|
||||||
|
|
||||||
|
import language.postfixOps
|
||||||
|
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
import scala.concurrent.{ Promise, Await, Future }
|
||||||
|
import scala.collection.immutable
|
||||||
import akka.camel.TestSupport.NonSharedCamelSystem
|
import akka.camel.TestSupport.NonSharedCamelSystem
|
||||||
import akka.actor.{ ActorRef, Props, Actor }
|
import akka.actor.{ ActorRef, Props, Actor }
|
||||||
import akka.routing.BroadcastRouter
|
import akka.routing.BroadcastRouter
|
||||||
import concurrent.{ Promise, Await, Future }
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import language.postfixOps
|
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
import org.apache.camel.model.RouteDefinition
|
import org.apache.camel.model.RouteDefinition
|
||||||
|
|
@ -58,7 +63,7 @@ class ConcurrentActivationTest extends WordSpec with MustMatchers with NonShared
|
||||||
activations.size must be(2 * number * number)
|
activations.size must be(2 * number * number)
|
||||||
// must be the size of the activated activated producers and consumers
|
// must be the size of the activated activated producers and consumers
|
||||||
deactivations.size must be(2 * number * number)
|
deactivations.size must be(2 * number * number)
|
||||||
def partitionNames(refs: Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer"))
|
def partitionNames(refs: immutable.Seq[ActorRef]) = refs.map(_.path.name).partition(_.startsWith("concurrent-test-echo-consumer"))
|
||||||
def assertContainsSameElements(lists: (Seq[_], Seq[_])) {
|
def assertContainsSameElements(lists: (Seq[_], Seq[_])) {
|
||||||
val (a, b) = lists
|
val (a, b) = lists
|
||||||
a.intersect(b).size must be(a.size)
|
a.intersect(b).size must be(a.size)
|
||||||
|
|
|
||||||
|
|
@ -75,7 +75,7 @@ class Cluster(val system: ExtendedActorSystem) extends Extension {
|
||||||
val failureDetector: FailureDetector = {
|
val failureDetector: FailureDetector = {
|
||||||
import settings.{ FailureDetectorImplementationClass ⇒ fqcn }
|
import settings.{ FailureDetectorImplementationClass ⇒ fqcn }
|
||||||
system.dynamicAccess.createInstanceFor[FailureDetector](
|
system.dynamicAccess.createInstanceFor[FailureDetector](
|
||||||
fqcn, Seq(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({
|
fqcn, List(classOf[ActorSystem] -> system, classOf[ClusterSettings] -> settings)).recover({
|
||||||
case e ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString)
|
case e ⇒ throw new ConfigurationException("Could not create custom failure detector [" + fqcn + "] due to:" + e.toString)
|
||||||
}).get
|
}).get
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -9,8 +9,8 @@ import scala.concurrent.duration._
|
||||||
import scala.collection.immutable.{ SortedSet, Map }
|
import scala.collection.immutable.{ SortedSet, Map }
|
||||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||||
import scala.util.{ Try, Success, Failure }
|
import scala.util.{ Try, Success, Failure }
|
||||||
import scala.math.ScalaNumericConversions
|
import scala.math.ScalaNumericAnyConversions
|
||||||
import scala.runtime.{ RichLong, RichDouble, RichInt }
|
import runtime.{ ScalaNumberProxy, RichLong, RichDouble, RichInt }
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.event.LoggingAdapter
|
import akka.event.LoggingAdapter
|
||||||
|
|
@ -238,7 +238,7 @@ private[cluster] case class MetricsGossipEnvelope(from: Address, gossip: Metrics
|
||||||
*
|
*
|
||||||
* @param startTime the time of initial sampling for this data stream
|
* @param startTime the time of initial sampling for this data stream
|
||||||
*/
|
*/
|
||||||
private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions, startTime: Long, timestamp: Long)
|
private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericAnyConversions, startTime: Long, timestamp: Long)
|
||||||
extends ClusterMessage with MetricNumericConverter {
|
extends ClusterMessage with MetricNumericConverter {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -249,13 +249,13 @@ private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Calculates the exponentially weighted moving average for a given monitored data set.
|
* Calculates the exponentially weighted moving average for a given monitored data set.
|
||||||
* The datam can be too large to fit into an int or long, thus we use ScalaNumericConversions,
|
* The datam can be too large to fit into an int or long, thus we use ScalaNumericAnyConversions,
|
||||||
* and defer to BigInt or BigDecimal.
|
* and defer to BigInt or BigDecimal.
|
||||||
*
|
*
|
||||||
* @param xn the new data point
|
* @param xn the new data point
|
||||||
* @return an new [[akka.cluster.DataStream]] with the updated yn and timestamp
|
* @return an new [[akka.cluster.DataStream]] with the updated yn and timestamp
|
||||||
*/
|
*/
|
||||||
def :+(xn: ScalaNumericConversions): DataStream = convert(xn) fold (
|
def :+(xn: ScalaNumericAnyConversions): DataStream = convert(xn) fold (
|
||||||
nl ⇒ copy(ewma = BigInt(α * nl + 1 - α * ewma.longValue()), timestamp = newTimestamp),
|
nl ⇒ copy(ewma = BigInt(α * nl + 1 - α * ewma.longValue()), timestamp = newTimestamp),
|
||||||
nd ⇒ copy(ewma = BigDecimal(α * nd + 1 - α * ewma.doubleValue()), timestamp = newTimestamp))
|
nd ⇒ copy(ewma = BigDecimal(α * nd + 1 - α * ewma.doubleValue()), timestamp = newTimestamp))
|
||||||
|
|
||||||
|
|
@ -273,7 +273,7 @@ private[cluster] case class DataStream(decay: Int, ewma: ScalaNumericConversions
|
||||||
*/
|
*/
|
||||||
private[cluster] object DataStream {
|
private[cluster] object DataStream {
|
||||||
|
|
||||||
def apply(decay: Int, data: ScalaNumericConversions): Option[DataStream] = if (decay > 0)
|
def apply(decay: Int, data: ScalaNumericAnyConversions): Option[DataStream] = if (decay > 0)
|
||||||
Some(DataStream(decay, data, newTimestamp, newTimestamp)) else None
|
Some(DataStream(decay, data, newTimestamp, newTimestamp)) else None
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -288,7 +288,7 @@ private[cluster] object DataStream {
|
||||||
* @param average the data stream of the metric value, for trending over time. Metrics that are already
|
* @param average the data stream of the metric value, for trending over time. Metrics that are already
|
||||||
* averages (e.g. system load average) or finite (e.g. as total cores), are not trended.
|
* averages (e.g. system load average) or finite (e.g. as total cores), are not trended.
|
||||||
*/
|
*/
|
||||||
private[cluster] case class Metric(name: String, value: Option[ScalaNumericConversions], average: Option[DataStream])
|
private[cluster] case class Metric(name: String, value: Option[ScalaNumericAnyConversions], average: Option[DataStream])
|
||||||
extends ClusterMessage with MetricNumericConverter {
|
extends ClusterMessage with MetricNumericConverter {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -352,7 +352,7 @@ private[cluster] object Metric extends MetricNumericConverter {
|
||||||
* or defined for the OS (JMX). If undefined we set the value option to None and do not modify
|
* or defined for the OS (JMX). If undefined we set the value option to None and do not modify
|
||||||
* the latest sampled metric to avoid skewing the statistical trend.
|
* the latest sampled metric to avoid skewing the statistical trend.
|
||||||
*/
|
*/
|
||||||
def apply(name: String, value: Option[ScalaNumericConversions]): Metric = value match {
|
def apply(name: String, value: Option[ScalaNumericAnyConversions]): Metric = value match {
|
||||||
case Some(v) if defined(v) ⇒ Metric(name, value, None)
|
case Some(v) if defined(v) ⇒ Metric(name, value, None)
|
||||||
case _ ⇒ Metric(name, None, None)
|
case _ ⇒ Metric(name, None, None)
|
||||||
}
|
}
|
||||||
|
|
@ -409,13 +409,13 @@ private[cluster] trait MetricNumericConverter {
|
||||||
* <ul><li>JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned</li>
|
* <ul><li>JMX system load average and max heap can be 'undefined' for certain OS, in which case a -1 is returned</li>
|
||||||
* <li>SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)</li></ul>
|
* <li>SIGAR combined CPU can occasionally return a NaN or Infinite (known bug)</li></ul>
|
||||||
*/
|
*/
|
||||||
def defined(value: ScalaNumericConversions): Boolean =
|
def defined(value: ScalaNumericAnyConversions): Boolean =
|
||||||
convert(value) fold (a ⇒ value.underlying != -1, b ⇒ !(b.isNaN || b.isInfinite))
|
convert(value) fold (a ⇒ value.underlying != -1, b ⇒ !(b.isNaN || b.isInfinite))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* May involve rounding or truncation.
|
* May involve rounding or truncation.
|
||||||
*/
|
*/
|
||||||
def convert(from: ScalaNumericConversions): Either[Long, Double] = from match {
|
def convert(from: ScalaNumericAnyConversions): Either[Long, Double] = from match {
|
||||||
case n: BigInt ⇒ Left(n.longValue())
|
case n: BigInt ⇒ Left(n.longValue())
|
||||||
case n: BigDecimal ⇒ Right(n.doubleValue())
|
case n: BigDecimal ⇒ Right(n.doubleValue())
|
||||||
case n: RichInt ⇒ Left(n.abs)
|
case n: RichInt ⇒ Left(n.abs)
|
||||||
|
|
@ -463,11 +463,16 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* (SIGAR / JMX) Returns the OS-specific average system load on the CPUs in the system, for the past 1 minute.
|
* (SIGAR / JMX) Returns the OS-specific average system load on the CPUs in the system, for the past 1 minute.
|
||||||
* On some systems the JMX OS system load average may not be available, in which case a -1 is returned.
|
* On some systems the JMX OS system load average may not be available, in which case a Metric with
|
||||||
|
* undefined value is returned.
|
||||||
* Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default.
|
* Hyperic SIGAR provides more precise values, thus, if the library is on the classpath, it is the default.
|
||||||
*/
|
*/
|
||||||
def systemLoadAverage: Metric = Metric("system-load-average", Some(BigDecimal(Try(
|
def systemLoadAverage: Metric = Metric("system-load-average",
|
||||||
LoadAverage.get.invoke(sigar.get).asInstanceOf[Array[Double]].toSeq.head) getOrElse osMBean.getSystemLoadAverage)))
|
Try(LoadAverage.get.invoke(sigar.get).asInstanceOf[Array[Double]].toSeq.head).getOrElse(
|
||||||
|
osMBean.getSystemLoadAverage) match {
|
||||||
|
case x if x < 0 ⇒ None // load average may be unavailable on some platform
|
||||||
|
case x ⇒ Some(BigDecimal(x))
|
||||||
|
})
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* (JMX) Returns the number of available processors
|
* (JMX) Returns the number of available processors
|
||||||
|
|
@ -552,7 +557,7 @@ private[cluster] class MetricsCollector private (private val sigar: Option[AnyRe
|
||||||
*/
|
*/
|
||||||
private[cluster] object MetricsCollector {
|
private[cluster] object MetricsCollector {
|
||||||
def apply(address: Address, log: LoggingAdapter, dynamicAccess: DynamicAccess): MetricsCollector =
|
def apply(address: Address, log: LoggingAdapter, dynamicAccess: DynamicAccess): MetricsCollector =
|
||||||
dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Seq.empty) match {
|
dynamicAccess.createInstanceFor[AnyRef]("org.hyperic.sigar.Sigar", Nil) match {
|
||||||
case Success(identity) ⇒ new MetricsCollector(Some(identity), address)
|
case Success(identity) ⇒ new MetricsCollector(Some(identity), address)
|
||||||
case Failure(e) ⇒
|
case Failure(e) ⇒
|
||||||
log.debug(e.toString)
|
log.debug(e.toString)
|
||||||
|
|
|
||||||
|
|
@ -45,9 +45,7 @@ class ClusterSettings(val config: Config, val systemName: String) {
|
||||||
require(n > 0, "failure-detector.monitored-by-nr-of-members must be > 0"); n
|
require(n > 0, "failure-detector.monitored-by-nr-of-members must be > 0"); n
|
||||||
}
|
}
|
||||||
|
|
||||||
final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map {
|
final val SeedNodes: IndexedSeq[Address] = getStringList("akka.cluster.seed-nodes").asScala.map { case AddressFromURIString(addr) ⇒ addr }.toIndexedSeq
|
||||||
case AddressFromURIString(addr) ⇒ addr
|
|
||||||
}.toIndexedSeq
|
|
||||||
final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS)
|
final val SeedNodeTimeout: FiniteDuration = Duration(getMilliseconds("akka.cluster.seed-node-timeout"), MILLISECONDS)
|
||||||
final val PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
|
final val PeriodicTasksInitialDelay: FiniteDuration = Duration(getMilliseconds("akka.cluster.periodic-tasks-initial-delay"), MILLISECONDS)
|
||||||
final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
|
final val GossipInterval: FiniteDuration = Duration(getMilliseconds("akka.cluster.gossip-interval"), MILLISECONDS)
|
||||||
|
|
|
||||||
|
|
@ -5,16 +5,13 @@ package akka.cluster.routing
|
||||||
|
|
||||||
import java.lang.IllegalStateException
|
import java.lang.IllegalStateException
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
import scala.collection.immutable.SortedSet
|
import scala.collection.immutable
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.ConfigurationException
|
import akka.ConfigurationException
|
||||||
import akka.actor.Actor
|
|
||||||
import akka.actor.ActorContext
|
import akka.actor.ActorContext
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.ActorSystemImpl
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.actor.Deploy
|
import akka.actor.Deploy
|
||||||
import akka.actor.InternalActorRef
|
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.actor.SupervisorStrategy
|
import akka.actor.SupervisorStrategy
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
|
|
@ -51,7 +48,7 @@ final case class ClusterRouterConfig(local: RouterConfig, settings: ClusterRoute
|
||||||
|
|
||||||
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
||||||
({
|
({
|
||||||
case (sender, message: ClusterDomainEvent) ⇒ Seq(Destination(sender, routeeProvider.context.self))
|
case (sender, message: ClusterDomainEvent) ⇒ List(Destination(sender, routeeProvider.context.self))
|
||||||
}: Route) orElse localRoute
|
}: Route) orElse localRoute
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -156,7 +153,7 @@ private[akka] class ClusterRouteeProvider(
|
||||||
// need this counter as instance variable since Resizer may call createRoutees several times
|
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||||
private val childNameCounter = new AtomicInteger
|
private val childNameCounter = new AtomicInteger
|
||||||
|
|
||||||
override def registerRouteesFor(paths: Iterable[String]): Unit =
|
override def registerRouteesFor(paths: immutable.Iterable[String]): Unit =
|
||||||
throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
|
throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
|
||||||
format context.self.path.toString)
|
format context.self.path.toString)
|
||||||
|
|
||||||
|
|
@ -183,7 +180,7 @@ private[akka] class ClusterRouteeProvider(
|
||||||
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
||||||
}
|
}
|
||||||
// must register each one, since registered routees are used in selectDeploymentTarget
|
// must register each one, since registered routees are used in selectDeploymentTarget
|
||||||
registerRoutees(Some(ref))
|
registerRoutees(List(ref))
|
||||||
|
|
||||||
// recursion until all created
|
// recursion until all created
|
||||||
doCreateRoutees()
|
doCreateRoutees()
|
||||||
|
|
@ -202,7 +199,7 @@ private[akka] class ClusterRouteeProvider(
|
||||||
} else {
|
} else {
|
||||||
// find the node with least routees
|
// find the node with least routees
|
||||||
val numberOfRouteesPerNode: Map[Address, Int] =
|
val numberOfRouteesPerNode: Map[Address, Int] =
|
||||||
currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefault(_ ⇒ 0)) { (acc, x) ⇒
|
currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefaultValue(0)) { (acc, x) ⇒
|
||||||
val address = fullAddress(x)
|
val address = fullAddress(x)
|
||||||
acc + (address -> (acc(address) + 1))
|
acc + (address -> (acc(address) + 1))
|
||||||
}
|
}
|
||||||
|
|
@ -222,27 +219,26 @@ private[akka] class ClusterRouteeProvider(
|
||||||
case a ⇒ a
|
case a ⇒ a
|
||||||
}
|
}
|
||||||
|
|
||||||
private[routing] def availableNodes: SortedSet[Address] = {
|
private[routing] def availableNodes: immutable.SortedSet[Address] = {
|
||||||
import Member.addressOrdering
|
import Member.addressOrdering
|
||||||
val currentNodes = nodes
|
val currentNodes = nodes
|
||||||
if (currentNodes.isEmpty && settings.allowLocalRoutees)
|
if (currentNodes.isEmpty && settings.allowLocalRoutees)
|
||||||
//use my own node, cluster information not updated yet
|
//use my own node, cluster information not updated yet
|
||||||
SortedSet(cluster.selfAddress)
|
immutable.SortedSet(cluster.selfAddress)
|
||||||
else
|
else
|
||||||
currentNodes
|
currentNodes
|
||||||
}
|
}
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private[routing] var nodes: SortedSet[Address] = {
|
private[routing] var nodes: immutable.SortedSet[Address] = {
|
||||||
import Member.addressOrdering
|
import Member.addressOrdering
|
||||||
cluster.readView.members.collect {
|
cluster.readView.members.collect {
|
||||||
case m if isAvailable(m) ⇒ m.address
|
case m if isAvailable(m) ⇒ m.address
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private[routing] def isAvailable(m: Member): Boolean = {
|
private[routing] def isAvailable(m: Member): Boolean =
|
||||||
m.status == MemberStatus.Up && (settings.allowLocalRoutees || m.address != cluster.selfAddress)
|
m.status == MemberStatus.Up && (settings.allowLocalRoutees || m.address != cluster.selfAddress)
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class ClientDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -51,7 +52,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
|
||||||
cluster.down(thirdAddress)
|
cluster.down(thirdAddress)
|
||||||
enterBarrier("down-third-node")
|
enterBarrier("down-third-node")
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||||
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -62,7 +63,7 @@ abstract class ClientDowningNodeThatIsUnreachableSpec(multiNodeConfig: ClientDow
|
||||||
runOn(second, fourth) {
|
runOn(second, fourth) {
|
||||||
enterBarrier("down-third-node")
|
enterBarrier("down-third-node")
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("await-completion")
|
enterBarrier("await-completion")
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class ClientDowningNodeThatIsUpMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -49,7 +50,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
|
||||||
|
|
||||||
markNodeAsUnavailable(thirdAddress)
|
markNodeAsUnavailable(thirdAddress)
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||||
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
clusterView.members.exists(_.address == thirdAddress) must be(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -60,7 +61,7 @@ abstract class ClientDowningNodeThatIsUpSpec(multiNodeConfig: ClientDowningNodeT
|
||||||
runOn(second, fourth) {
|
runOn(second, fourth) {
|
||||||
enterBarrier("down-third-node")
|
enterBarrier("down-third-node")
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(thirdAddress))
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(thirdAddress))
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("await-completion")
|
enterBarrier("await-completion")
|
||||||
|
|
|
||||||
|
|
@ -55,7 +55,6 @@ object LargeClusterMultiJvmSpec extends MultiNodeConfig {
|
||||||
akka.remote.log-remote-lifecycle-events = off
|
akka.remote.log-remote-lifecycle-events = off
|
||||||
akka.remote.netty.execution-pool-size = 4
|
akka.remote.netty.execution-pool-size = 4
|
||||||
#akka.remote.netty.reconnection-time-window = 10s
|
#akka.remote.netty.reconnection-time-window = 10s
|
||||||
akka.remote.netty.read-timeout = 5s
|
|
||||||
akka.remote.netty.write-timeout = 5s
|
akka.remote.netty.write-timeout = 5s
|
||||||
akka.remote.netty.backoff-timeout = 500ms
|
akka.remote.netty.backoff-timeout = 500ms
|
||||||
akka.remote.netty.connection-timeout = 500ms
|
akka.remote.netty.connection-timeout = 500ms
|
||||||
|
|
@ -148,7 +147,7 @@ abstract class LargeClusterSpec
|
||||||
runOn(runOnRoles: _*) {
|
runOn(runOnRoles: _*) {
|
||||||
systems.size must be(nodesPerDatacenter) // make sure it is initialized
|
systems.size must be(nodesPerDatacenter) // make sure it is initialized
|
||||||
|
|
||||||
val clusterNodes = ifNode(from)(joiningClusterNodes)(systems.map(Cluster(_)).toSet)
|
val clusterNodes = if(isNode(from)) joiningClusterNodes else systems.map(Cluster(_)).toSet
|
||||||
val startGossipCounts = Map.empty[Cluster, Long] ++
|
val startGossipCounts = Map.empty[Cluster, Long] ++
|
||||||
clusterNodes.map(c ⇒ (c -> c.readView.latestStats.receivedGossipCount))
|
clusterNodes.map(c ⇒ (c -> c.readView.latestStats.receivedGossipCount))
|
||||||
def gossipCount(c: Cluster): Long = {
|
def gossipCount(c: Cluster): Long = {
|
||||||
|
|
@ -260,7 +259,7 @@ abstract class LargeClusterSpec
|
||||||
if (bulk.nonEmpty) {
|
if (bulk.nonEmpty) {
|
||||||
val totalNodes = nodesPerDatacenter * 4 + bulk.size
|
val totalNodes = nodesPerDatacenter * 4 + bulk.size
|
||||||
within(expectedMaxDuration(totalNodes)) {
|
within(expectedMaxDuration(totalNodes)) {
|
||||||
val joiningClusters = ifNode(fifthDatacenter)(bulk.map(Cluster(_)).toSet)(Set.empty)
|
val joiningClusters = if(isNode(fifthDatacenter)) bulk.map(Cluster(_)).toSet else Set.empty[Cluster]
|
||||||
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
||||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
||||||
enterBarrier("fifth-datacenter-joined-" + bulk.size)
|
enterBarrier("fifth-datacenter-joined-" + bulk.size)
|
||||||
|
|
@ -270,7 +269,7 @@ abstract class LargeClusterSpec
|
||||||
for (i ← 0 until oneByOne.size) {
|
for (i ← 0 until oneByOne.size) {
|
||||||
val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1
|
val totalNodes = nodesPerDatacenter * 4 + bulk.size + i + 1
|
||||||
within(expectedMaxDuration(totalNodes)) {
|
within(expectedMaxDuration(totalNodes)) {
|
||||||
val joiningClusters = ifNode(fifthDatacenter)(Set(Cluster(oneByOne(i))))(Set.empty)
|
val joiningClusters = if(isNode(fifthDatacenter)) Set(Cluster(oneByOne(i))) else Set.empty[Cluster]
|
||||||
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
join(joiningClusters, from = fifthDatacenter, to = firstDatacenter, totalNodes,
|
||||||
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
runOnRoles = firstDatacenter, secondDatacenter, thirdDatacenter, fourthDatacenter, fifthDatacenter)
|
||||||
enterBarrier("fifth-datacenter-joined-" + (bulk.size + i))
|
enterBarrier("fifth-datacenter-joined-" + (bulk.size + i))
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class LeaderDowningNodeThatIsUnreachableMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -59,7 +60,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
||||||
|
|
||||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(fourth) {
|
runOn(fourth) {
|
||||||
|
|
@ -69,7 +70,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
||||||
runOn(second, third) {
|
runOn(second, third) {
|
||||||
enterBarrier("down-fourth-node")
|
enterBarrier("down-fourth-node")
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = Seq(fourthAddress), 30.seconds)
|
awaitUpConvergence(numberOfMembers = 3, canNotBePartOfMemberRing = List(fourthAddress), 30.seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("await-completion-1")
|
enterBarrier("await-completion-1")
|
||||||
|
|
@ -89,7 +90,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
||||||
|
|
||||||
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
// --- HERE THE LEADER SHOULD DETECT FAILURE AND AUTO-DOWN THE UNREACHABLE NODE ---
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30.seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
|
|
@ -99,7 +100,7 @@ abstract class LeaderDowningNodeThatIsUnreachableSpec(multiNodeConfig: LeaderDow
|
||||||
runOn(third) {
|
runOn(third) {
|
||||||
enterBarrier("down-second-node")
|
enterBarrier("down-second-node")
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = Seq(secondAddress), 30 seconds)
|
awaitUpConvergence(numberOfMembers = 2, canNotBePartOfMemberRing = List(secondAddress), 30 seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("await-completion-2")
|
enterBarrier("await-completion-2")
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class LeaderElectionMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val controller = role("controller")
|
val controller = role("controller")
|
||||||
|
|
@ -42,7 +43,7 @@ abstract class LeaderElectionSpec(multiNodeConfig: LeaderElectionMultiNodeConfig
|
||||||
import multiNodeConfig._
|
import multiNodeConfig._
|
||||||
|
|
||||||
// sorted in the order used by the cluster
|
// sorted in the order used by the cluster
|
||||||
lazy val sortedRoles = Seq(first, second, third, fourth).sorted
|
lazy val sortedRoles = List(first, second, third, fourth).sorted
|
||||||
|
|
||||||
"A cluster of four nodes" must {
|
"A cluster of four nodes" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,21 +4,21 @@
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import language.implicitConversions
|
import language.implicitConversions
|
||||||
|
|
||||||
|
import org.scalatest.Suite
|
||||||
|
import org.scalatest.exceptions.TestFailedException
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.actor.{ Address, ExtendedActorSystem }
|
|
||||||
import akka.remote.testconductor.RoleName
|
import akka.remote.testconductor.RoleName
|
||||||
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec }
|
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeSpec }
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.testkit.TestEvent._
|
import akka.testkit.TestEvent._
|
||||||
import scala.concurrent.duration._
|
import akka.actor.{ ActorSystem, Address }
|
||||||
import org.scalatest.Suite
|
|
||||||
import org.scalatest.exceptions.TestFailedException
|
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
|
||||||
import akka.actor.ActorPath
|
|
||||||
import akka.actor.RootActorPath
|
|
||||||
import akka.event.Logging.ErrorLevel
|
import akka.event.Logging.ErrorLevel
|
||||||
import akka.actor.ActorSystem
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
|
|
||||||
object MultiNodeClusterSpec {
|
object MultiNodeClusterSpec {
|
||||||
|
|
||||||
|
|
@ -158,7 +158,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
||||||
* nodes (roles). First node will be started first
|
* nodes (roles). First node will be started first
|
||||||
* and others will join the first.
|
* and others will join the first.
|
||||||
*/
|
*/
|
||||||
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.toSeq)
|
def startCluster(roles: RoleName*): Unit = awaitStartCluster(false, roles.to[immutable.Seq])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the cluster of the specified member
|
* Initialize the cluster of the specified member
|
||||||
|
|
@ -166,11 +166,9 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
||||||
* First node will be started first and others will join
|
* First node will be started first and others will join
|
||||||
* the first.
|
* the first.
|
||||||
*/
|
*/
|
||||||
def awaitClusterUp(roles: RoleName*): Unit = {
|
def awaitClusterUp(roles: RoleName*): Unit = awaitStartCluster(true, roles.to[immutable.Seq])
|
||||||
awaitStartCluster(true, roles.toSeq)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def awaitStartCluster(upConvergence: Boolean = true, roles: Seq[RoleName]): Unit = {
|
private def awaitStartCluster(upConvergence: Boolean = true, roles: immutable.Seq[RoleName]): Unit = {
|
||||||
runOn(roles.head) {
|
runOn(roles.head) {
|
||||||
// make sure that the node-to-join is started before other join
|
// make sure that the node-to-join is started before other join
|
||||||
startClusterNode()
|
startClusterNode()
|
||||||
|
|
@ -196,21 +194,20 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
||||||
expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) ⇒ members(i).address must be(a) }
|
expectedAddresses.sorted.zipWithIndex.foreach { case (a, i) ⇒ members(i).address must be(a) }
|
||||||
}
|
}
|
||||||
|
|
||||||
def assertLeader(nodesInCluster: RoleName*): Unit = if (nodesInCluster.contains(myself)) {
|
def assertLeader(nodesInCluster: RoleName*): Unit =
|
||||||
assertLeaderIn(nodesInCluster)
|
if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq])
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Assert that the cluster has elected the correct leader
|
* Assert that the cluster has elected the correct leader
|
||||||
* out of all nodes in the cluster. First
|
* out of all nodes in the cluster. First
|
||||||
* member in the cluster ring is expected leader.
|
* member in the cluster ring is expected leader.
|
||||||
*/
|
*/
|
||||||
def assertLeaderIn(nodesInCluster: Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) {
|
def assertLeaderIn(nodesInCluster: immutable.Seq[RoleName]): Unit = if (nodesInCluster.contains(myself)) {
|
||||||
nodesInCluster.length must not be (0)
|
nodesInCluster.length must not be (0)
|
||||||
val expectedLeader = roleOfLeader(nodesInCluster)
|
val expectedLeader = roleOfLeader(nodesInCluster)
|
||||||
val leader = clusterView.leader
|
val leader = clusterView.leader
|
||||||
val isLeader = leader == Some(clusterView.selfAddress)
|
val isLeader = leader == Some(clusterView.selfAddress)
|
||||||
assert(isLeader == ifNode(expectedLeader)(true)(false),
|
assert(isLeader == isNode(expectedLeader),
|
||||||
"expectedLeader [%s], got leader [%s], members [%s]".format(expectedLeader, leader, clusterView.members))
|
"expectedLeader [%s], got leader [%s], members [%s]".format(expectedLeader, leader, clusterView.members))
|
||||||
clusterView.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving))
|
clusterView.status must (be(MemberStatus.Up) or be(MemberStatus.Leaving))
|
||||||
}
|
}
|
||||||
|
|
@ -221,7 +218,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
||||||
*/
|
*/
|
||||||
def awaitUpConvergence(
|
def awaitUpConvergence(
|
||||||
numberOfMembers: Int,
|
numberOfMembers: Int,
|
||||||
canNotBePartOfMemberRing: Seq[Address] = Seq.empty[Address],
|
canNotBePartOfMemberRing: immutable.Seq[Address] = Nil,
|
||||||
timeout: FiniteDuration = 20.seconds): Unit = {
|
timeout: FiniteDuration = 20.seconds): Unit = {
|
||||||
within(timeout) {
|
within(timeout) {
|
||||||
awaitCond(clusterView.members.size == numberOfMembers)
|
awaitCond(clusterView.members.size == numberOfMembers)
|
||||||
|
|
@ -239,7 +236,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec { self: MultiNodeS
|
||||||
def awaitSeenSameState(addresses: Address*): Unit =
|
def awaitSeenSameState(addresses: Address*): Unit =
|
||||||
awaitCond((addresses.toSet -- clusterView.seenBy).isEmpty)
|
awaitCond((addresses.toSet -- clusterView.seenBy).isEmpty)
|
||||||
|
|
||||||
def roleOfLeader(nodesInCluster: Seq[RoleName] = roles): RoleName = {
|
def roleOfLeader(nodesInCluster: immutable.Seq[RoleName] = roles): RoleName = {
|
||||||
nodesInCluster.length must not be (0)
|
nodesInCluster.length must not be (0)
|
||||||
nodesInCluster.sorted.head
|
nodesInCluster.sorted.head
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class SingletonClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -65,7 +66,7 @@ abstract class SingletonClusterSpec(multiNodeConfig: SingletonClusterMultiNodeCo
|
||||||
|
|
||||||
markNodeAsUnavailable(secondAddress)
|
markNodeAsUnavailable(secondAddress)
|
||||||
|
|
||||||
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = Seq(secondAddress), 30.seconds)
|
awaitUpConvergence(numberOfMembers = 1, canNotBePartOfMemberRing = List(secondAddress), 30.seconds)
|
||||||
clusterView.isSingletonCluster must be(true)
|
clusterView.isSingletonCluster must be(true)
|
||||||
awaitCond(clusterView.isLeader)
|
awaitCond(clusterView.isLeader)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -9,9 +9,10 @@ import com.typesafe.config.ConfigFactory
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.remote.testconductor.Direction
|
import akka.remote.testconductor.Direction
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class SplitBrainMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -53,8 +54,8 @@ abstract class SplitBrainSpec(multiNodeConfig: SplitBrainMultiNodeConfig)
|
||||||
|
|
||||||
muteMarkingAsUnreachable()
|
muteMarkingAsUnreachable()
|
||||||
|
|
||||||
val side1 = IndexedSeq(first, second)
|
val side1 = Vector(first, second)
|
||||||
val side2 = IndexedSeq(third, fourth, fifth)
|
val side2 = Vector(third, fourth, fifth)
|
||||||
|
|
||||||
"A cluster of 5 members" must {
|
"A cluster of 5 members" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -6,13 +6,14 @@ package akka.cluster
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
import org.scalatest.BeforeAndAfter
|
import org.scalatest.BeforeAndAfter
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.remote.testconductor.{ RoleName, Direction }
|
import akka.remote.testconductor.{ RoleName, Direction }
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
case class UnreachableNodeRejoinsClusterMultiNodeConfig(failureDetectorPuppet: Boolean) extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -45,7 +46,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
|
||||||
|
|
||||||
muteMarkingAsUnreachable()
|
muteMarkingAsUnreachable()
|
||||||
|
|
||||||
def allBut(role: RoleName, roles: Seq[RoleName] = roles): Seq[RoleName] = {
|
def allBut(role: RoleName, roles: immutable.Seq[RoleName] = roles): immutable.Seq[RoleName] = {
|
||||||
roles.filterNot(_ == role)
|
roles.filterNot(_ == role)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -125,7 +126,7 @@ abstract class UnreachableNodeRejoinsClusterSpec(multiNodeConfig: UnreachableNod
|
||||||
}
|
}
|
||||||
|
|
||||||
runOn(allBut(victim): _*) {
|
runOn(allBut(victim): _*) {
|
||||||
awaitUpConvergence(roles.size - 1, Seq(victim))
|
awaitUpConvergence(roles.size - 1, List(victim))
|
||||||
}
|
}
|
||||||
|
|
||||||
endBarrier
|
endBarrier
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ package akka.cluster
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.testkit.TestEvent._
|
import akka.testkit.TestEvent._
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
|
|
@ -27,7 +27,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
||||||
val conn = Address("akka", "", "localhost", 2552)
|
val conn = Address("akka", "", "localhost", 2552)
|
||||||
val conn2 = Address("akka", "", "localhost", 2553)
|
val conn2 = Address("akka", "", "localhost", 2553)
|
||||||
|
|
||||||
def fakeTimeGenerator(timeIntervals: Seq[Long]): () ⇒ Long = {
|
def fakeTimeGenerator(timeIntervals: immutable.Seq[Long]): () ⇒ Long = {
|
||||||
var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c))
|
var times = timeIntervals.tail.foldLeft(List[Long](timeIntervals.head))((acc, c) ⇒ acc ::: List[Long](acc.last + c))
|
||||||
def timeGenerator(): Long = {
|
def timeGenerator(): Long = {
|
||||||
val currentTime = times.head
|
val currentTime = times.head
|
||||||
|
|
@ -73,7 +73,7 @@ class AccrualFailureDetectorSpec extends AkkaSpec("""
|
||||||
|
|
||||||
"return realistic phi values" in {
|
"return realistic phi values" in {
|
||||||
val fd = createFailureDetector()
|
val fd = createFailureDetector()
|
||||||
val test = TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3)
|
val test = immutable.TreeMap(0 -> 0.0, 500 -> 0.1, 1000 -> 0.3, 1200 -> 1.6, 1400 -> 4.7, 1600 -> 10.8, 1700 -> 15.3)
|
||||||
for ((timeDiff, expectedPhi) ← test) {
|
for ((timeDiff, expectedPhi) ← test) {
|
||||||
fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1))
|
fd.phi(timeDiff = timeDiff, mean = 1000.0, stdDeviation = 100.0) must be(expectedPhi plusOrMinus (0.1))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,14 +5,16 @@
|
||||||
package akka.cluster
|
package akka.cluster
|
||||||
|
|
||||||
import scala.language.postfixOps
|
import scala.language.postfixOps
|
||||||
|
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
|
import scala.util.{ Success, Try, Failure }
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import org.scalatest.WordSpec
|
import org.scalatest.WordSpec
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
import util.{ Success, Try, Failure }
|
|
||||||
|
|
||||||
object MetricsEnabledSpec {
|
object MetricsEnabledSpec {
|
||||||
val config = """
|
val config = """
|
||||||
|
|
@ -109,8 +111,7 @@ class MetricsCollectorSpec extends AkkaSpec(MetricsEnabledSpec.config) with Impl
|
||||||
|
|
||||||
"collect JMX metrics" in {
|
"collect JMX metrics" in {
|
||||||
// heap max may be undefined depending on the OS
|
// heap max may be undefined depending on the OS
|
||||||
// systemLoadAverage is JMX is SIGAR not present
|
// systemLoadAverage is JMX if SIGAR not present, but not available on all OS
|
||||||
collector.systemLoadAverage.isDefined must be(true)
|
|
||||||
collector.used.isDefined must be(true)
|
collector.used.isDefined must be(true)
|
||||||
collector.committed.isDefined must be(true)
|
collector.committed.isDefined must be(true)
|
||||||
collector.processors.isDefined must be(true)
|
collector.processors.isDefined must be(true)
|
||||||
|
|
@ -208,10 +209,9 @@ trait MetricSpec extends WordSpec with MustMatchers {
|
||||||
if (decay > 0) metrics.collect { case m if m.trendable && (!m.initializable) ⇒ m }.foreach(_.average.isDefined must be(true))
|
if (decay > 0) metrics.collect { case m if m.trendable && (!m.initializable) ⇒ m }.foreach(_.average.isDefined must be(true))
|
||||||
}
|
}
|
||||||
|
|
||||||
def collectNodeMetrics(nodes: Set[NodeMetrics]): Seq[Metric] = {
|
def collectNodeMetrics(nodes: Set[NodeMetrics]): immutable.Seq[Metric] =
|
||||||
var r: Seq[Metric] = Seq.empty
|
nodes.foldLeft(Vector[Metric]()) {
|
||||||
nodes.foreach(n ⇒ r ++= n.metrics.filter(_.isDefined))
|
case (r, n) ⇒ r ++ n.metrics.filter(_.isDefined)
|
||||||
r
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -29,6 +29,7 @@ The Current List of Modules
|
||||||
.. toctree::
|
.. toctree::
|
||||||
|
|
||||||
reliable-proxy
|
reliable-proxy
|
||||||
|
throttle
|
||||||
|
|
||||||
Suggested Way of Using these Contributions
|
Suggested Way of Using these Contributions
|
||||||
------------------------------------------
|
------------------------------------------
|
||||||
|
|
|
||||||
60
akka-contrib/docs/throttle.rst
Normal file
60
akka-contrib/docs/throttle.rst
Normal file
|
|
@ -0,0 +1,60 @@
|
||||||
|
Throttling Actor Messages
|
||||||
|
=========================
|
||||||
|
|
||||||
|
Introduction
|
||||||
|
------------
|
||||||
|
|
||||||
|
Suppose you are writing an application that makes HTTP requests to an external
|
||||||
|
web service and that this web service has a restriction in place: you may not
|
||||||
|
make more than 10 requests in 1 minute. You will get blocked or need to pay if
|
||||||
|
you don’t stay under this limit. In such a scenario you will want to employ
|
||||||
|
a *message throttler*.
|
||||||
|
|
||||||
|
This extension module provides a simple implementation of a throttling actor,
|
||||||
|
the :class:`TimerBasedThrottler`.
|
||||||
|
|
||||||
|
|
||||||
|
How to use it
|
||||||
|
-------------
|
||||||
|
|
||||||
|
You can use a :class:`TimerBasedThrottler` as follows:
|
||||||
|
|
||||||
|
.. includecode:: @contribSrc@/src/test/scala/akka/contrib/throttle/TimerBasedThrottlerSpec.scala#demo-code
|
||||||
|
|
||||||
|
Please refer to the ScalaDoc documentation for the details.
|
||||||
|
|
||||||
|
|
||||||
|
The guarantees
|
||||||
|
--------------
|
||||||
|
|
||||||
|
:class:`TimerBasedThrottler` uses a timer internally. When the throttler’s rate is 3 msg/s,
|
||||||
|
for example, the throttler will start a timer that triggers
|
||||||
|
every second and each time will give the throttler exactly three "vouchers";
|
||||||
|
each voucher gives the throttler a right to deliver a message. In this way,
|
||||||
|
at most 3 messages will be sent out by the throttler in each interval.
|
||||||
|
|
||||||
|
It should be noted that such timer-based throttlers provide relatively **weak guarantees**:
|
||||||
|
|
||||||
|
* Only *start times* are taken into account. This may be a problem if, for example, the
|
||||||
|
throttler is used to throttle requests to an external web service. If a web request
|
||||||
|
takes very long on the server then the rate *observed on the server* may be higher.
|
||||||
|
* A timer-based throttler only makes guarantees for the intervals of its own timer. In
|
||||||
|
our example, no more than 3 messages are delivered within such intervals. Other
|
||||||
|
intervals on the timeline, however, may contain more calls.
|
||||||
|
|
||||||
|
The two cases are illustrated in the two figures below, each showing a timeline and three
|
||||||
|
intervals of the timer. The message delivery times chosen by the throttler are indicated
|
||||||
|
by dots, and as you can see, each interval contains at most 3 point, so the throttler
|
||||||
|
works correctly. Still, there is in each example an interval (the red one) that is
|
||||||
|
problematic. In the first scenario, this is because the delivery times are merely the
|
||||||
|
start times of longer requests (indicated by the four bars above the timeline that start
|
||||||
|
at the dots), so that the server observes four requests during the red interval. In the
|
||||||
|
second scenario, the messages are centered around one of the points in time where the
|
||||||
|
timer triggers, causing the red interval to contain too many messages.
|
||||||
|
|
||||||
|
.. image:: throttler.png
|
||||||
|
|
||||||
|
For some application scenarios, the guarantees provided by a timer-based throttler might
|
||||||
|
be too weak. Charles Cordingley’s `blog post <http://www.cordinc.com/blog/2010/04/java-multichannel-asynchronous.html>`_
|
||||||
|
discusses a throttler with stronger guarantees (it solves problem 2 from above).
|
||||||
|
Future versions of this module may feature throttlers with better guarantees.
|
||||||
BIN
akka-contrib/docs/throttler.png
Normal file
BIN
akka-contrib/docs/throttler.png
Normal file
Binary file not shown.
|
After Width: | Height: | Size: 3.9 KiB |
|
|
@ -0,0 +1,296 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.contrib.throttle
|
||||||
|
|
||||||
|
import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||||
|
import scala.util.control.NonFatal
|
||||||
|
import scala.collection.immutable.{ Queue ⇒ Q }
|
||||||
|
import akka.actor.{ ActorRef, Actor, FSM }
|
||||||
|
import Throttler._
|
||||||
|
import TimerBasedThrottler._
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
import akka.AkkaException
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marker trait for throttlers.
|
||||||
|
*
|
||||||
|
* == Throttling ==
|
||||||
|
* A <em>throttler</em> is an actor that is defined through a <em>target actor</em> and a <em>rate</em>
|
||||||
|
* (of type [[akka.contrib.throttle.Throttler.Rate]]). You set or change the target and rate at any time through the `SetTarget(target)`
|
||||||
|
* and `SetRate(rate)` messages, respectively. When you send the throttler any other message `msg`, it will
|
||||||
|
* put the message `msg` into an internal queue and eventually send all queued messages to the target, at
|
||||||
|
* a speed that respects the given rate. If no target is currently defined then the messages will be queued
|
||||||
|
* and will be delivered as soon as a target gets set.
|
||||||
|
*
|
||||||
|
* A [[akka.contrib.throttle.Throttler]] understands actor messages of type
|
||||||
|
* [[akka.contrib.throttle.Throttler.SetTarget]], [[akka.contrib.throttle.Throttler.SetRate]], in
|
||||||
|
* addition to any other messages, which the throttler will consider as messages to be sent to
|
||||||
|
* the target.
|
||||||
|
*
|
||||||
|
* == Transparency ==
|
||||||
|
* Notice that the throttler `forward`s messages, i.e., the target will see the original message sender (and not the throttler) as the sender of the message.
|
||||||
|
*
|
||||||
|
* == Persistence ==
|
||||||
|
* Throttlers usually use an internal queue to keep the messages that need to be sent to the target.
|
||||||
|
* You therefore cannot rely on the throttler's inbox size in order to learn how much messages are
|
||||||
|
* outstanding.
|
||||||
|
*
|
||||||
|
* It is left to the implementation whether the internal queue is persisted over application restarts or
|
||||||
|
* actor failure.
|
||||||
|
*
|
||||||
|
* == Processing messages ==
|
||||||
|
* The target should process messages as fast as possible. If the target requires substantial time to
|
||||||
|
* process messages, it should distribute its work to other actors (using for example something like
|
||||||
|
* a `BalancingDispatcher`), otherwise the resulting system will always work <em>below</em>
|
||||||
|
* the threshold rate.
|
||||||
|
*
|
||||||
|
* <em>Example:</em> Suppose the throttler has a rate of 3msg/s and the target requires 1s to process a message.
|
||||||
|
* This system will only process messages at a rate of 1msg/s: the target will receive messages at at most 3msg/s
|
||||||
|
* but as it handles them synchronously and each of them takes 1s, its inbox will grow and grow. In such
|
||||||
|
* a situation, the target should <em>distribute</em> its messages to a set of worker actors so that individual messages
|
||||||
|
* can be handled in parallel.
|
||||||
|
*
|
||||||
|
* @see [[akka.contrib.throttle.TimerBasedThrottler]]
|
||||||
|
*/
|
||||||
|
trait Throttler { this: Actor ⇒ }
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Message types understood by [[akka.contrib.throttle.Throttler]]'s.
|
||||||
|
*
|
||||||
|
* @see [[akka.contrib.throttle.Throttler]]
|
||||||
|
* @see [[akka.contrib.throttle.Throttler.Rate]]
|
||||||
|
*/
|
||||||
|
object Throttler {
|
||||||
|
/**
|
||||||
|
* A rate used for throttling.
|
||||||
|
*
|
||||||
|
* There are some shorthands available to construct rates:
|
||||||
|
* {{{
|
||||||
|
* import java.util.concurrent.TimeUnit._
|
||||||
|
* import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||||
|
*
|
||||||
|
* val rate1 = 1 msgsPer (1, SECONDS)
|
||||||
|
* val rate2 = 1 msgsPer Duration(1, SECONDS)
|
||||||
|
* val rate3 = 1 msgsPer (1 seconds)
|
||||||
|
* val rate4 = 1 msgsPerSecond
|
||||||
|
* val rate5 = 1 msgsPerMinute
|
||||||
|
* val rate6 = 1 msgsPerHour
|
||||||
|
* }}}
|
||||||
|
*
|
||||||
|
* @param numberOfCalls the number of calls that may take place in a period
|
||||||
|
* @param duration the length of the period
|
||||||
|
* @see [[akka.contrib.throttle.Throttler]]
|
||||||
|
*/
|
||||||
|
case class Rate(val numberOfCalls: Int, val duration: FiniteDuration) {
|
||||||
|
/**
|
||||||
|
* The duration in milliseconds.
|
||||||
|
*/
|
||||||
|
def durationInMillis(): Long = duration.toMillis
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the target of a [[akka.contrib.throttle.Throttler]].
|
||||||
|
*
|
||||||
|
* You may change a throttler's target at any time.
|
||||||
|
*
|
||||||
|
* Notice that the messages sent by the throttler to the target will have the original sender (and
|
||||||
|
* not the throttler) as the sender. (In Akka terms, the throttler `forward`s the message.)
|
||||||
|
*
|
||||||
|
* @param target if `target` is `None`, the throttler will stop delivering messages and the messages already received
|
||||||
|
* but not yet delivered, as well as any messages received in the future will be queued
|
||||||
|
* and eventually be delivered when a new target is set. If `target` is not `None`, the currently queued messages
|
||||||
|
* as well as any messages received in the the future will be delivered to the new target at a rate not exceeding the current throttler's rate.
|
||||||
|
*/
|
||||||
|
case class SetTarget(target: Option[ActorRef])
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the rate of a [[akka.contrib.throttle.Throttler]].
|
||||||
|
*
|
||||||
|
* You may change a throttler's rate at any time.
|
||||||
|
*
|
||||||
|
* @param rate the rate at which messages will be delivered to the target of the throttler
|
||||||
|
*/
|
||||||
|
case class SetRate(rate: Rate)
|
||||||
|
|
||||||
|
import language.implicitConversions
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper for some syntactic sugar.
|
||||||
|
*
|
||||||
|
* @see [[akka.contrib.throttle.Throttler.Rate]]
|
||||||
|
*/
|
||||||
|
implicit class RateInt(val numberOfCalls: Int) extends AnyVal {
|
||||||
|
def msgsPer(duration: Int, timeUnit: TimeUnit) = Rate(numberOfCalls, Duration(duration, timeUnit))
|
||||||
|
def msgsPer(duration: FiniteDuration) = Rate(numberOfCalls, duration)
|
||||||
|
def msgsPerSecond = Rate(numberOfCalls, Duration(1, TimeUnit.SECONDS))
|
||||||
|
def msgsPerMinute = Rate(numberOfCalls, Duration(1, TimeUnit.MINUTES))
|
||||||
|
def msgsPerHour = Rate(numberOfCalls, Duration(1, TimeUnit.HOURS))
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implementation-specific internals.
|
||||||
|
*/
|
||||||
|
object TimerBasedThrottler {
|
||||||
|
private[throttle] case object Tick
|
||||||
|
|
||||||
|
// States of the FSM: A `TimerBasedThrottler` is in state `Active` iff the timer is running.
|
||||||
|
private[throttle] sealed trait State
|
||||||
|
private[throttle] case object Idle extends State
|
||||||
|
private[throttle] case object Active extends State
|
||||||
|
|
||||||
|
// Messages, as we queue them to be sent later
|
||||||
|
private[throttle] case class Message(message: Any, sender: ActorRef)
|
||||||
|
|
||||||
|
// The data of the FSM
|
||||||
|
private[throttle] sealed case class Data(target: Option[ActorRef],
|
||||||
|
callsLeftInThisPeriod: Int,
|
||||||
|
queue: Q[Message])
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A [[akka.contrib.throttle.Throttler]] that uses a timer to control the message delivery rate.
|
||||||
|
*
|
||||||
|
* ==Example==
|
||||||
|
* For example, if you set a rate like "3 messages in 1 second", the throttler
|
||||||
|
* will send the first three messages immediately to the target actor but will need to impose a delay before
|
||||||
|
* sending out further messages:
|
||||||
|
* {{{
|
||||||
|
* // A simple actor that prints whatever it receives
|
||||||
|
* val printer = system.actorOf(Props(new Actor {
|
||||||
|
* def receive = {
|
||||||
|
* case x => println(x)
|
||||||
|
* }
|
||||||
|
* }))
|
||||||
|
* // The throttler for this example, setting the rate
|
||||||
|
* val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second))))
|
||||||
|
* // Set the target
|
||||||
|
* throttler ! SetTarget(Some(printer))
|
||||||
|
* // These three messages will be sent to the printer immediately
|
||||||
|
* throttler ! "1"
|
||||||
|
* throttler ! "2"
|
||||||
|
* throttler ! "3"
|
||||||
|
* // These two will wait at least until 1 second has passed
|
||||||
|
* throttler ! "4"
|
||||||
|
* throttler ! "5"
|
||||||
|
* }}}
|
||||||
|
*
|
||||||
|
* ==Implementation notes==
|
||||||
|
* This throttler implementation internally installs a timer that repeats every `rate.durationInMillis` and enables `rate.numberOfCalls`
|
||||||
|
* additional calls to take place. A `TimerBasedThrottler` uses very few system resources, provided the rate's duration is not too
|
||||||
|
* fine-grained (which would cause a lot of timer invocations); for example, it does not store the calling history
|
||||||
|
* as other throttlers may need to do.
|
||||||
|
*
|
||||||
|
* However, a `TimerBasedThrottler` only provides ''weak guarantees'' on the rate (see also
|
||||||
|
* <a href='http://letitcrash.com/post/28901663062/throttling-messages-in-akka-2'>this blog post</a>):
|
||||||
|
*
|
||||||
|
* - Only ''delivery'' times are taken into account: if, for example, the throttler is used to throttle
|
||||||
|
* requests to an external web service then only the start times of the web requests are considered.
|
||||||
|
* If a web request takes very long on the server then more than `rate.numberOfCalls`-many requests
|
||||||
|
* may be observed on the server in an interval of duration `rate.durationInMillis()`.
|
||||||
|
* - There may be intervals of duration `rate.durationInMillis()` that contain more than `rate.numberOfCalls`
|
||||||
|
* message deliveries: a `TimerBasedThrottler` only makes guarantees for the intervals
|
||||||
|
* of its ''own'' timer, namely that no more than `rate.numberOfCalls`-many messages are delivered within such intervals. Other intervals on the
|
||||||
|
* timeline may contain more calls.
|
||||||
|
*
|
||||||
|
* For some applications, these guarantees may not be sufficient.
|
||||||
|
*
|
||||||
|
* ==Known issues==
|
||||||
|
*
|
||||||
|
* - If you change the rate using `SetRate(rate)`, the actual rate may in fact be higher for the
|
||||||
|
* overlapping period (i.e., `durationInMillis()`) of the new and old rate. Therefore,
|
||||||
|
* changing the rate frequently is not recommended with the current implementation.
|
||||||
|
* - The queue of messages to be delivered is not persisted in any way; actor or system failure will
|
||||||
|
* cause the queued messages to be lost.
|
||||||
|
*
|
||||||
|
* @see [[akka.contrib.throttle.Throttler]]
|
||||||
|
*/
|
||||||
|
class TimerBasedThrottler(var rate: Rate) extends Actor with Throttler with FSM[State, Data] {
|
||||||
|
startWith(Idle, Data(None, rate.numberOfCalls, Q()))
|
||||||
|
|
||||||
|
// Idle: no messages, or target not set
|
||||||
|
when(Idle) {
|
||||||
|
// Set the rate
|
||||||
|
case Event(SetRate(rate), d) ⇒
|
||||||
|
this.rate = rate
|
||||||
|
stay using d.copy(callsLeftInThisPeriod = rate.numberOfCalls)
|
||||||
|
|
||||||
|
// Set the target
|
||||||
|
case Event(SetTarget(t @ Some(_)), d) if !d.queue.isEmpty ⇒
|
||||||
|
goto(Active) using deliverMessages(d.copy(target = t))
|
||||||
|
case Event(SetTarget(t), d) ⇒
|
||||||
|
stay using d.copy(target = t)
|
||||||
|
|
||||||
|
// Queuing
|
||||||
|
case Event(msg, d @ Data(None, _, queue)) ⇒
|
||||||
|
stay using d.copy(queue = queue.enqueue(Message(msg, context.sender)))
|
||||||
|
case Event(msg, d @ Data(Some(_), _, Seq())) ⇒
|
||||||
|
goto(Active) using deliverMessages(d.copy(queue = Q(Message(msg, context.sender))))
|
||||||
|
// Note: The case Event(msg, t @ Data(Some(_), _, _, Seq(_*))) should never happen here.
|
||||||
|
}
|
||||||
|
|
||||||
|
when(Active) {
|
||||||
|
// Set the rate
|
||||||
|
case Event(SetRate(rate), d) ⇒
|
||||||
|
this.rate = rate
|
||||||
|
// Note: this should be improved (see "Known issues" in class comments)
|
||||||
|
stopTimer()
|
||||||
|
startTimer(rate)
|
||||||
|
stay using d.copy(callsLeftInThisPeriod = rate.numberOfCalls)
|
||||||
|
|
||||||
|
// Set the target (when the new target is None)
|
||||||
|
case Event(SetTarget(None), d) ⇒
|
||||||
|
// Note: We do not yet switch to state `Inactive` because we need the timer to tick once more before
|
||||||
|
stay using d.copy(target = None)
|
||||||
|
|
||||||
|
// Set the target (when the new target is not None)
|
||||||
|
case Event(SetTarget(t @ Some(_)), d) ⇒
|
||||||
|
stay using d.copy(target = t)
|
||||||
|
|
||||||
|
// Tick after a `SetTarget(None)`: take the additional permits and go to `Idle`
|
||||||
|
case Event(Tick, d @ Data(None, _, _)) ⇒
|
||||||
|
goto(Idle) using d.copy(callsLeftInThisPeriod = rate.numberOfCalls)
|
||||||
|
|
||||||
|
// Period ends and we have no more messages: take the additional permits and go to `Idle`
|
||||||
|
case Event(Tick, d @ Data(_, _, Seq())) ⇒
|
||||||
|
goto(Idle) using d.copy(callsLeftInThisPeriod = rate.numberOfCalls)
|
||||||
|
|
||||||
|
// Period ends and we get more occasions to send messages
|
||||||
|
case Event(Tick, d @ Data(_, _, _)) ⇒
|
||||||
|
stay using deliverMessages(d.copy(callsLeftInThisPeriod = rate.numberOfCalls))
|
||||||
|
|
||||||
|
// Queue a message (when we cannot send messages in the current period anymore)
|
||||||
|
case Event(msg, d @ Data(_, 0, queue)) ⇒
|
||||||
|
stay using d.copy(queue = queue.enqueue(Message(msg, context.sender)))
|
||||||
|
|
||||||
|
// Queue a message (when we can send some more messages in the current period)
|
||||||
|
case Event(msg, d @ Data(_, _, queue)) ⇒
|
||||||
|
stay using deliverMessages(d.copy(queue = queue.enqueue(Message(msg, context.sender))))
|
||||||
|
}
|
||||||
|
|
||||||
|
onTransition {
|
||||||
|
case Idle -> Active ⇒ startTimer(rate)
|
||||||
|
case Active -> Idle ⇒ stopTimer()
|
||||||
|
}
|
||||||
|
|
||||||
|
initialize
|
||||||
|
|
||||||
|
private def startTimer(rate: Rate) = setTimer("morePermits", Tick, rate.duration, true)
|
||||||
|
private def stopTimer() = cancelTimer("morePermits")
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Send as many messages as we can (while respecting the rate) to the target and
|
||||||
|
* return the state data (with the queue containing the remaining ones).
|
||||||
|
*/
|
||||||
|
private def deliverMessages(data: Data): Data = {
|
||||||
|
val queue = data.queue
|
||||||
|
val nrOfMsgToSend = scala.math.min(queue.length, data.callsLeftInThisPeriod)
|
||||||
|
|
||||||
|
queue.take(nrOfMsgToSend).foreach(x ⇒ data.target.get.tell(x.message, x.sender))
|
||||||
|
|
||||||
|
data.copy(queue = queue.drop(nrOfMsgToSend), callsLeftInThisPeriod = data.callsLeftInThisPeriod - nrOfMsgToSend)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -122,8 +122,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
|
||||||
enterBarrier("test2b")
|
enterBarrier("test2b")
|
||||||
|
|
||||||
runOn(local) {
|
runOn(local) {
|
||||||
testConductor.throttle(local, remote, Direction.Send, -1)
|
testConductor.throttle(local, remote, Direction.Send, -1).await
|
||||||
expectTransition(Active, Idle)
|
within(5 seconds) { expectTransition(Active, Idle) }
|
||||||
}
|
}
|
||||||
runOn(remote) {
|
runOn(remote) {
|
||||||
within(1 second) {
|
within(1 second) {
|
||||||
|
|
@ -152,8 +152,8 @@ class ReliableProxySpec extends MultiNodeSpec(ReliableProxySpec) with STMultiNod
|
||||||
enterBarrier("test3a")
|
enterBarrier("test3a")
|
||||||
|
|
||||||
runOn(local) {
|
runOn(local) {
|
||||||
testConductor.throttle(local, remote, Direction.Receive, -1)
|
testConductor.throttle(local, remote, Direction.Receive, -1).await
|
||||||
expectTransition(Active, Idle)
|
within(5 seconds) { expectTransition(Active, Idle) }
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("test3b")
|
enterBarrier("test3b")
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,205 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.contrib.throttle
|
||||||
|
|
||||||
|
import language.postfixOps
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
import akka.actor.ActorSystem
|
||||||
|
import akka.actor.Actor
|
||||||
|
import akka.actor.Props
|
||||||
|
import akka.testkit.TestKit
|
||||||
|
import akka.testkit.ImplicitSender
|
||||||
|
import akka.contrib.throttle.Throttler._
|
||||||
|
import org.junit.runner.RunWith
|
||||||
|
import org.scalatest.junit.JUnitRunner
|
||||||
|
import org.scalatest.WordSpec
|
||||||
|
import org.scalatest.matchers.MustMatchers
|
||||||
|
import org.scalatest.BeforeAndAfterAll
|
||||||
|
import akka.testkit._
|
||||||
|
|
||||||
|
object TimerBasedThrottlerSpec {
|
||||||
|
class EchoActor extends Actor {
|
||||||
|
def receive = {
|
||||||
|
case x ⇒ sender ! x
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@RunWith(classOf[JUnitRunner])
|
||||||
|
class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSpec")) with ImplicitSender
|
||||||
|
with WordSpec with MustMatchers with BeforeAndAfterAll {
|
||||||
|
|
||||||
|
override def afterAll {
|
||||||
|
system.shutdown()
|
||||||
|
}
|
||||||
|
|
||||||
|
"A throttler" must {
|
||||||
|
def println(a: Any) = ()
|
||||||
|
"must pass the ScalaDoc class documentation example program" in {
|
||||||
|
//#demo-code
|
||||||
|
// A simple actor that prints whatever it receives
|
||||||
|
val printer = system.actorOf(Props(new Actor {
|
||||||
|
def receive = {
|
||||||
|
case x ⇒ println(x)
|
||||||
|
}
|
||||||
|
}))
|
||||||
|
// The throttler for this example, setting the rate
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated))))
|
||||||
|
// Set the target
|
||||||
|
throttler ! SetTarget(Some(printer))
|
||||||
|
// These three messages will be sent to the echoer immediately
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
// These two will wait until a second has passed
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
//#demo-code
|
||||||
|
}
|
||||||
|
|
||||||
|
"keep messages until a target is set" in {
|
||||||
|
val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor])
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated))))
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
throttler ! "6"
|
||||||
|
expectNoMsg(1 second)
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
within(2 seconds) {
|
||||||
|
expectMsg("1")
|
||||||
|
expectMsg("2")
|
||||||
|
expectMsg("3")
|
||||||
|
expectMsg("4")
|
||||||
|
expectMsg("5")
|
||||||
|
expectMsg("6")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"send messages after a `SetTarget(None)` pause" in {
|
||||||
|
val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor])
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated))))
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
throttler ! SetTarget(None)
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("1")
|
||||||
|
expectMsg("2")
|
||||||
|
expectMsg("3")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
expectNoMsg(1 second)
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
throttler ! "6"
|
||||||
|
throttler ! "7"
|
||||||
|
within(1 seconds) {
|
||||||
|
expectMsg("4")
|
||||||
|
expectMsg("5")
|
||||||
|
expectMsg("6")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("7")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"keep messages when the target is set to None" in {
|
||||||
|
val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor])
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated))))
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
throttler ! "6"
|
||||||
|
throttler ! "7"
|
||||||
|
throttler ! SetTarget(None)
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("1")
|
||||||
|
expectMsg("2")
|
||||||
|
expectMsg("3")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
expectNoMsg(1 second)
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
within(1 seconds) {
|
||||||
|
expectMsg("4")
|
||||||
|
expectMsg("5")
|
||||||
|
expectMsg("6")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("7")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"respect the rate (3 msg/s)" in {
|
||||||
|
val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor])
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(3 msgsPer (1.second.dilated))))
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
throttler ! "6"
|
||||||
|
throttler ! "7"
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("1")
|
||||||
|
expectMsg("2")
|
||||||
|
expectMsg("3")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("4")
|
||||||
|
expectMsg("5")
|
||||||
|
expectMsg("6")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("7")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
"respect the rate (4 msg/s)" in {
|
||||||
|
val echo = system.actorOf(Props[TimerBasedThrottlerSpec.EchoActor])
|
||||||
|
val throttler = system.actorOf(Props(new TimerBasedThrottler(4 msgsPer (1.second.dilated))))
|
||||||
|
throttler ! SetTarget(Some(echo))
|
||||||
|
throttler ! "1"
|
||||||
|
throttler ! "2"
|
||||||
|
throttler ! "3"
|
||||||
|
throttler ! "4"
|
||||||
|
throttler ! "5"
|
||||||
|
throttler ! "6"
|
||||||
|
throttler ! "7"
|
||||||
|
throttler ! "8"
|
||||||
|
throttler ! "9"
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("1")
|
||||||
|
expectMsg("2")
|
||||||
|
expectMsg("3")
|
||||||
|
expectMsg("4")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("5")
|
||||||
|
expectMsg("6")
|
||||||
|
expectMsg("7")
|
||||||
|
expectMsg("8")
|
||||||
|
expectNoMsg()
|
||||||
|
}
|
||||||
|
within(1 second) {
|
||||||
|
expectMsg("9")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -46,7 +46,7 @@ package object dataflow {
|
||||||
implicit class DataflowPromise[T](val promise: Promise[T]) extends AnyVal {
|
implicit class DataflowPromise[T](val promise: Promise[T]) extends AnyVal {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Completes the Promise with the speicifed value or throws an exception if already
|
* Completes the Promise with the specified value or throws an exception if already
|
||||||
* completed. See Promise.success(value) for semantics.
|
* completed. See Promise.success(value) for semantics.
|
||||||
*
|
*
|
||||||
* @param value The value which denotes the successful value of the Promise
|
* @param value The value which denotes the successful value of the Promise
|
||||||
|
|
@ -59,7 +59,7 @@ package object dataflow {
|
||||||
/**
|
/**
|
||||||
* Completes this Promise with the value of the specified Future when/if it completes.
|
* Completes this Promise with the value of the specified Future when/if it completes.
|
||||||
*
|
*
|
||||||
* @param other The Future whose value will be transfered to this Promise upon completion
|
* @param other The Future whose value will be transferred to this Promise upon completion
|
||||||
* @param ec An ExecutionContext which will be used to execute callbacks registered in this method
|
* @param ec An ExecutionContext which will be used to execute callbacks registered in this method
|
||||||
* @return A Future representing the result of this operation
|
* @return A Future representing the result of this operation
|
||||||
*/
|
*/
|
||||||
|
|
@ -75,7 +75,7 @@ package object dataflow {
|
||||||
/**
|
/**
|
||||||
* Completes this Promise with the value of the specified Promise when/if it completes.
|
* Completes this Promise with the value of the specified Promise when/if it completes.
|
||||||
*
|
*
|
||||||
* @param other The Promise whose value will be transfered to this Promise upon completion
|
* @param other The Promise whose value will be transferred to this Promise upon completion
|
||||||
* @param ec An ExecutionContext which will be used to execute callbacks registered in this method
|
* @param ec An ExecutionContext which will be used to execute callbacks registered in this method
|
||||||
* @return A Future representing the result of this operation
|
* @return A Future representing the result of this operation
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@ Duration
|
||||||
########
|
########
|
||||||
|
|
||||||
Durations are used throughout the Akka library, wherefore this concept is
|
Durations are used throughout the Akka library, wherefore this concept is
|
||||||
represented by a special data type, :class:`scala.concurrent.util.Duration`.
|
represented by a special data type, :class:`scala.concurrent.duration.Duration`.
|
||||||
Values of this type may represent infinite (:obj:`Duration.Inf`,
|
Values of this type may represent infinite (:obj:`Duration.Inf`,
|
||||||
:obj:`Duration.MinusInf`) or finite durations, or be :obj:`Duration.Undefined`.
|
:obj:`Duration.MinusInf`) or finite durations, or be :obj:`Duration.Undefined`.
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,13 +17,12 @@ built using `Sphinx`_.
|
||||||
Sphinx
|
Sphinx
|
||||||
======
|
======
|
||||||
|
|
||||||
More to come...
|
For more details see `The Sphinx Documentation <http://sphinx.pocoo.org/contents.html>`_
|
||||||
|
|
||||||
|
|
||||||
reStructuredText
|
reStructuredText
|
||||||
================
|
================
|
||||||
|
|
||||||
More to come...
|
For more details see `The reST Quickref <http://docutils.sourceforge.net/docs/user/rst/quickref.html>`_
|
||||||
|
|
||||||
Sections
|
Sections
|
||||||
--------
|
--------
|
||||||
|
|
@ -75,16 +74,17 @@ First install `Sphinx`_. See below.
|
||||||
Building
|
Building
|
||||||
--------
|
--------
|
||||||
|
|
||||||
::
|
For the html version of the docs::
|
||||||
|
|
||||||
cd akka-docs
|
sbt sphinx:generate-html
|
||||||
|
|
||||||
make html
|
open <project-dir>/akka-docs/target/sphinx/html/index.html
|
||||||
open _build/html/index.html
|
|
||||||
|
|
||||||
make pdf
|
For the pdf version of the docs::
|
||||||
open _build/latex/Akka.pdf
|
|
||||||
|
|
||||||
|
sbt sphinx:generate-pdf
|
||||||
|
|
||||||
|
open <project-dir>/akka-docs/target/sphinx/latex/Akka.pdf
|
||||||
|
|
||||||
Installing Sphinx on OS X
|
Installing Sphinx on OS X
|
||||||
-------------------------
|
-------------------------
|
||||||
|
|
|
||||||
|
|
@ -18,51 +18,64 @@ You can add it as a plugin by adding the following to your project/plugins.sbt:
|
||||||
.. includecode:: ../../../project/plugins.sbt#sbt-multi-jvm
|
.. includecode:: ../../../project/plugins.sbt#sbt-multi-jvm
|
||||||
|
|
||||||
You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm``
|
You can then add multi-JVM testing to ``project/Build.scala`` by including the ``MultiJvm``
|
||||||
settings and config. For example, here is an example of how the akka-remote-tests project adds
|
settings and config. Please note that MultiJvm test sources are located in ``src/multi-jvm/...``,
|
||||||
multi-JVM testing (Simplified for clarity):
|
and not in ``src/test/...``.
|
||||||
|
|
||||||
|
Here is an example Build.scala file that uses the MultiJvm plugin:
|
||||||
|
|
||||||
.. parsed-literal::
|
.. parsed-literal::
|
||||||
|
|
||||||
import sbt._
|
import sbt._
|
||||||
import Keys._
|
import Keys._
|
||||||
import com.typesafe.sbt.SbtMultiJvm
|
import com.typesafe.sbt.SbtMultiJvm
|
||||||
import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.{ MultiJvm, extraOptions }
|
import com.typesafe.sbt.SbtMultiJvm.MultiJvmKeys.{ MultiJvm }
|
||||||
|
|
||||||
object AkkaBuild extends Build {
|
object ExampleBuild extends Build {
|
||||||
|
|
||||||
lazy val remoteTests = Project(
|
lazy val buildSettings = Defaults.defaultSettings ++ multiJvmSettings ++ Seq(
|
||||||
id = "akka-remote-tests",
|
organization := "example",
|
||||||
base = file("akka-remote-tests"),
|
version := "1.0",
|
||||||
dependencies = Seq(remote, actorTests % "test->test",
|
scalaVersion := "@scalaVersion@",
|
||||||
testkit % "test->test"),
|
// make sure that the artifacts don't have the scala version in the name
|
||||||
settings = defaultSettings ++ Seq(
|
crossPaths := false
|
||||||
|
)
|
||||||
|
|
||||||
|
lazy val example = Project(
|
||||||
|
id = "example",
|
||||||
|
base = file("."),
|
||||||
|
settings = buildSettings ++
|
||||||
|
Seq(libraryDependencies ++= Dependencies.example)
|
||||||
|
) configs(MultiJvm)
|
||||||
|
|
||||||
|
lazy val multiJvmSettings = SbtMultiJvm.multiJvmSettings ++ Seq(
|
||||||
|
// make sure that MultiJvm test are compiled by the default test compilation
|
||||||
|
compile in MultiJvm <<= (compile in MultiJvm) triggeredBy (compile in Test),
|
||||||
// disable parallel tests
|
// disable parallel tests
|
||||||
parallelExecution in Test := false,
|
parallelExecution in Test := false,
|
||||||
extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src =>
|
// make sure that MultiJvm tests are executed by the default test target
|
||||||
(name: String) => (src ** (name + ".conf")).get.
|
executeTests in Test <<=
|
||||||
headOption.map("-Dakka.config=" + _.absolutePath).toSeq
|
((executeTests in Test), (executeTests in MultiJvm)) map {
|
||||||
},
|
|
||||||
executeTests in Test <<= ((executeTests in Test),
|
|
||||||
(executeTests in MultiJvm)) map {
|
|
||||||
case ((_, testResults), (_, multiJvmResults)) =>
|
case ((_, testResults), (_, multiJvmResults)) =>
|
||||||
val results = testResults ++ multiJvmResults
|
val results = testResults ++ multiJvmResults
|
||||||
(Tests.overall(results.values), results)
|
(Tests.overall(results.values), results)
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
) configs (MultiJvm)
|
|
||||||
|
|
||||||
lazy val buildSettings = Defaults.defaultSettings ++
|
object Dependencies {
|
||||||
SbtMultiJvm.multiJvmSettings ++ Seq(
|
val example = Seq(
|
||||||
organization := "com.typesafe.akka",
|
// ---- application dependencies ----
|
||||||
version := "@version@",
|
"com.typesafe.akka" %% "akka-actor" % "@version@" @crossString@,
|
||||||
scalaVersion := "@scalaVersion@",
|
"com.typesafe.akka" %% "akka-remote" % "@version@" @crossString@,
|
||||||
crossPaths := false
|
|
||||||
|
// ---- test dependencies ----
|
||||||
|
"com.typesafe.akka" %% "akka-testkit" % "@version@" %
|
||||||
|
"test" cross CrossVersion.full,
|
||||||
|
"com.typesafe.akka" %% "akka-remote-tests-experimental" % "@version@" %
|
||||||
|
"test" cross CrossVersion.full,
|
||||||
|
"org.scalatest" %% "scalatest" % "1.8-B2" % "test" cross CrossVersion.full,
|
||||||
|
"junit" % "junit" % "4.5" % "test"
|
||||||
)
|
)
|
||||||
|
}
|
||||||
lazy val defaultSettings = buildSettings ++ Seq(
|
|
||||||
resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/"
|
|
||||||
)
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
You can specify JVM options for the forked JVMs::
|
You can specify JVM options for the forked JVMs::
|
||||||
|
|
@ -73,7 +86,7 @@ You can specify JVM options for the forked JVMs::
|
||||||
Running tests
|
Running tests
|
||||||
=============
|
=============
|
||||||
|
|
||||||
The multi-jvm tasks are similar to the normal tasks: ``test``, ``test-only``,
|
The multi-JVM tasks are similar to the normal tasks: ``test``, ``test-only``,
|
||||||
and ``run``, but are under the ``multi-jvm`` configuration.
|
and ``run``, but are under the ``multi-jvm`` configuration.
|
||||||
|
|
||||||
So in Akka, to run all the multi-JVM tests in the akka-remote project use (at
|
So in Akka, to run all the multi-JVM tests in the akka-remote project use (at
|
||||||
|
|
@ -111,8 +124,8 @@ options after the test names and ``--``. For example:
|
||||||
Creating application tests
|
Creating application tests
|
||||||
==========================
|
==========================
|
||||||
|
|
||||||
The tests are discovered, and combined, through a naming convention. MultiJvm tests are
|
The tests are discovered, and combined, through a naming convention. MultiJvm test sources
|
||||||
located in ``src/multi-jvm/scala`` directory. A test is named with the following pattern:
|
are located in ``src/multi-jvm/...``. A test is named with the following pattern:
|
||||||
|
|
||||||
.. code-block:: none
|
.. code-block:: none
|
||||||
|
|
||||||
|
|
@ -162,14 +175,26 @@ spawned, one for each node. It will look like this:
|
||||||
[success] Total time: ...
|
[success] Total time: ...
|
||||||
|
|
||||||
|
|
||||||
Naming
|
Changing Defaults
|
||||||
======
|
=================
|
||||||
|
|
||||||
|
You can chenge the name of the multi-JVM test source directory by adding the following
|
||||||
|
configuration to your project:
|
||||||
|
|
||||||
|
.. code-block:: none
|
||||||
|
|
||||||
|
unmanagedSourceDirectories in MultiJvm <<=
|
||||||
|
Seq(baseDirectory(_ / "src/some_directory_here")).join
|
||||||
|
|
||||||
|
|
||||||
You can change what the ``MultiJvm`` identifier is. For example, to change it to
|
You can change what the ``MultiJvm`` identifier is. For example, to change it to
|
||||||
``ClusterTest`` use the ``multiJvmMarker`` setting::
|
``ClusterTest`` use the ``multiJvmMarker`` setting:
|
||||||
|
|
||||||
|
.. code-block:: none
|
||||||
|
|
||||||
multiJvmMarker in MultiJvm := "ClusterTest"
|
multiJvmMarker in MultiJvm := "ClusterTest"
|
||||||
|
|
||||||
|
|
||||||
Your tests should now be named ``{TestName}ClusterTest{NodeName}``.
|
Your tests should now be named ``{TestName}ClusterTest{NodeName}``.
|
||||||
|
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -108,8 +108,8 @@ Actor Best Practices
|
||||||
#. Top-level actors are the innermost part of your Error Kernel, so create them
|
#. Top-level actors are the innermost part of your Error Kernel, so create them
|
||||||
sparingly and prefer truly hierarchical systems. This has benefits wrt.
|
sparingly and prefer truly hierarchical systems. This has benefits wrt.
|
||||||
fault-handling (both considering the granularity of configuration and the
|
fault-handling (both considering the granularity of configuration and the
|
||||||
performance) and it also reduces the number of blocking calls made, since
|
performance) and it also reduces the strain on the guardian actor, which is
|
||||||
the creation of top-level actors involves synchronous messaging.
|
a single point of contention if over-used.
|
||||||
|
|
||||||
Blocking Needs Careful Management
|
Blocking Needs Careful Management
|
||||||
---------------------------------
|
---------------------------------
|
||||||
|
|
|
||||||
|
|
@ -27,10 +27,11 @@ import akka.testkit.ErrorFilter;
|
||||||
import akka.testkit.EventFilter;
|
import akka.testkit.EventFilter;
|
||||||
import akka.testkit.TestEvent;
|
import akka.testkit.TestEvent;
|
||||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||||
|
import static akka.japi.Util.immutableSeq;
|
||||||
import akka.japi.Function;
|
import akka.japi.Function;
|
||||||
import scala.Option;
|
import scala.Option;
|
||||||
import scala.collection.JavaConverters;
|
import scala.collection.JavaConverters;
|
||||||
import scala.collection.Seq;
|
import scala.collection.immutable.Seq;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
|
|
@ -219,8 +220,7 @@ public class FaultHandlingTestBase {
|
||||||
|
|
||||||
//#testkit
|
//#testkit
|
||||||
public <A> Seq<A> seq(A... args) {
|
public <A> Seq<A> seq(A... args) {
|
||||||
return JavaConverters.collectionAsScalaIterableConverter(
|
return immutableSeq(args);
|
||||||
java.util.Arrays.asList(args)).asScala().toSeq();
|
|
||||||
}
|
}
|
||||||
//#testkit
|
//#testkit
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -351,27 +351,26 @@ public class UntypedActorDocTestBase {
|
||||||
static
|
static
|
||||||
//#stash
|
//#stash
|
||||||
public class ActorWithProtocol extends UntypedActorWithStash {
|
public class ActorWithProtocol extends UntypedActorWithStash {
|
||||||
private Boolean isOpen = false;
|
|
||||||
public void onReceive(Object msg) {
|
public void onReceive(Object msg) {
|
||||||
if (isOpen) {
|
if (msg.equals("open")) {
|
||||||
|
unstashAll();
|
||||||
|
getContext().become(new Procedure<Object>() {
|
||||||
|
public void apply(Object msg) throws Exception {
|
||||||
if (msg.equals("write")) {
|
if (msg.equals("write")) {
|
||||||
// do writing...
|
// do writing...
|
||||||
} else if (msg.equals("close")) {
|
} else if (msg.equals("close")) {
|
||||||
unstashAll();
|
unstashAll();
|
||||||
isOpen = false;
|
getContext().unbecome();
|
||||||
} else {
|
} else {
|
||||||
stash();
|
stash();
|
||||||
}
|
}
|
||||||
} else {
|
}
|
||||||
if (msg.equals("open")) {
|
}, false); // add behavior on top instead of replacing
|
||||||
unstashAll();
|
|
||||||
isOpen = true;
|
|
||||||
} else {
|
} else {
|
||||||
stash();
|
stash();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
//#stash
|
//#stash
|
||||||
|
|
||||||
static
|
static
|
||||||
|
|
|
||||||
|
|
@ -32,9 +32,9 @@ public class UntypedActorSwapper {
|
||||||
@Override
|
@Override
|
||||||
public void apply(Object message) {
|
public void apply(Object message) {
|
||||||
log.info("Ho");
|
log.info("Ho");
|
||||||
getContext().unbecome(); // resets the latest 'become' (just for fun)
|
getContext().unbecome(); // resets the latest 'become'
|
||||||
}
|
}
|
||||||
});
|
}, false); // this signals stacking of the new behavior
|
||||||
} else {
|
} else {
|
||||||
unhandled(message);
|
unhandled(message);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -119,5 +119,4 @@ public class LoggingDocTestBase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//#deadletter-actor
|
//#deadletter-actor
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import static docs.jrouting.CustomRouterDocTestBase.Message.RepublicanVote;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
|
@ -69,7 +70,7 @@ public class CustomRouterDocTestBase {
|
||||||
//#supervision
|
//#supervision
|
||||||
final SupervisorStrategy strategy =
|
final SupervisorStrategy strategy =
|
||||||
new OneForOneStrategy(5, Duration.create("1 minute"),
|
new OneForOneStrategy(5, Duration.create("1 minute"),
|
||||||
new Class<?>[] { Exception.class });
|
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
||||||
final ActorRef router = system.actorOf(new Props(MyActor.class)
|
final ActorRef router = system.actorOf(new Props(MyActor.class)
|
||||||
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
||||||
//#supervision
|
//#supervision
|
||||||
|
|
@ -179,16 +180,14 @@ public class CustomRouterDocTestBase {
|
||||||
//#crRoutingLogic
|
//#crRoutingLogic
|
||||||
return new CustomRoute() {
|
return new CustomRoute() {
|
||||||
@Override
|
@Override
|
||||||
public Iterable<Destination> destinationsFor(ActorRef sender, Object msg) {
|
public scala.collection.immutable.Seq<Destination> destinationsFor(ActorRef sender, Object msg) {
|
||||||
switch ((Message) msg) {
|
switch ((Message) msg) {
|
||||||
case DemocratVote:
|
case DemocratVote:
|
||||||
case DemocratCountResult:
|
case DemocratCountResult:
|
||||||
return Arrays.asList(
|
return akka.japi.Util.immutableSingletonSeq(new Destination(sender, democratActor));
|
||||||
new Destination[] { new Destination(sender, democratActor) });
|
|
||||||
case RepublicanVote:
|
case RepublicanVote:
|
||||||
case RepublicanCountResult:
|
case RepublicanCountResult:
|
||||||
return Arrays.asList(
|
return akka.japi.Util.immutableSingletonSeq(new Destination(sender, republicanActor));
|
||||||
new Destination[] { new Destination(sender, republicanActor) });
|
|
||||||
default:
|
default:
|
||||||
throw new IllegalArgumentException("Unknown message: " + msg);
|
throw new IllegalArgumentException("Unknown message: " + msg);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -549,7 +549,8 @@ Upgrade
|
||||||
|
|
||||||
Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at
|
Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at
|
||||||
runtime. Use the ``getContext().become`` method from within the Actor.
|
runtime. Use the ``getContext().become`` method from within the Actor.
|
||||||
The hotswapped code is kept in a Stack which can be pushed and popped.
|
The hotswapped code is kept in a Stack which can be pushed (replacing or adding
|
||||||
|
at the top) and popped.
|
||||||
|
|
||||||
.. warning::
|
.. warning::
|
||||||
|
|
||||||
|
|
@ -563,26 +564,19 @@ To hotswap the Actor using ``getContext().become``:
|
||||||
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
.. includecode:: code/docs/actor/UntypedActorDocTestBase.java
|
||||||
:include: hot-swap-actor
|
:include: hot-swap-actor
|
||||||
|
|
||||||
The ``become`` method is useful for many different things, such as to implement
|
This variant of the :meth:`become` method is useful for many different things,
|
||||||
a Finite State Machine (FSM).
|
such as to implement a Finite State Machine (FSM). It will replace the current
|
||||||
|
behavior (i.e. the top of the behavior stack), which means that you do not use
|
||||||
|
:meth:`unbecome`, instead always the next behavior is explicitly installed.
|
||||||
|
|
||||||
Here is another little cute example of ``become`` and ``unbecome`` in action:
|
The other way of using :meth:`become` does not replace but add to the top of
|
||||||
|
the behavior stack. In this case care must be taken to ensure that the number
|
||||||
|
of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones
|
||||||
|
in the long run, otherwise this amounts to a memory leak (which is why this
|
||||||
|
behavior is not the default).
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper
|
.. includecode:: code/docs/actor/UntypedActorSwapper.java#swapper
|
||||||
|
|
||||||
Downgrade
|
|
||||||
---------
|
|
||||||
|
|
||||||
Since the hotswapped code is pushed to a Stack you can downgrade the code as
|
|
||||||
well. Use the ``getContext().unbecome`` method from within the Actor.
|
|
||||||
|
|
||||||
.. code-block:: java
|
|
||||||
|
|
||||||
public void onReceive(Object message) {
|
|
||||||
if (message.equals("revert")) getContext().unbecome();
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
Stash
|
Stash
|
||||||
=====
|
=====
|
||||||
|
|
||||||
|
|
@ -627,9 +621,11 @@ The stash is backed by a ``scala.collection.immutable.Vector``. As a
|
||||||
result, even a very large number of messages may be stashed without a
|
result, even a very large number of messages may be stashed without a
|
||||||
major impact on performance.
|
major impact on performance.
|
||||||
|
|
||||||
Note that the stash is not persisted across restarts of an actor,
|
Note that the stash is part of the ephemeral actor state, unlike the
|
||||||
unlike the actor's mailbox. Therefore, it should be managed like other
|
mailbox. Therefore, it should be managed like other parts of the
|
||||||
parts of the actor's state which have the same property.
|
actor's state which have the same property. The :class:`Stash` trait’s
|
||||||
|
implementation of :meth:`preRestart` will call ``unstashAll()``, which is
|
||||||
|
usually the desired behavior.
|
||||||
|
|
||||||
|
|
||||||
Killing an Actor
|
Killing an Actor
|
||||||
|
|
|
||||||
26
akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst
Normal file
26
akka-docs/rst/project/migration-guide-2.1.x-2.2.x.rst
Normal file
|
|
@ -0,0 +1,26 @@
|
||||||
|
.. _migration-2.2:
|
||||||
|
|
||||||
|
################################
|
||||||
|
Migration Guide 2.1.x to 2.2.x
|
||||||
|
################################
|
||||||
|
|
||||||
|
The 2.2 release contains several structural changes that require some
|
||||||
|
simple, mechanical source-level changes in client code.
|
||||||
|
|
||||||
|
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
|
||||||
|
migrating `1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_ and then :ref:`2.0.x to 2.1.x <migration-2.1>`.
|
||||||
|
|
||||||
|
Immutable everywhere
|
||||||
|
====================
|
||||||
|
|
||||||
|
Akka has in 2.2 been refactored to require ``scala.collection.immutable`` data structures as much as possible,
|
||||||
|
this leads to fewer bugs and more opportunity for sharing data safely.
|
||||||
|
|
||||||
|
==================================== ====================================
|
||||||
|
Search Replace with
|
||||||
|
==================================== ====================================
|
||||||
|
``akka.japi.Util.arrayToSeq`` ``akka.japi.Util.immutableSeq``
|
||||||
|
==================================== ====================================
|
||||||
|
|
||||||
|
If you need to convert from Java to ``scala.collection.immutable.Seq`` or ``scala.collection.immutable.Iterable`` you should use ``akka.japi.Util.immutableSeq(…)``,
|
||||||
|
and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.<collection-type>]`` method.
|
||||||
|
|
@ -8,3 +8,4 @@ Migration Guides
|
||||||
|
|
||||||
migration-guide-1.3.x-2.0.x
|
migration-guide-1.3.x-2.0.x
|
||||||
migration-guide-2.0.x-2.1.x
|
migration-guide-2.0.x-2.1.x
|
||||||
|
migration-guide-2.1.x-2.2.x
|
||||||
|
|
|
||||||
|
|
@ -174,6 +174,17 @@ form of the ``implicit val context: ActorContext``. Outside of an actor, you
|
||||||
have to either declare an implicit :class:`ActorSystem`, or you can give the
|
have to either declare an implicit :class:`ActorSystem`, or you can give the
|
||||||
factory explicitly (see further below).
|
factory explicitly (see further below).
|
||||||
|
|
||||||
|
The two possible ways of issuing a ``context.become`` (replacing or adding the
|
||||||
|
new behavior) are offered separately to enable a clutter-free notation of
|
||||||
|
nested receives:
|
||||||
|
|
||||||
|
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#becomeStacked
|
||||||
|
|
||||||
|
Please note that calling ``unbecome`` more often than ``becomeStacked`` results
|
||||||
|
in the original behavior being installed, which in case of the :class:`Act`
|
||||||
|
trait is the empty behavior (the outer ``become`` just replaces it during
|
||||||
|
construction).
|
||||||
|
|
||||||
Life-cycle hooks are also exposed as DSL elements (see `Start Hook`_ and `Stop
|
Life-cycle hooks are also exposed as DSL elements (see `Start Hook`_ and `Stop
|
||||||
Hook`_ below), where later invocations of the methods shown below will replace
|
Hook`_ below), where later invocations of the methods shown below will replace
|
||||||
the contents of the respective hooks:
|
the contents of the respective hooks:
|
||||||
|
|
@ -653,11 +664,10 @@ Upgrade
|
||||||
-------
|
-------
|
||||||
|
|
||||||
Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at
|
Akka supports hotswapping the Actor’s message loop (e.g. its implementation) at
|
||||||
runtime: Invoke the ``context.become`` method from within the Actor.
|
runtime: invoke the ``context.become`` method from within the Actor.
|
||||||
|
:meth:`become` takes a ``PartialFunction[Any, Unit]`` that implements the new
|
||||||
Become takes a ``PartialFunction[Any, Unit]`` that implements
|
message handler. The hotswapped code is kept in a Stack which can be pushed and
|
||||||
the new message handler. The hotswapped code is kept in a Stack which can be
|
popped.
|
||||||
pushed and popped.
|
|
||||||
|
|
||||||
.. warning::
|
.. warning::
|
||||||
|
|
||||||
|
|
@ -667,38 +677,26 @@ To hotswap the Actor behavior using ``become``:
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#hot-swap-actor
|
||||||
|
|
||||||
The ``become`` method is useful for many different things, but a particular nice
|
This variant of the :meth:`become` method is useful for many different things,
|
||||||
example of it is in example where it is used to implement a Finite State Machine
|
such as to implement a Finite State Machine (FSM, for an example see `Dining
|
||||||
(FSM): `Dining Hakkers`_.
|
Hakkers`_). It will replace the current behavior (i.e. the top of the behavior
|
||||||
|
stack), which means that you do not use :meth:`unbecome`, instead always the
|
||||||
|
next behavior is explicitly installed.
|
||||||
|
|
||||||
.. _Dining Hakkers: @github@/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala
|
.. _Dining Hakkers: @github@/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala
|
||||||
|
|
||||||
Here is another little cute example of ``become`` and ``unbecome`` in action:
|
The other way of using :meth:`become` does not replace but add to the top of
|
||||||
|
the behavior stack. In this case care must be taken to ensure that the number
|
||||||
|
of “pop” operations (i.e. :meth:`unbecome`) matches the number of “push” ones
|
||||||
|
in the long run, otherwise this amounts to a memory leak (which is why this
|
||||||
|
behavior is not the default).
|
||||||
|
|
||||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#swapper
|
.. includecode:: code/docs/actor/ActorDocSpec.scala#swapper
|
||||||
|
|
||||||
Encoding Scala Actors nested receives without accidentally leaking memory
|
Encoding Scala Actors nested receives without accidentally leaking memory
|
||||||
-------------------------------------------------------------------------
|
-------------------------------------------------------------------------
|
||||||
|
|
||||||
See this `Unnested receive example <https://github.com/akka/akka/blob/master/akka-docs/scala/code/docs/actor/UnnestedReceives.scala>`_.
|
See this `Unnested receive example <@github@/akka-docs/rst/scala/code/docs/actor/UnnestedReceives.scala>`_.
|
||||||
|
|
||||||
|
|
||||||
Downgrade
|
|
||||||
---------
|
|
||||||
|
|
||||||
Since the hotswapped code is pushed to a Stack you can downgrade the code as
|
|
||||||
well, all you need to do is to: Invoke the ``context.unbecome`` method from within the Actor.
|
|
||||||
|
|
||||||
This will pop the Stack and replace the Actor's implementation with the
|
|
||||||
``PartialFunction[Any, Unit]`` that is at the top of the Stack.
|
|
||||||
|
|
||||||
Here's how you use the ``unbecome`` method:
|
|
||||||
|
|
||||||
.. code-block:: scala
|
|
||||||
|
|
||||||
def receive = {
|
|
||||||
case "revert" => context.unbecome()
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
Stash
|
Stash
|
||||||
|
|
@ -752,9 +750,11 @@ major impact on performance.
|
||||||
callback. This means it's not possible to write
|
callback. This means it's not possible to write
|
||||||
``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``.
|
``Actor with MyActor with Stash`` if ``MyActor`` overrides ``preRestart``.
|
||||||
|
|
||||||
Note that the stash is not persisted across restarts of an actor,
|
Note that the stash is part of the ephemeral actor state, unlike the
|
||||||
unlike the actor's mailbox. Therefore, it should be managed like other
|
mailbox. Therefore, it should be managed like other parts of the
|
||||||
parts of the actor's state which have the same property.
|
actor's state which have the same property. The :class:`Stash` trait’s
|
||||||
|
implementation of :meth:`preRestart` will call ``unstashAll()``, which is
|
||||||
|
usually the desired behavior.
|
||||||
|
|
||||||
|
|
||||||
Killing an Actor
|
Killing an Actor
|
||||||
|
|
|
||||||
|
|
@ -96,11 +96,11 @@ class Swapper extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case Swap ⇒
|
case Swap ⇒
|
||||||
log.info("Hi")
|
log.info("Hi")
|
||||||
become {
|
become({
|
||||||
case Swap ⇒
|
case Swap ⇒
|
||||||
log.info("Ho")
|
log.info("Ho")
|
||||||
unbecome() // resets the latest 'become' (just for fun)
|
unbecome() // resets the latest 'become' (just for fun)
|
||||||
}
|
}, discardOld = false) // push on top instead of replace
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -316,13 +316,13 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
||||||
def receive = {
|
def receive = {
|
||||||
case "open" ⇒
|
case "open" ⇒
|
||||||
unstashAll()
|
unstashAll()
|
||||||
context.become {
|
context.become({
|
||||||
case "write" ⇒ // do writing...
|
case "write" ⇒ // do writing...
|
||||||
case "close" ⇒
|
case "close" ⇒
|
||||||
unstashAll()
|
unstashAll()
|
||||||
context.unbecome()
|
context.unbecome()
|
||||||
case msg ⇒ stash()
|
case msg ⇒ stash()
|
||||||
}
|
}, discardOld = false) // stack on top instead of replacing
|
||||||
case msg ⇒ stash()
|
case msg ⇒ stash()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import language.postfixOps
|
||||||
import akka.testkit.{ AkkaSpec ⇒ MyFavoriteTestFrameWorkPlusAkkaTestKit }
|
import akka.testkit.{ AkkaSpec ⇒ MyFavoriteTestFrameWorkPlusAkkaTestKit }
|
||||||
//#test-code
|
//#test-code
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
|
|
||||||
|
|
@ -24,7 +25,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
case object Flush
|
case object Flush
|
||||||
|
|
||||||
// sent events
|
// sent events
|
||||||
case class Batch(obj: Seq[Any])
|
case class Batch(obj: immutable.Seq[Any])
|
||||||
//#simple-events
|
//#simple-events
|
||||||
//#simple-state
|
//#simple-state
|
||||||
// states
|
// states
|
||||||
|
|
@ -34,7 +35,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
|
|
||||||
sealed trait Data
|
sealed trait Data
|
||||||
case object Uninitialized extends Data
|
case object Uninitialized extends Data
|
||||||
case class Todo(target: ActorRef, queue: Seq[Any]) extends Data
|
case class Todo(target: ActorRef, queue: immutable.Seq[Any]) extends Data
|
||||||
//#simple-state
|
//#simple-state
|
||||||
//#simple-fsm
|
//#simple-fsm
|
||||||
class Buncher extends Actor with FSM[State, Data] {
|
class Buncher extends Actor with FSM[State, Data] {
|
||||||
|
|
@ -193,12 +194,12 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
|
||||||
buncher ! SetTarget(testActor)
|
buncher ! SetTarget(testActor)
|
||||||
buncher ! Queue(42)
|
buncher ! Queue(42)
|
||||||
buncher ! Queue(43)
|
buncher ! Queue(43)
|
||||||
expectMsg(Batch(Seq(42, 43)))
|
expectMsg(Batch(immutable.Seq(42, 43)))
|
||||||
buncher ! Queue(44)
|
buncher ! Queue(44)
|
||||||
buncher ! Flush
|
buncher ! Flush
|
||||||
buncher ! Queue(45)
|
buncher ! Queue(45)
|
||||||
expectMsg(Batch(Seq(44)))
|
expectMsg(Batch(immutable.Seq(44)))
|
||||||
expectMsg(Batch(Seq(45)))
|
expectMsg(Batch(immutable.Seq(45)))
|
||||||
}
|
}
|
||||||
|
|
||||||
"batch not if uninitialized" in {
|
"batch not if uninitialized" in {
|
||||||
|
|
|
||||||
|
|
@ -44,21 +44,23 @@ class DataflowDocSpec extends WordSpec with MustMatchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"demonstrate the use of dataflow variables" in {
|
"demonstrate the use of dataflow variables" in {
|
||||||
def println[T](any: Try[T]): Unit = any.get must be === 20
|
val result = Promise[Int]()
|
||||||
|
def println(any: Try[Int]): Unit = result.complete(any)
|
||||||
//#dataflow-variable-a
|
//#dataflow-variable-a
|
||||||
flow {
|
|
||||||
val v1, v2 = Promise[Int]()
|
val v1, v2 = Promise[Int]()
|
||||||
|
flow {
|
||||||
// v1 will become the value of v2 + 10 when v2 gets a value
|
// v1 will become the value of v2 + 10 when v2 gets a value
|
||||||
v1 << v2() + 10
|
v1 << v2() + 10
|
||||||
v2 << flow { 5 } // As you can see, no blocking!
|
|
||||||
v1() + v2()
|
v1() + v2()
|
||||||
} onComplete println
|
} onComplete println
|
||||||
|
flow { v2 << 5 } // As you can see, no blocking above!
|
||||||
//#dataflow-variable-a
|
//#dataflow-variable-a
|
||||||
|
Await.result(result.future, 10.seconds) must be === 20
|
||||||
}
|
}
|
||||||
|
|
||||||
"demonstrate the difference between for and flow" in {
|
"demonstrate the difference between for and flow" in {
|
||||||
def println[T](any: Try[T]): Unit = any.get must be === 2
|
val result = Promise[Int]()
|
||||||
|
def println(any: Try[Int]): Unit = result.tryComplete(any)
|
||||||
//#for-vs-flow
|
//#for-vs-flow
|
||||||
val f1, f2 = Future { 1 }
|
val f1, f2 = Future { 1 }
|
||||||
|
|
||||||
|
|
@ -68,6 +70,7 @@ class DataflowDocSpec extends WordSpec with MustMatchers {
|
||||||
usingFor onComplete println
|
usingFor onComplete println
|
||||||
usingFlow onComplete println
|
usingFlow onComplete println
|
||||||
//#for-vs-flow
|
//#for-vs-flow
|
||||||
|
Await.result(result.future, 10.seconds) must be === 2
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ import akka.testkit.DefaultTimeout
|
||||||
import akka.testkit.ImplicitSender
|
import akka.testkit.ImplicitSender
|
||||||
import akka.testkit.TestKit
|
import akka.testkit.TestKit
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* a Test to show some TestKit examples
|
* a Test to show some TestKit examples
|
||||||
|
|
@ -38,8 +39,8 @@ class TestKitUsageSpec
|
||||||
val filterRef = system.actorOf(Props(new FilteringActor(testActor)))
|
val filterRef = system.actorOf(Props(new FilteringActor(testActor)))
|
||||||
val randomHead = Random.nextInt(6)
|
val randomHead = Random.nextInt(6)
|
||||||
val randomTail = Random.nextInt(10)
|
val randomTail = Random.nextInt(10)
|
||||||
val headList = Seq().padTo(randomHead, "0")
|
val headList = immutable.Seq().padTo(randomHead, "0")
|
||||||
val tailList = Seq().padTo(randomTail, "1")
|
val tailList = immutable.Seq().padTo(randomTail, "1")
|
||||||
val seqRef =
|
val seqRef =
|
||||||
system.actorOf(Props(new SequencingActor(testActor, headList, tailList)))
|
system.actorOf(Props(new SequencingActor(testActor, headList, tailList)))
|
||||||
|
|
||||||
|
|
@ -145,7 +146,7 @@ object TestKitUsageSpec {
|
||||||
* like to test that the interesting value is received and that you cant
|
* like to test that the interesting value is received and that you cant
|
||||||
* be bothered with the rest
|
* be bothered with the rest
|
||||||
*/
|
*/
|
||||||
class SequencingActor(next: ActorRef, head: Seq[String], tail: Seq[String])
|
class SequencingActor(next: ActorRef, head: immutable.Seq[String], tail: immutable.Seq[String])
|
||||||
extends Actor {
|
extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case msg ⇒ {
|
case msg ⇒ {
|
||||||
|
|
|
||||||
|
|
@ -5,13 +5,13 @@ package docs.zeromq
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
|
||||||
import akka.actor.{ Actor, Props }
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import scala.collection.immutable
|
||||||
|
import akka.actor.{ Actor, Props }
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension }
|
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
|
||||||
import java.text.SimpleDateFormat
|
import java.text.SimpleDateFormat
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
import akka.zeromq.{ SocketType, Bind }
|
|
||||||
|
|
||||||
object ZeromqDocSpec {
|
object ZeromqDocSpec {
|
||||||
|
|
||||||
|
|
@ -52,12 +52,12 @@ object ZeromqDocSpec {
|
||||||
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed,
|
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed,
|
||||||
currentHeap.getMax)).get
|
currentHeap.getMax)).get
|
||||||
// the first frame is the topic, second is the message
|
// the first frame is the topic, second is the message
|
||||||
pubSocket ! ZMQMessage(Seq(Frame("health.heap"), Frame(heapPayload)))
|
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.heap"), Frame(heapPayload)))
|
||||||
|
|
||||||
// use akka SerializationExtension to convert to bytes
|
// use akka SerializationExtension to convert to bytes
|
||||||
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get
|
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get
|
||||||
// the first frame is the topic, second is the message
|
// the first frame is the topic, second is the message
|
||||||
pubSocket ! ZMQMessage(Seq(Frame("health.load"), Frame(loadPayload)))
|
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.load"), Frame(loadPayload)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
//#health
|
//#health
|
||||||
|
|
@ -146,7 +146,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
|
||||||
|
|
||||||
val payload = Array.empty[Byte]
|
val payload = Array.empty[Byte]
|
||||||
//#pub-topic
|
//#pub-topic
|
||||||
pubSocket ! ZMQMessage(Seq(Frame("foo.bar"), Frame(payload)))
|
pubSocket ! ZMQMessage(Frame("foo.bar"), Frame(payload))
|
||||||
//#pub-topic
|
//#pub-topic
|
||||||
|
|
||||||
system.stop(subSocket)
|
system.stop(subSocket)
|
||||||
|
|
|
||||||
|
|
@ -124,6 +124,14 @@ obvious that an actor is actually created:
|
||||||
:include: simple-fsm
|
:include: simple-fsm
|
||||||
:exclude: fsm-body
|
:exclude: fsm-body
|
||||||
|
|
||||||
|
.. note::
|
||||||
|
|
||||||
|
The FSM trait defines a ``receive`` method which handles internal messages
|
||||||
|
and passes everything else through to the FSM logic (according to the
|
||||||
|
current state). When overriding the ``receive`` method, keep in mind that
|
||||||
|
e.g. state timeout handling depends on actually passing the messages through
|
||||||
|
the FSM logic.
|
||||||
|
|
||||||
The :class:`FSM` trait takes two type parameters:
|
The :class:`FSM` trait takes two type parameters:
|
||||||
|
|
||||||
#. the supertype of all state names, usually a sealed trait with case objects
|
#. the supertype of all state names, usually a sealed trait with case objects
|
||||||
|
|
|
||||||
|
|
@ -713,8 +713,8 @@ Some `Specs2 <http://specs2.org>`_ users have contributed examples of how to wor
|
||||||
actually beneficial also for the third point—is to apply the TestKit together
|
actually beneficial also for the third point—is to apply the TestKit together
|
||||||
with :class:`org.specs2.specification.Scope`.
|
with :class:`org.specs2.specification.Scope`.
|
||||||
* The Specification traits provide a :class:`Duration` DSL which uses partly
|
* The Specification traits provide a :class:`Duration` DSL which uses partly
|
||||||
the same method names as :class:`scala.concurrent.util.Duration`, resulting in ambiguous
|
the same method names as :class:`scala.concurrent.duration.Duration`, resulting in ambiguous
|
||||||
implicits if ``akka.util.duration._`` is imported. There are two work-arounds:
|
implicits if ``scala.concurrent.duration._`` is imported. There are two work-arounds:
|
||||||
|
|
||||||
* either use the Specification variant of Duration and supply an implicit
|
* either use the Specification variant of Duration and supply an implicit
|
||||||
conversion to the Akka Duration. This conversion is not supplied with the
|
conversion to the Akka Duration. This conversion is not supplied with the
|
||||||
|
|
|
||||||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue