Merge branch 'master' of github.com:jboner/akka into wip-distributed-tests
This commit is contained in:
commit
a7fc87a9d1
149 changed files with 2313 additions and 20244 deletions
1
.gitignore
vendored
1
.gitignore
vendored
|
|
@ -11,6 +11,7 @@ lib_managed
|
|||
etags
|
||||
tags
|
||||
.tags
|
||||
.tags_sorted_by_file
|
||||
TAGS
|
||||
akka.tmproj
|
||||
reports
|
||||
|
|
|
|||
15609
.tags_sorted_by_file
15609
.tags_sorted_by_file
File diff suppressed because it is too large
Load diff
|
|
@ -5,6 +5,18 @@ import akka.dispatch.UnboundedMailbox
|
|||
import akka.util.duration._
|
||||
|
||||
object ConsistencySpec {
|
||||
val config = """
|
||||
consistency-dispatcher {
|
||||
throughput = 1
|
||||
keep-alive-time = 1 ms
|
||||
core-pool-size-min = 10
|
||||
core-pool-size-max = 10
|
||||
max-pool-size-min = 10
|
||||
max-pool-size-max = 10
|
||||
task-queue-type = array
|
||||
task-queue-size = 7
|
||||
}
|
||||
"""
|
||||
class CacheMisaligned(var value: Long, var padding1: Long, var padding2: Long, var padding3: Int) //Vars, no final fences
|
||||
|
||||
class ConsistencyCheckingActor extends Actor {
|
||||
|
|
@ -31,22 +43,12 @@ object ConsistencySpec {
|
|||
}
|
||||
}
|
||||
|
||||
class ConsistencySpec extends AkkaSpec {
|
||||
class ConsistencySpec extends AkkaSpec(ConsistencySpec.config) {
|
||||
import ConsistencySpec._
|
||||
"The Akka actor model implementation" must {
|
||||
"provide memory consistency" in {
|
||||
val noOfActors = 7
|
||||
val dispatcher = system
|
||||
.dispatcherFactory
|
||||
.newDispatcher("consistency-dispatcher", 1, UnboundedMailbox())
|
||||
.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(noOfActors, true)
|
||||
.setCorePoolSize(10)
|
||||
.setMaxPoolSize(10)
|
||||
.setKeepAliveTimeInMillis(1)
|
||||
.setAllowCoreThreadTimeout(true)
|
||||
.build
|
||||
|
||||
val props = Props[ConsistencyCheckingActor].withDispatcher(dispatcher)
|
||||
val props = Props[ConsistencyCheckingActor].withDispatcher("consistency-dispatcher")
|
||||
val actors = Vector.fill(noOfActors)(system.actorOf(props))
|
||||
|
||||
for (i ← 0L until 600000L) {
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.testkit.AkkaSpec
|
|||
import com.typesafe.config.ConfigFactory
|
||||
import com.typesafe.config.ConfigParseOptions
|
||||
import akka.routing._
|
||||
import akka.util.duration._
|
||||
|
||||
object DeployerSpec {
|
||||
val deployerConf = ConfigFactory.parseString("""
|
||||
|
|
@ -35,6 +36,7 @@ object DeployerSpec {
|
|||
}
|
||||
/user/service-scatter-gather {
|
||||
router = scatter-gather
|
||||
within = 2 seconds
|
||||
}
|
||||
}
|
||||
""", ConfigParseOptions.defaults)
|
||||
|
|
@ -116,7 +118,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with scatter-gather router" in {
|
||||
assertRouting(ScatterGatherFirstCompletedRouter(1), "/user/service-scatter-gather")
|
||||
assertRouting(ScatterGatherFirstCompletedRouter(nrOfInstances = 1, within = 2 seconds), "/user/service-scatter-gather")
|
||||
}
|
||||
|
||||
def assertRouting(expected: RouterConfig, service: String) {
|
||||
|
|
|
|||
|
|
@ -9,8 +9,18 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.DefaultTimeout
|
||||
|
||||
object SupervisorMiscSpec {
|
||||
val config = """
|
||||
pinned-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
test-dispatcher {
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class SupervisorMiscSpec extends AkkaSpec with DefaultTimeout {
|
||||
class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with DefaultTimeout {
|
||||
|
||||
"A Supervisor" must {
|
||||
|
||||
|
|
@ -28,11 +38,11 @@ class SupervisorMiscSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
})
|
||||
|
||||
val actor1, actor2 = Await.result((supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).mapTo[ActorRef], timeout.duration)
|
||||
val actor1, actor2 = Await.result((supervisor ? workerProps.withDispatcher("pinned-dispatcher")).mapTo[ActorRef], timeout.duration)
|
||||
|
||||
val actor3 = Await.result((supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newDispatcher("test").build)).mapTo[ActorRef], timeout.duration)
|
||||
val actor3 = Await.result((supervisor ? workerProps.withDispatcher("test-dispatcher")).mapTo[ActorRef], timeout.duration)
|
||||
|
||||
val actor4 = Await.result((supervisor ? workerProps.withDispatcher(system.dispatcherFactory.newPinnedDispatcher("pinned"))).mapTo[ActorRef], timeout.duration)
|
||||
val actor4 = Await.result((supervisor ? workerProps.withDispatcher("pinned-dispatcher")).mapTo[ActorRef], timeout.duration)
|
||||
|
||||
actor1 ! Kill
|
||||
actor2 ! Kill
|
||||
|
|
|
|||
|
|
@ -21,6 +21,16 @@ import akka.dispatch.{ Await, Dispatchers, Future, Promise }
|
|||
|
||||
object TypedActorSpec {
|
||||
|
||||
val config = """
|
||||
pooled-dispatcher {
|
||||
type = BalancingDispatcher
|
||||
core-pool-size-min = 60
|
||||
core-pool-size-max = 60
|
||||
max-pool-size-min = 60
|
||||
max-pool-size-max = 60
|
||||
}
|
||||
"""
|
||||
|
||||
class CyclicIterator[T](val items: Seq[T]) extends Iterator[T] {
|
||||
|
||||
private[this] val current: AtomicReference[Seq[T]] = new AtomicReference(items)
|
||||
|
|
@ -161,7 +171,8 @@ object TypedActorSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfterAll with DefaultTimeout {
|
||||
class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
|
||||
with BeforeAndAfterEach with BeforeAndAfterAll with DefaultTimeout {
|
||||
|
||||
import TypedActorSpec._
|
||||
|
||||
|
|
@ -336,13 +347,7 @@ class TypedActorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte
|
|||
}
|
||||
|
||||
"be able to use balancing dispatcher" in {
|
||||
val props = Props(
|
||||
timeout = Timeout(6600),
|
||||
dispatcher = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(60)
|
||||
.setMaxPoolSize(60)
|
||||
.build)
|
||||
val props = Props(timeout = Timeout(6600), dispatcher = "pooled-dispatcher")
|
||||
|
||||
val thais = for (i ← 1 to 60) yield newFooBar(props)
|
||||
val iterator = new CyclicIterator(thais)
|
||||
|
|
|
|||
|
|
@ -17,6 +17,9 @@ import util.control.NoStackTrace
|
|||
import akka.actor.ActorSystem
|
||||
import akka.util.duration._
|
||||
import akka.event.Logging.Error
|
||||
import com.typesafe.config.Config
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.util.Duration
|
||||
|
||||
object ActorModelSpec {
|
||||
|
||||
|
|
@ -224,21 +227,26 @@ object ActorModelSpec {
|
|||
}
|
||||
}
|
||||
|
||||
abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
||||
abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with DefaultTimeout {
|
||||
|
||||
import ActorModelSpec._
|
||||
|
||||
def newTestActor(dispatcher: MessageDispatcher) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
// FIXME Remove these settings as part of ticket #1563
|
||||
val DispatcherThroughput = system.settings.config.getInt("akka.actor.default-dispatcher.throughput")
|
||||
val DispatcherDefaultShutdown = Duration(system.settings.config.getMilliseconds("akka.actor.default-dispatcher.shutdown-timeout"), TimeUnit.MILLISECONDS)
|
||||
val DispatcherThroughputDeadlineTime = Duration(system.settings.config.getNanoseconds("akka.actor.default-dispatcher.throughput-deadline-time"), TimeUnit.NANOSECONDS)
|
||||
|
||||
protected def newInterceptedDispatcher: MessageDispatcherInterceptor
|
||||
def newTestActor(dispatcher: String) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||
|
||||
protected def registerInterceptedDispatcher(): MessageDispatcherInterceptor
|
||||
protected def dispatcherType: String
|
||||
|
||||
"A " + dispatcherType must {
|
||||
|
||||
"must dynamically handle its own life cycle" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
assertDispatcher(dispatcher)(stops = 0)
|
||||
val a = newTestActor(dispatcher)
|
||||
val a = newTestActor(dispatcher.id)
|
||||
assertDispatcher(dispatcher)(stops = 0)
|
||||
system.stop(a)
|
||||
assertDispatcher(dispatcher)(stops = 1)
|
||||
|
|
@ -256,7 +264,7 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
assertDispatcher(dispatcher)(stops = 2)
|
||||
|
||||
val a2 = newTestActor(dispatcher)
|
||||
val a2 = newTestActor(dispatcher.id)
|
||||
val futures2 = for (i ← 1 to 10) yield Future { i }
|
||||
|
||||
assertDispatcher(dispatcher)(stops = 2)
|
||||
|
|
@ -266,9 +274,9 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"process messages one at a time" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val start, oneAtATime = new CountDownLatch(1)
|
||||
val a = newTestActor(dispatcher)
|
||||
val a = newTestActor(dispatcher.id)
|
||||
|
||||
a ! CountDown(start)
|
||||
assertCountDown(start, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds")
|
||||
|
|
@ -285,9 +293,9 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"handle queueing from multiple threads" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val counter = new CountDownLatch(200)
|
||||
val a = newTestActor(dispatcher)
|
||||
val a = newTestActor(dispatcher.id)
|
||||
|
||||
for (i ← 1 to 10) {
|
||||
spawn {
|
||||
|
|
@ -316,8 +324,8 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"not process messages for a suspended actor" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
val a = newTestActor(dispatcher).asInstanceOf[LocalActorRef]
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val a = newTestActor(dispatcher.id).asInstanceOf[LocalActorRef]
|
||||
val done = new CountDownLatch(1)
|
||||
a.suspend
|
||||
a ! CountDown(done)
|
||||
|
|
@ -335,8 +343,8 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
|
||||
"handle waves of actors" in {
|
||||
val dispatcher = newInterceptedDispatcher
|
||||
val props = Props[DispatcherActor].withDispatcher(dispatcher)
|
||||
val dispatcher = registerInterceptedDispatcher()
|
||||
val props = Props[DispatcherActor].withDispatcher(dispatcher.id)
|
||||
|
||||
def flood(num: Int) {
|
||||
val cachedMessage = CountDownNStop(new CountDownLatch(num))
|
||||
|
|
@ -347,7 +355,7 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
case "run" ⇒ for (_ ← 1 to num) (context.watch(context.actorOf(props))) ! cachedMessage
|
||||
case Terminated(child) ⇒ stopLatch.countDown()
|
||||
}
|
||||
}).withDispatcher(system.dispatcherFactory.newPinnedDispatcher("boss")))
|
||||
}).withDispatcher("boss"))
|
||||
boss ! "run"
|
||||
try {
|
||||
assertCountDown(cachedMessage.latch, waitTime, "Counting down from " + num)
|
||||
|
|
@ -381,9 +389,9 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
"continue to process messages when a thread gets interrupted" in {
|
||||
filterEvents(EventFilter[InterruptedException](), EventFilter[akka.event.Logging.EventHandlerException]()) {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
val a = newTestActor(dispatcher)
|
||||
val a = newTestActor(dispatcher.id)
|
||||
val f1 = a ? Reply("foo")
|
||||
val f2 = a ? Reply("bar")
|
||||
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) }
|
||||
|
|
@ -402,8 +410,8 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
"continue to process messages when exception is thrown" in {
|
||||
filterEvents(EventFilter[IndexOutOfBoundsException](), EventFilter[RemoteException]()) {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
val a = newTestActor(dispatcher)
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val a = newTestActor(dispatcher.id)
|
||||
val f1 = a ? Reply("foo")
|
||||
val f2 = a ? Reply("bar")
|
||||
val f3 = a ? ThrowException(new IndexOutOfBoundsException("IndexOutOfBoundsException"))
|
||||
|
|
@ -422,23 +430,47 @@ abstract class ActorModelSpec extends AkkaSpec with DefaultTimeout {
|
|||
}
|
||||
}
|
||||
|
||||
object DispatcherModelSpec {
|
||||
val config = """
|
||||
dispatcher {
|
||||
type = Dispatcher
|
||||
}
|
||||
boss {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DispatcherModelSpec extends ActorModelSpec {
|
||||
class DispatcherModelSpec extends ActorModelSpec(DispatcherModelSpec.config) {
|
||||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, "foo", system.settings.DispatcherThroughput,
|
||||
system.settings.DispatcherThroughputDeadlineTime, system.dispatcherFactory.MailboxType,
|
||||
config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
val dispatcherCount = new AtomicInteger()
|
||||
|
||||
def dispatcherType = "Dispatcher"
|
||||
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
|
||||
// use new id for each invocation, since the MessageDispatcherInterceptor holds state
|
||||
val id = "dispatcher-" + dispatcherCount.incrementAndGet()
|
||||
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatchers.prerequisites) {
|
||||
val instance = {
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatchers.prerequisites, id, id, DispatcherThroughput,
|
||||
DispatcherThroughputDeadlineTime, UnboundedMailbox(), config,
|
||||
DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build
|
||||
}
|
||||
override def dispatcher(): MessageDispatcher = instance
|
||||
}
|
||||
system.dispatchers.register(id, dispatcherConfigurator)
|
||||
system.dispatchers.lookup(id).asInstanceOf[MessageDispatcherInterceptor]
|
||||
}
|
||||
|
||||
override def dispatcherType = "Dispatcher"
|
||||
|
||||
"A " + dispatcherType must {
|
||||
"process messages in parallel" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val aStart, aStop, bParallel = new CountDownLatch(1)
|
||||
val a, b = newTestActor(dispatcher)
|
||||
val a, b = newTestActor(dispatcher.id)
|
||||
|
||||
a ! Meet(aStart, aStop)
|
||||
assertCountDown(aStart, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds")
|
||||
|
|
@ -459,23 +491,48 @@ class DispatcherModelSpec extends ActorModelSpec {
|
|||
}
|
||||
}
|
||||
|
||||
object BalancingDispatcherModelSpec {
|
||||
val config = """
|
||||
dispatcher {
|
||||
type = BalancingDispatcher
|
||||
}
|
||||
boss {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class BalancingDispatcherModelSpec extends ActorModelSpec {
|
||||
class BalancingDispatcherModelSpec extends ActorModelSpec(BalancingDispatcherModelSpec.config) {
|
||||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(system.dispatcherFactory.prerequisites, "foo", 1, // TODO check why 1 here? (came from old test)
|
||||
system.settings.DispatcherThroughputDeadlineTime, system.dispatcherFactory.MailboxType,
|
||||
config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build.asInstanceOf[MessageDispatcherInterceptor]
|
||||
val dispatcherCount = new AtomicInteger()
|
||||
|
||||
def dispatcherType = "Balancing Dispatcher"
|
||||
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
|
||||
// use new id for each invocation, since the MessageDispatcherInterceptor holds state
|
||||
val id = "dispatcher-" + dispatcherCount.incrementAndGet()
|
||||
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatchers.prerequisites) {
|
||||
val instance = {
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(system.dispatchers.prerequisites, id, id, 1, // TODO check why 1 here? (came from old test)
|
||||
DispatcherThroughputDeadlineTime, UnboundedMailbox(),
|
||||
config, DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
|
||||
ThreadPoolConfig()).build
|
||||
}
|
||||
|
||||
override def dispatcher(): MessageDispatcher = instance
|
||||
}
|
||||
system.dispatchers.register(id, dispatcherConfigurator)
|
||||
system.dispatchers.lookup(id).asInstanceOf[MessageDispatcherInterceptor]
|
||||
}
|
||||
|
||||
override def dispatcherType = "Balancing Dispatcher"
|
||||
|
||||
"A " + dispatcherType must {
|
||||
"process messages in parallel" in {
|
||||
implicit val dispatcher = newInterceptedDispatcher
|
||||
implicit val dispatcher = registerInterceptedDispatcher()
|
||||
val aStart, aStop, bParallel = new CountDownLatch(1)
|
||||
val a, b = newTestActor(dispatcher)
|
||||
val a, b = newTestActor(dispatcher.id)
|
||||
|
||||
a ! Meet(aStart, aStop)
|
||||
assertCountDown(aStart, 3.seconds.dilated.toMillis, "Should process first message within 3 seconds")
|
||||
|
|
|
|||
|
|
@ -5,12 +5,19 @@ import akka.dispatch.{ Mailbox, Dispatchers }
|
|||
import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props }
|
||||
import akka.testkit.AkkaSpec
|
||||
|
||||
object BalancingDispatcherSpec {
|
||||
val config = """
|
||||
pooled-dispatcher {
|
||||
type = BalancingDispatcher
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class BalancingDispatcherSpec extends AkkaSpec {
|
||||
class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) {
|
||||
|
||||
def newWorkStealer() = system.dispatcherFactory.newBalancingDispatcher("pooled-dispatcher", 1).build
|
||||
|
||||
val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer()
|
||||
val delayableActorDispatcher = "pooled-dispatcher"
|
||||
|
||||
class DelayableActor(delay: Int, finishedCounter: CountDownLatch) extends Actor {
|
||||
@volatile
|
||||
|
|
|
|||
|
|
@ -10,6 +10,22 @@ import akka.testkit.DefaultTimeout
|
|||
import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher }
|
||||
|
||||
object DispatcherActorSpec {
|
||||
val config = """
|
||||
test-dispatcher {
|
||||
}
|
||||
test-throughput-dispatcher {
|
||||
throughput = 101
|
||||
core-pool-size-min = 1
|
||||
core-pool-size-max = 1
|
||||
}
|
||||
test-throughput-deadline-dispatcher {
|
||||
throughput = 2
|
||||
throughput-deadline-time = 100 milliseconds
|
||||
core-pool-size-min = 1
|
||||
core-pool-size-max = 1
|
||||
}
|
||||
|
||||
"""
|
||||
class TestActor extends Actor {
|
||||
def receive = {
|
||||
case "Hello" ⇒ sender ! "World"
|
||||
|
|
@ -28,7 +44,7 @@ object DispatcherActorSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DispatcherActorSpec extends AkkaSpec with DefaultTimeout {
|
||||
class DispatcherActorSpec extends AkkaSpec(DispatcherActorSpec.config) with DefaultTimeout {
|
||||
import DispatcherActorSpec._
|
||||
|
||||
private val unit = TimeUnit.MILLISECONDS
|
||||
|
|
@ -36,23 +52,20 @@ class DispatcherActorSpec extends AkkaSpec with DefaultTimeout {
|
|||
"A Dispatcher and an Actor" must {
|
||||
|
||||
"support tell" in {
|
||||
val actor = system.actorOf(Props[OneWayTestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
|
||||
val actor = system.actorOf(Props[OneWayTestActor].withDispatcher("test-dispatcher"))
|
||||
val result = actor ! "OneWay"
|
||||
assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
|
||||
system.stop(actor)
|
||||
}
|
||||
|
||||
"support ask/reply" in {
|
||||
val actor = system.actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newDispatcher("test").build))
|
||||
val actor = system.actorOf(Props[TestActor].withDispatcher("test-dispatcher"))
|
||||
assert("World" === Await.result(actor ? "Hello", timeout.duration))
|
||||
system.stop(actor)
|
||||
}
|
||||
|
||||
"respect the throughput setting" in {
|
||||
val throughputDispatcher = system.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 101, Duration.Zero, system.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
val throughputDispatcher = "test-throughput-dispatcher"
|
||||
|
||||
val works = new AtomicBoolean(true)
|
||||
val latch = new CountDownLatch(100)
|
||||
|
|
@ -78,10 +91,8 @@ class DispatcherActorSpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
"respect throughput deadline" in {
|
||||
val deadline = 100 millis
|
||||
val throughputDispatcher = system.dispatcherFactory.
|
||||
newDispatcher("THROUGHPUT", 2, deadline, system.dispatcherFactory.MailboxType).
|
||||
setCorePoolSize(1).
|
||||
build
|
||||
val throughputDispatcher = "test-throughput-deadline-dispatcher"
|
||||
|
||||
val works = new AtomicBoolean(true)
|
||||
val latch = new CountDownLatch(1)
|
||||
val start = new CountDownLatch(1)
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ object DispatchersSpec {
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) {
|
||||
|
||||
val df = system.dispatcherFactory
|
||||
val df = system.dispatchers
|
||||
import df._
|
||||
|
||||
val tipe = "type"
|
||||
|
|
@ -31,64 +31,67 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) {
|
|||
val corepoolsizefactor = "core-pool-size-factor"
|
||||
val maxpoolsizefactor = "max-pool-size-factor"
|
||||
val allowcoretimeout = "allow-core-timeout"
|
||||
val throughput = "throughput" // Throughput for Dispatcher
|
||||
val throughput = "throughput"
|
||||
val id = "id"
|
||||
|
||||
def instance(dispatcher: MessageDispatcher): (MessageDispatcher) ⇒ Boolean = _ == dispatcher
|
||||
def ofType[T <: MessageDispatcher: Manifest]: (MessageDispatcher) ⇒ Boolean = _.getClass == manifest[T].erasure
|
||||
|
||||
def typesAndValidators: Map[String, (MessageDispatcher) ⇒ Boolean] = Map(
|
||||
"BalancingDispatcher" -> ofType[BalancingDispatcher],
|
||||
"PinnedDispatcher" -> ofType[PinnedDispatcher],
|
||||
"Dispatcher" -> ofType[Dispatcher])
|
||||
|
||||
def validTypes = typesAndValidators.keys.toList
|
||||
|
||||
val defaultDispatcherConfig = settings.config.getConfig("akka.actor.default-dispatcher")
|
||||
|
||||
lazy val allDispatchers: Map[String, Option[MessageDispatcher]] = {
|
||||
validTypes.map(t ⇒ (t, from(ConfigFactory.parseMap(Map(tipe -> t).asJava).withFallback(defaultDispatcherConfig)))).toMap
|
||||
lazy val allDispatchers: Map[String, MessageDispatcher] = {
|
||||
validTypes.map(t ⇒ (t, from(ConfigFactory.parseMap(Map(tipe -> t, id -> t).asJava).
|
||||
withFallback(defaultDispatcherConfig)))).toMap
|
||||
}
|
||||
|
||||
"Dispatchers" must {
|
||||
|
||||
"use default dispatcher if type is missing" in {
|
||||
val dispatcher = from(ConfigFactory.empty.withFallback(defaultDispatcherConfig))
|
||||
dispatcher.map(_.name) must be(Some("DefaultDispatcher"))
|
||||
}
|
||||
|
||||
"use defined properties" in {
|
||||
val dispatcher = from(ConfigFactory.parseMap(Map("throughput" -> 17).asJava).withFallback(defaultDispatcherConfig))
|
||||
dispatcher.map(_.throughput) must be(Some(17))
|
||||
}
|
||||
|
||||
"use defined properties when newFromConfig" in {
|
||||
val dispatcher = newFromConfig("myapp.mydispatcher")
|
||||
val dispatcher = lookup("myapp.mydispatcher")
|
||||
dispatcher.throughput must be(17)
|
||||
}
|
||||
|
||||
"use specific name when newFromConfig" in {
|
||||
val dispatcher = newFromConfig("myapp.mydispatcher")
|
||||
"use specific name" in {
|
||||
val dispatcher = lookup("myapp.mydispatcher")
|
||||
dispatcher.name must be("mydispatcher")
|
||||
}
|
||||
|
||||
"use default dispatcher when not configured" in {
|
||||
val dispatcher = newFromConfig("myapp.other-dispatcher")
|
||||
"use specific id" in {
|
||||
val dispatcher = lookup("myapp.mydispatcher")
|
||||
dispatcher.id must be("myapp.mydispatcher")
|
||||
}
|
||||
|
||||
"use default dispatcher for missing config" in {
|
||||
val dispatcher = lookup("myapp.other-dispatcher")
|
||||
dispatcher must be === defaultGlobalDispatcher
|
||||
}
|
||||
|
||||
"have only one default dispatcher" in {
|
||||
val dispatcher = lookup(Dispatchers.DefaultDispatcherId)
|
||||
dispatcher must be === defaultGlobalDispatcher
|
||||
dispatcher must be === system.dispatcher
|
||||
}
|
||||
|
||||
"throw IllegalArgumentException if type does not exist" in {
|
||||
intercept[IllegalArgumentException] {
|
||||
from(ConfigFactory.parseMap(Map(tipe -> "typedoesntexist").asJava).withFallback(defaultDispatcherConfig))
|
||||
from(ConfigFactory.parseMap(Map(tipe -> "typedoesntexist", id -> "invalid-dispatcher").asJava).
|
||||
withFallback(defaultDispatcherConfig))
|
||||
}
|
||||
}
|
||||
|
||||
"get the correct types of dispatchers" in {
|
||||
//It can create/obtain all defined types
|
||||
assert(allDispatchers.values.forall(_.isDefined))
|
||||
//All created/obtained dispatchers are of the expeced type/instance
|
||||
assert(typesAndValidators.forall(tuple ⇒ tuple._2(allDispatchers(tuple._1).get)))
|
||||
assert(typesAndValidators.forall(tuple ⇒ tuple._2(allDispatchers(tuple._1))))
|
||||
}
|
||||
|
||||
"provide lookup of dispatchers by key" in {
|
||||
"provide lookup of dispatchers by id" in {
|
||||
val d1 = lookup("myapp.mydispatcher")
|
||||
val d2 = lookup("myapp.mydispatcher")
|
||||
d1 must be === d2
|
||||
|
|
|
|||
|
|
@ -9,6 +9,12 @@ import org.scalatest.BeforeAndAfterEach
|
|||
import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers }
|
||||
|
||||
object PinnedActorSpec {
|
||||
val config = """
|
||||
pinned-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
"""
|
||||
|
||||
class TestActor extends Actor {
|
||||
def receive = {
|
||||
case "Hello" ⇒ sender ! "World"
|
||||
|
|
@ -18,7 +24,7 @@ object PinnedActorSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout {
|
||||
class PinnedActorSpec extends AkkaSpec(PinnedActorSpec.config) with BeforeAndAfterEach with DefaultTimeout {
|
||||
import PinnedActorSpec._
|
||||
|
||||
private val unit = TimeUnit.MILLISECONDS
|
||||
|
|
@ -27,14 +33,14 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeo
|
|||
|
||||
"support tell" in {
|
||||
var oneWay = new CountDownLatch(1)
|
||||
val actor = system.actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val actor = system.actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher("pinned-dispatcher"))
|
||||
val result = actor ! "OneWay"
|
||||
assert(oneWay.await(1, TimeUnit.SECONDS))
|
||||
system.stop(actor)
|
||||
}
|
||||
|
||||
"support ask/reply" in {
|
||||
val actor = system.actorOf(Props[TestActor].withDispatcher(system.dispatcherFactory.newPinnedDispatcher("test")))
|
||||
val actor = system.actorOf(Props[TestActor].withDispatcher("pinned-dispatcher"))
|
||||
assert("World" === Await.result(actor ? "Hello", timeout.duration))
|
||||
system.stop(actor)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,6 +24,7 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
|
|||
settings.ConfigVersion must equal("2.0-SNAPSHOT")
|
||||
|
||||
getString("akka.actor.default-dispatcher.type") must equal("Dispatcher")
|
||||
getString("akka.actor.default-dispatcher.name") must equal("default-dispatcher")
|
||||
getMilliseconds("akka.actor.default-dispatcher.keep-alive-time") must equal(60 * 1000)
|
||||
getDouble("akka.actor.default-dispatcher.core-pool-size-factor") must equal(8.0)
|
||||
getDouble("akka.actor.default-dispatcher.max-pool-size-factor") must equal(8.0)
|
||||
|
|
@ -34,11 +35,8 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) {
|
|||
getMilliseconds("akka.actor.default-dispatcher.mailbox-push-timeout-time") must equal(10 * 1000)
|
||||
getString("akka.actor.default-dispatcher.mailboxType") must be("")
|
||||
getMilliseconds("akka.actor.default-dispatcher.shutdown-timeout") must equal(1 * 1000)
|
||||
settings.DispatcherDefaultShutdown must equal(1 second)
|
||||
getInt("akka.actor.default-dispatcher.throughput") must equal(5)
|
||||
settings.DispatcherThroughput must equal(5)
|
||||
getMilliseconds("akka.actor.default-dispatcher.throughput-deadline-time") must equal(0)
|
||||
settings.DispatcherThroughputDeadlineTime must equal(Duration.Zero)
|
||||
getBoolean("akka.actor.serialize-messages") must equal(false)
|
||||
settings.SerializeAllMessages must equal(false)
|
||||
|
||||
|
|
|
|||
|
|
@ -532,7 +532,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
|
|||
|
||||
assert(Await.result(y, timeout.duration) === 5)
|
||||
assert(Await.result(z, timeout.duration) === 5)
|
||||
assert(lz.isOpen)
|
||||
Await.ready(lz, timeout.duration)
|
||||
assert(Await.result(result, timeout.duration) === 10)
|
||||
|
||||
val a, b, c = Promise[Int]()
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.util.duration._
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorContext
|
||||
import com.typesafe.config.Config
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
||||
|
|
@ -152,10 +153,14 @@ class PriorityMailboxSpec extends MailboxSpec {
|
|||
object CustomMailboxSpec {
|
||||
val config = """
|
||||
my-dispatcher {
|
||||
mailboxType = "akka.dispatch.CustomMailboxSpec$MyMailbox"
|
||||
mailboxType = "akka.dispatch.CustomMailboxSpec$MyMailboxType"
|
||||
}
|
||||
"""
|
||||
|
||||
class MyMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new MyMailbox(owner)
|
||||
}
|
||||
|
||||
class MyMailbox(owner: ActorContext) extends CustomMailbox(owner)
|
||||
with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
|
||||
final val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||
|
|
@ -166,7 +171,7 @@ object CustomMailboxSpec {
|
|||
class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) {
|
||||
"Dispatcher configuration" must {
|
||||
"support custom mailboxType" in {
|
||||
val dispatcher = system.dispatcherFactory.newFromConfig("my-dispatcher")
|
||||
val dispatcher = system.dispatchers.lookup("my-dispatcher")
|
||||
dispatcher.createMailbox(null).getClass must be(classOf[CustomMailboxSpec.MyMailbox])
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,29 +3,48 @@ package akka.dispatch
|
|||
import akka.actor.{ Props, LocalActorRef, Actor }
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import com.typesafe.config.Config
|
||||
|
||||
object PriorityDispatcherSpec {
|
||||
val config = """
|
||||
unbounded-prio-dispatcher {
|
||||
mailboxType = "akka.dispatch.PriorityDispatcherSpec$Unbounded"
|
||||
}
|
||||
bounded-prio-dispatcher {
|
||||
mailboxType = "akka.dispatch.PriorityDispatcherSpec$Bounded"
|
||||
}
|
||||
"""
|
||||
|
||||
class Unbounded(config: Config) extends UnboundedPriorityMailbox(PriorityGenerator({
|
||||
case i: Int ⇒ i //Reverse order
|
||||
case 'Result ⇒ Int.MaxValue
|
||||
}: Any ⇒ Int))
|
||||
|
||||
class Bounded(config: Config) extends BoundedPriorityMailbox(PriorityGenerator({
|
||||
case i: Int ⇒ i //Reverse order
|
||||
case 'Result ⇒ Int.MaxValue
|
||||
}: Any ⇒ Int), 1000, 10 seconds)
|
||||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class PriorityDispatcherSpec extends AkkaSpec with DefaultTimeout {
|
||||
class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) with DefaultTimeout {
|
||||
|
||||
"A PriorityDispatcher" must {
|
||||
"Order it's messages according to the specified comparator using an unbounded mailbox" in {
|
||||
testOrdering(UnboundedPriorityMailbox(PriorityGenerator({
|
||||
case i: Int ⇒ i //Reverse order
|
||||
case 'Result ⇒ Int.MaxValue
|
||||
}: Any ⇒ Int)))
|
||||
val dispatcherKey = "unbounded-prio-dispatcher"
|
||||
testOrdering(dispatcherKey)
|
||||
}
|
||||
|
||||
"Order it's messages according to the specified comparator using a bounded mailbox" in {
|
||||
testOrdering(BoundedPriorityMailbox(PriorityGenerator({
|
||||
case i: Int ⇒ i //Reverse order
|
||||
case 'Result ⇒ Int.MaxValue
|
||||
}: Any ⇒ Int), 1000, system.settings.MailboxPushTimeout))
|
||||
val dispatcherKey = "bounded-prio-dispatcher"
|
||||
testOrdering(dispatcherKey)
|
||||
}
|
||||
}
|
||||
|
||||
def testOrdering(mboxType: MailboxType) {
|
||||
val dispatcher = system.dispatcherFactory.newDispatcher("Test", 1, Duration.Zero, mboxType).build
|
||||
def testOrdering(dispatcherKey: String) {
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
var acc: List[Int] = Nil
|
||||
|
|
@ -34,7 +53,7 @@ class PriorityDispatcherSpec extends AkkaSpec with DefaultTimeout {
|
|||
case i: Int ⇒ acc = i :: acc
|
||||
case 'Result ⇒ sender.tell(acc)
|
||||
}
|
||||
}).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef]
|
||||
}).withDispatcher(dispatcherKey)).asInstanceOf[LocalActorRef]
|
||||
|
||||
actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages
|
||||
|
||||
|
|
|
|||
|
|
@ -16,11 +16,6 @@ import org.apache.commons.math.stat.descriptive.SynchronizedDescriptiveStatistic
|
|||
class TellLatencyPerformanceSpec extends PerformanceSpec {
|
||||
import TellLatencyPerformanceSpec._
|
||||
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(8)
|
||||
.build
|
||||
|
||||
val repeat = 200L * repeatFactor
|
||||
|
||||
var stat: DescriptiveStatistics = _
|
||||
|
|
@ -55,15 +50,16 @@ class TellLatencyPerformanceSpec extends PerformanceSpec {
|
|||
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
|
||||
if (acceptClients(numberOfClients)) {
|
||||
|
||||
val dispatcherKey = "benchmark.latency-dispatcher"
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val clients = (for (i ← 0 until numberOfClients) yield {
|
||||
val destination = system.actorOf(Props[Destination])
|
||||
val w4 = system.actorOf(Props(new Waypoint(destination)))
|
||||
val w3 = system.actorOf(Props(new Waypoint(w4)))
|
||||
val w2 = system.actorOf(Props(new Waypoint(w3)))
|
||||
val w1 = system.actorOf(Props(new Waypoint(w2)))
|
||||
Props(new Client(w1, latch, repeatsPerClient, clientDelay.toMicros.intValue, stat)).withDispatcher(clientDispatcher)
|
||||
val destination = system.actorOf(Props[Destination].withDispatcher(dispatcherKey))
|
||||
val w4 = system.actorOf(Props(new Waypoint(destination)).withDispatcher(dispatcherKey))
|
||||
val w3 = system.actorOf(Props(new Waypoint(w4)).withDispatcher(dispatcherKey))
|
||||
val w2 = system.actorOf(Props(new Waypoint(w3)).withDispatcher(dispatcherKey))
|
||||
val w1 = system.actorOf(Props(new Waypoint(w2)).withDispatcher(dispatcherKey))
|
||||
Props(new Client(w1, latch, repeatsPerClient, clientDelay.toMicros.intValue, stat)).withDispatcher(dispatcherKey)
|
||||
}).toList.map(system.actorOf(_))
|
||||
|
||||
val start = System.nanoTime
|
||||
|
|
|
|||
|
|
@ -16,32 +16,6 @@ import akka.util.duration._
|
|||
class TellThroughput10000PerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughput10000PerformanceSpec._
|
||||
|
||||
/* Experiment with java 7 LinkedTransferQueue
|
||||
def linkedTransferQueue(): () ⇒ BlockingQueue[Runnable] =
|
||||
() ⇒ new java.util.concurrent.LinkedTransferQueue[Runnable]()
|
||||
|
||||
def createDispatcher(name: String) = {
|
||||
val threadPoolConfig = ThreadPoolConfig()
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
|
||||
0, UnboundedMailbox(), config, 60000), threadPoolConfig)
|
||||
//.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.copy(config = threadPoolConfig.copy(queueFactory = linkedTransferQueue()))
|
||||
.setCorePoolSize(maxClients * 2)
|
||||
.build
|
||||
}
|
||||
*/
|
||||
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 10000,
|
||||
Duration.Zero, UnboundedMailbox(), config, Duration(1, TimeUnit.SECONDS)), ThreadPoolConfig())
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients * 2)
|
||||
.build
|
||||
|
||||
val clientDispatcher = createDispatcher("client-dispatcher")
|
||||
//val destinationDispatcher = createDispatcher("destination-dispatcher")
|
||||
|
||||
val repeat = 30000L * repeatFactor
|
||||
|
||||
"Tell" must {
|
||||
|
|
@ -130,45 +104,19 @@ class TellThroughput10000PerformanceSpec extends PerformanceSpec {
|
|||
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
|
||||
if (acceptClients(numberOfClients)) {
|
||||
|
||||
val dispatcherKey = "benchmark.high-throughput-dispatcher"
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
/*
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(createDispatcher("destination-" + i)))
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(dispatcherKey))
|
||||
val clients = for ((dest, j) ← destinations.zipWithIndex)
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(createDispatcher("client-" + j)))
|
||||
*/
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(clientDispatcher))
|
||||
val clients = for ((dest, j) ← destinations.zipWithIndex)
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(dispatcherKey))
|
||||
|
||||
val start = System.nanoTime
|
||||
clients.foreach(_ ! Run)
|
||||
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
|
||||
val durationNs = (System.nanoTime - start)
|
||||
|
||||
if (!ok) {
|
||||
System.err.println("Destinations: ")
|
||||
destinations.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
System.err.println("")
|
||||
System.err.println("Clients: ")
|
||||
|
||||
clients.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
|
||||
//val e = clientDispatcher.asInstanceOf[Dispatcher].executorService.get().asInstanceOf[ExecutorServiceDelegate].executor.asInstanceOf[ThreadPoolExecutor]
|
||||
//val q = e.getQueue
|
||||
//System.err.println("Client Dispatcher: " + e.getActiveCount + " " + Stream.continually(q.poll()).takeWhile(_ != null).mkString(", "))
|
||||
}
|
||||
|
||||
if (!warmup) {
|
||||
ok must be(true)
|
||||
logMeasurement(numberOfClients, durationNs, repeat)
|
||||
|
|
|
|||
|
|
@ -12,16 +12,6 @@ import akka.util.duration._
|
|||
class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputComputationPerformanceSpec._
|
||||
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
|
||||
Duration.Zero, UnboundedMailbox(), config, 1 seconds), ThreadPoolConfig())
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
||||
val clientDispatcher = createDispatcher("client-dispatcher")
|
||||
val destinationDispatcher = createDispatcher("destination-dispatcher")
|
||||
|
||||
val repeat = 500L * repeatFactor
|
||||
|
||||
"Tell" must {
|
||||
|
|
@ -110,6 +100,9 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
|
|||
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
|
||||
if (acceptClients(numberOfClients)) {
|
||||
|
||||
val clientDispatcher = "benchmark.client-dispatcher"
|
||||
val destinationDispatcher = "benchmark.destination-dispatcher"
|
||||
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
|
|
@ -122,27 +115,6 @@ class TellThroughputComputationPerformanceSpec extends PerformanceSpec {
|
|||
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
|
||||
val durationNs = (System.nanoTime - start)
|
||||
|
||||
if (!ok) {
|
||||
System.err.println("Destinations: ")
|
||||
destinations.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
System.err.println("")
|
||||
System.err.println("Clients: ")
|
||||
|
||||
clients.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
|
||||
val e = clientDispatcher.asInstanceOf[Dispatcher].executorService.get().asInstanceOf[ExecutorServiceDelegate].executor.asInstanceOf[ThreadPoolExecutor]
|
||||
val q = e.getQueue
|
||||
System.err.println("Client Dispatcher: " + e.getActiveCount + " " + Stream.continually(q.poll()).takeWhile(_ != null).mkString(", "))
|
||||
}
|
||||
|
||||
if (!warmup) {
|
||||
ok must be(true)
|
||||
logMeasurement(numberOfClients, durationNs, repeat)
|
||||
|
|
|
|||
|
|
@ -12,16 +12,6 @@ import akka.util.duration._
|
|||
class TellThroughputPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputPerformanceSpec._
|
||||
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
|
||||
Duration.Zero, UnboundedMailbox(), config, 1 seconds), ThreadPoolConfig())
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
||||
val clientDispatcher = createDispatcher("client-dispatcher")
|
||||
val destinationDispatcher = createDispatcher("destination-dispatcher")
|
||||
|
||||
val repeat = 30000L * repeatFactor
|
||||
|
||||
"Tell" must {
|
||||
|
|
@ -62,6 +52,9 @@ class TellThroughputPerformanceSpec extends PerformanceSpec {
|
|||
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
|
||||
if (acceptClients(numberOfClients)) {
|
||||
|
||||
val clientDispatcher = "benchmark.client-dispatcher"
|
||||
val destinationDispatcher = "benchmark.destination-dispatcher"
|
||||
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
|
|
|
|||
|
|
@ -13,18 +13,8 @@ import akka.util.duration._
|
|||
|
||||
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputSeparateDispatchersPerformanceSpec._
|
||||
|
||||
def createDispatcher(name: String) = ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(system.dispatcherFactory.prerequisites, name, 5,
|
||||
Duration.Zero, UnboundedMailbox(), config, Duration(1, TimeUnit.SECONDS)), ThreadPoolConfig())
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(1)
|
||||
.build
|
||||
|
||||
//val clientDispatcher = createDispatcher("client-dispatcher")
|
||||
//val destinationDispatcher = createDispatcher("destination-dispatcher")
|
||||
class TellThroughputPinnedDispatchersPerformanceSpec extends PerformanceSpec {
|
||||
import TellThroughputPinnedDispatchersPerformanceSpec._
|
||||
|
||||
val repeat = 30000L * repeatFactor
|
||||
|
||||
|
|
@ -114,47 +104,21 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
|
|||
def runScenario(numberOfClients: Int, warmup: Boolean = false) {
|
||||
if (acceptClients(numberOfClients)) {
|
||||
|
||||
val pinnedDispatcher = "benchmark.pinned-dispatcher"
|
||||
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
val repeatsPerClient = repeat / numberOfClients
|
||||
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(createDispatcher("destination-" + i)))
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(pinnedDispatcher))
|
||||
val clients = for ((dest, j) ← destinations.zipWithIndex)
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(createDispatcher("client-" + j)))
|
||||
|
||||
/*
|
||||
val destinations = for (i ← 0 until numberOfClients)
|
||||
yield system.actorOf(Props(new Destination).withDispatcher(clientDispatcher))
|
||||
val clients = for ((dest, j) ← destinations.zipWithIndex)
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(clientDispatcher))
|
||||
*/
|
||||
yield system.actorOf(Props(new Client(dest, latch, repeatsPerClient)).withDispatcher(pinnedDispatcher))
|
||||
|
||||
val start = System.nanoTime
|
||||
clients.foreach(_ ! Run)
|
||||
val ok = latch.await(maxRunDuration.toMillis, TimeUnit.MILLISECONDS)
|
||||
val durationNs = (System.nanoTime - start)
|
||||
|
||||
if (!ok) {
|
||||
System.err.println("Destinations: ")
|
||||
destinations.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
System.err.println("")
|
||||
System.err.println("Clients: ")
|
||||
|
||||
clients.foreach {
|
||||
case l: LocalActorRef ⇒
|
||||
val m = l.underlying.mailbox
|
||||
System.err.println(" -" + l + " mbox(" + m.status + ")" + " containing [" + Stream.continually(m.dequeue()).takeWhile(_ != null).mkString(", ") + "] and has systemMsgs: " + m.hasSystemMessages)
|
||||
}
|
||||
|
||||
//val e = clientDispatcher.asInstanceOf[Dispatcher].executorService.get().asInstanceOf[ExecutorServiceDelegate].executor.asInstanceOf[ThreadPoolExecutor]
|
||||
//val q = e.getQueue
|
||||
//System.err.println("Client Dispatcher: " + e.getActiveCount + " " + Stream.continually(q.poll()).takeWhile(_ != null).mkString(", "))
|
||||
}
|
||||
|
||||
if (!warmup) {
|
||||
ok must be(true)
|
||||
logMeasurement(numberOfClients, durationNs, repeat)
|
||||
|
|
@ -167,7 +131,7 @@ class TellThroughputSeparateDispatchersPerformanceSpec extends PerformanceSpec {
|
|||
}
|
||||
}
|
||||
|
||||
object TellThroughputSeparateDispatchersPerformanceSpec {
|
||||
object TellThroughputPinnedDispatchersPerformanceSpec {
|
||||
|
||||
case object Run
|
||||
case object Msg
|
||||
|
|
@ -20,11 +20,6 @@ import akka.performance.trading.domain.Orderbook
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TradingLatencyPerformanceSpec extends PerformanceSpec {
|
||||
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
||||
var tradingSystem: AkkaTradingSystem = _
|
||||
|
||||
var stat: DescriptiveStatistics = _
|
||||
|
|
@ -86,6 +81,8 @@ class TradingLatencyPerformanceSpec extends PerformanceSpec {
|
|||
} yield Bid(s + i, 100 - i, 1000)
|
||||
val orders = askOrders.zip(bidOrders).map(x ⇒ Seq(x._1, x._2)).flatten
|
||||
|
||||
val clientDispatcher = "benchmark.client-dispatcher"
|
||||
|
||||
val ordersPerClient = repeat * orders.size / numberOfClients
|
||||
val totalNumberOfOrders = ordersPerClient * numberOfClients
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
|
|
|
|||
|
|
@ -38,14 +38,14 @@ class AkkaTradingSystem(val system: ActorSystem) extends TradingSystem {
|
|||
type ME = ActorRef
|
||||
type OR = ActorRef
|
||||
|
||||
val orDispatcher = createOrderReceiverDispatcher
|
||||
val meDispatcher = createMatchingEngineDispatcher
|
||||
val orDispatcher = orderReceiverDispatcher
|
||||
val meDispatcher = matchingEngineDispatcher
|
||||
|
||||
// by default we use default-dispatcher that is defined in akka.conf
|
||||
def createOrderReceiverDispatcher: Option[MessageDispatcher] = None
|
||||
// by default we use default-dispatcher
|
||||
def orderReceiverDispatcher: Option[String] = None
|
||||
|
||||
// by default we use default-dispatcher that is defined in akka.conf
|
||||
def createMatchingEngineDispatcher: Option[MessageDispatcher] = None
|
||||
// by default we use default-dispatcher
|
||||
def matchingEngineDispatcher: Option[String] = None
|
||||
|
||||
var matchingEngineForOrderbook: Map[String, ActorRef] = Map()
|
||||
|
||||
|
|
|
|||
|
|
@ -20,11 +20,6 @@ import akka.performance.trading.domain.Orderbook
|
|||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class TradingThroughputPerformanceSpec extends PerformanceSpec {
|
||||
|
||||
val clientDispatcher = system.dispatcherFactory.newDispatcher("client-dispatcher")
|
||||
.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
.setCorePoolSize(maxClients)
|
||||
.build
|
||||
|
||||
var tradingSystem: AkkaTradingSystem = _
|
||||
|
||||
override def beforeEach() {
|
||||
|
|
@ -83,6 +78,8 @@ class TradingThroughputPerformanceSpec extends PerformanceSpec {
|
|||
} yield Bid(s + i, 100 - i, 1000)
|
||||
val orders = askOrders.zip(bidOrders).map(x ⇒ Seq(x._1, x._2)).flatten
|
||||
|
||||
val clientDispatcher = "benchmark.client-dispatcher"
|
||||
|
||||
val ordersPerClient = repeat * orders.size / numberOfClients
|
||||
val totalNumberOfOrders = ordersPerClient * numberOfClients
|
||||
val latch = new CountDownLatch(numberOfClients)
|
||||
|
|
|
|||
|
|
@ -3,18 +3,49 @@ import com.typesafe.config.ConfigFactory
|
|||
|
||||
object BenchmarkConfig {
|
||||
private val benchmarkConfig = ConfigFactory.parseString("""
|
||||
akka {
|
||||
event-handlers = ["akka.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
}
|
||||
|
||||
benchmark {
|
||||
longRunning = false
|
||||
minClients = 1
|
||||
maxClients = 4
|
||||
repeatFactor = 2
|
||||
timeDilation = 1
|
||||
maxRunDuration = 10 seconds
|
||||
maxRunDuration = 20 seconds
|
||||
clientDelay = 250000 nanoseconds
|
||||
logResult = true
|
||||
resultDir = "target/benchmark"
|
||||
useDummyOrderbook = false
|
||||
}
|
||||
|
||||
client-dispatcher {
|
||||
core-pool-size-min = ${benchmark.maxClients}
|
||||
core-pool-size-max = ${benchmark.maxClients}
|
||||
}
|
||||
|
||||
destination-dispatcher {
|
||||
core-pool-size-min = ${benchmark.maxClients}
|
||||
core-pool-size-max = ${benchmark.maxClients}
|
||||
}
|
||||
|
||||
high-throughput-dispatcher {
|
||||
throughput = 10000
|
||||
core-pool-size-min = ${benchmark.maxClients}
|
||||
core-pool-size-max = ${benchmark.maxClients}
|
||||
}
|
||||
|
||||
pinned-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
latency-dispatcher {
|
||||
throughput = 1
|
||||
core-pool-size-min = ${benchmark.maxClients}
|
||||
core-pool-size-max = ${benchmark.maxClients}
|
||||
}
|
||||
}
|
||||
""")
|
||||
private val longRunningBenchmarkConfig = ConfigFactory.parseString("""
|
||||
benchmark {
|
||||
|
|
@ -23,10 +54,14 @@ object BenchmarkConfig {
|
|||
repeatFactor = 150
|
||||
maxRunDuration = 120 seconds
|
||||
useDummyOrderbook = true
|
||||
}
|
||||
}
|
||||
""").withFallback(benchmarkConfig)
|
||||
|
||||
def config = if (System.getProperty("benchmark.longRunning") == "true")
|
||||
longRunningBenchmarkConfig else benchmarkConfig
|
||||
def config = {
|
||||
val benchCfg =
|
||||
if (System.getProperty("benchmark.longRunning") == "true") longRunningBenchmarkConfig else benchmarkConfig
|
||||
// external config first, to be able to override
|
||||
ConfigFactory.load(benchCfg)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -6,11 +6,10 @@ package akka.routing
|
|||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor._
|
||||
import collection.mutable.LinkedList
|
||||
import java.util.concurrent.{ CountDownLatch, TimeUnit }
|
||||
import akka.testkit._
|
||||
import akka.util.duration._
|
||||
import akka.dispatch.Await
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.Duration
|
||||
|
||||
object RoutingSpec {
|
||||
|
||||
|
|
@ -30,18 +29,7 @@ object RoutingSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
||||
akka {
|
||||
actor {
|
||||
deployment {
|
||||
/a1 {
|
||||
router = round-robin
|
||||
nr-of-instances = 3
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
""")) with DefaultTimeout with ImplicitSender {
|
||||
class RoutingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||
|
||||
val impl = system.asInstanceOf[ActorSystemImpl]
|
||||
|
||||
|
|
@ -72,13 +60,16 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
}
|
||||
|
||||
"be able to send their routees" in {
|
||||
val doneLatch = new CountDownLatch(1)
|
||||
val doneLatch = new TestLatch(1)
|
||||
|
||||
class TheActor extends Actor {
|
||||
val routee1 = context.actorOf(Props[TestActor], "routee1")
|
||||
val routee2 = context.actorOf(Props[TestActor], "routee2")
|
||||
val routee3 = context.actorOf(Props[TestActor], "routee3")
|
||||
val router = context.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(routees = List(routee1, routee2, routee3))))
|
||||
val router = context.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(
|
||||
routees = List(routee1, routee2, routee3),
|
||||
within = 5 seconds)))
|
||||
|
||||
def receive = {
|
||||
case RouterRoutees(iterable) ⇒
|
||||
|
|
@ -93,7 +84,7 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
val theActor = system.actorOf(Props(new TheActor), "theActor")
|
||||
theActor ! "doIt"
|
||||
doneLatch.await(1, TimeUnit.SECONDS) must be(true)
|
||||
Await.ready(doneLatch, 1 seconds)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -314,7 +305,8 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
"Scatter-gather router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(routees = List(newActor(0)))))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(newActor(0)), within = 1 seconds)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
|
|
@ -337,7 +329,8 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2))))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 1 seconds)))
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
|
|
@ -350,12 +343,13 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
"return response, even if one of the actors has stopped" in {
|
||||
val shutdownLatch = new TestLatch(1)
|
||||
val actor1 = newActor(1, Some(shutdownLatch))
|
||||
val actor2 = newActor(22, Some(shutdownLatch))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2))))
|
||||
val actor2 = newActor(14, Some(shutdownLatch))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 3 seconds)))
|
||||
|
||||
routedActor ! Broadcast(Stop(Some(1)))
|
||||
Await.ready(shutdownLatch, TestLatch.DefaultTimeout)
|
||||
Await.result(routedActor ? Broadcast(0), timeout.duration) must be(22)
|
||||
Await.result(routedActor ? Broadcast(0), timeout.duration) must be(14)
|
||||
}
|
||||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
|
@ -379,12 +373,12 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
|
||||
"custom router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(VoteCountRouter()))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(VoteCountRouter))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"count votes as intended - not as in Florida" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(VoteCountRouter()))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(VoteCountRouter))
|
||||
routedActor ! DemocratVote
|
||||
routedActor ! DemocratVote
|
||||
routedActor ! RepublicanVote
|
||||
|
|
@ -428,8 +422,7 @@ class RoutingSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
//#crActors
|
||||
|
||||
//#crRouter
|
||||
case class VoteCountRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil)
|
||||
extends RouterConfig {
|
||||
object VoteCountRouter extends RouterConfig {
|
||||
|
||||
//#crRoute
|
||||
def createRoute(props: Props,
|
||||
|
|
|
|||
|
|
@ -5,13 +5,39 @@ package akka.testkit
|
|||
|
||||
import akka.actor.dispatch.ActorModelSpec
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.junit.{ After, Test }
|
||||
import com.typesafe.config.Config
|
||||
import akka.dispatch.DispatcherPrerequisites
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.MessageDispatcherConfigurator
|
||||
|
||||
object CallingThreadDispatcherModelSpec {
|
||||
val config = """
|
||||
boss {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class CallingThreadDispatcherModelSpec extends ActorModelSpec {
|
||||
class CallingThreadDispatcherModelSpec extends ActorModelSpec(CallingThreadDispatcherModelSpec.config) {
|
||||
import ActorModelSpec._
|
||||
|
||||
def newInterceptedDispatcher = new CallingThreadDispatcher(system.dispatcherFactory.prerequisites, "test") with MessageDispatcherInterceptor
|
||||
def dispatcherType = "Calling Thread Dispatcher"
|
||||
val dispatcherCount = new AtomicInteger()
|
||||
|
||||
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
|
||||
// use new id for each invocation, since the MessageDispatcherInterceptor holds state
|
||||
val dispatcherId = "test-calling-thread" + dispatcherCount.incrementAndGet()
|
||||
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers.defaultDispatcherConfig, system.dispatchers.prerequisites) {
|
||||
val instance = new CallingThreadDispatcher(prerequisites) with MessageDispatcherInterceptor {
|
||||
override def id: String = dispatcherId
|
||||
}
|
||||
override def dispatcher(): MessageDispatcher = instance
|
||||
}
|
||||
system.dispatchers.register(dispatcherId, dispatcherConfigurator)
|
||||
system.dispatchers.lookup(dispatcherId).asInstanceOf[MessageDispatcherInterceptor]
|
||||
}
|
||||
override def dispatcherType = "Calling Thread Dispatcher"
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -78,6 +78,9 @@ akka {
|
|||
# is ignored if routees.paths is given
|
||||
nr-of-instances = 1
|
||||
|
||||
# within is the timeout used for routers containing future calls
|
||||
within = 5 seconds
|
||||
|
||||
# FIXME document 'create-as', ticket 1511
|
||||
create-as {
|
||||
# fully qualified class name of recipe implementation
|
||||
|
|
@ -96,13 +99,14 @@ akka {
|
|||
default-dispatcher {
|
||||
# Must be one of the following
|
||||
# Dispatcher, (BalancingDispatcher, only valid when all actors using it are of
|
||||
# the same type),
|
||||
# A FQCN to a class inheriting MessageDispatcherConfigurator with a no-arg
|
||||
# visible constructor
|
||||
# the same type), PinnedDispatcher, or a FQCN to a class inheriting
|
||||
# MessageDispatcherConfigurator with a constructor with
|
||||
# com.typesafe.config.Config parameter and akka.dispatch.DispatcherPrerequisites
|
||||
# parameters
|
||||
type = "Dispatcher"
|
||||
|
||||
# Name used in log messages and thread names.
|
||||
name = "DefaultDispatcher"
|
||||
name = "default-dispatcher"
|
||||
|
||||
# Toggles whether the threads created by this dispatcher should be daemons or not
|
||||
daemonic = off
|
||||
|
|
@ -162,7 +166,8 @@ akka {
|
|||
mailbox-push-timeout-time = 10s
|
||||
|
||||
# FQCN of the MailboxType, if not specified the default bounded or unbounded
|
||||
# mailbox is used.
|
||||
# mailbox is used. The Class of the FQCN must have a constructor with a
|
||||
# com.typesafe.config.Config parameter.
|
||||
mailboxType = ""
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -258,7 +258,7 @@ private[akka] class ActorCell(
|
|||
}
|
||||
|
||||
@inline
|
||||
final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
|
||||
final def dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
|
||||
|
||||
/**
|
||||
* UntypedActorContext impl
|
||||
|
|
|
|||
|
|
@ -88,12 +88,6 @@ object ActorSystem {
|
|||
val FsmDebugEvent = getBoolean("akka.actor.debug.fsm")
|
||||
val DebugEventStream = getBoolean("akka.actor.debug.event-stream")
|
||||
|
||||
val DispatcherThroughput = getInt("akka.actor.default-dispatcher.throughput")
|
||||
val DispatcherDefaultShutdown = Duration(getMilliseconds("akka.actor.default-dispatcher.shutdown-timeout"), MILLISECONDS)
|
||||
val MailboxCapacity = getInt("akka.actor.default-dispatcher.mailbox-capacity")
|
||||
val MailboxPushTimeout = Duration(getNanoseconds("akka.actor.default-dispatcher.mailbox-push-timeout-time"), NANOSECONDS)
|
||||
val DispatcherThroughputDeadlineTime = Duration(getNanoseconds("akka.actor.default-dispatcher.throughput-deadline-time"), NANOSECONDS)
|
||||
|
||||
val Home = config.getString("akka.home") match {
|
||||
case "" ⇒ None
|
||||
case x ⇒ Some(x)
|
||||
|
|
@ -269,10 +263,9 @@ abstract class ActorSystem extends ActorRefFactory {
|
|||
//#scheduler
|
||||
|
||||
/**
|
||||
* Helper object for creating new dispatchers and passing in all required
|
||||
* information.
|
||||
* Helper object for looking up configured dispatchers.
|
||||
*/
|
||||
def dispatcherFactory: Dispatchers
|
||||
def dispatchers: Dispatchers
|
||||
|
||||
/**
|
||||
* Default dispatcher as configured. This dispatcher is used for all actors
|
||||
|
|
@ -413,8 +406,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor
|
|||
}
|
||||
}
|
||||
|
||||
val dispatcherFactory = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
|
||||
val dispatcher = dispatcherFactory.defaultGlobalDispatcher
|
||||
val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
|
||||
val dispatcher = dispatchers.defaultGlobalDispatcher
|
||||
|
||||
def terminationFuture: Future[Unit] = provider.terminationFuture
|
||||
def lookupRoot: InternalActorRef = provider.rootGuardian
|
||||
|
|
|
|||
|
|
@ -5,7 +5,6 @@
|
|||
package akka.actor
|
||||
|
||||
import collection.immutable.Seq
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import akka.event.Logging
|
||||
import akka.AkkaException
|
||||
import akka.config.ConfigurationException
|
||||
|
|
@ -13,6 +12,7 @@ import akka.util.Duration
|
|||
import akka.event.EventStream
|
||||
import com.typesafe.config._
|
||||
import akka.routing._
|
||||
import java.util.concurrent.{ TimeUnit, ConcurrentHashMap }
|
||||
|
||||
case class Deploy(path: String, config: Config, recipe: Option[ActorRecipe] = None, routing: RouterConfig = NoRouter, scope: Scope = LocalScope)
|
||||
|
||||
|
|
@ -53,11 +53,13 @@ class Deployer(val settings: ActorSystem.Settings) {
|
|||
|
||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||
|
||||
val within = Duration(deployment.getMilliseconds("within"), TimeUnit.MILLISECONDS)
|
||||
|
||||
val router: RouterConfig = deployment.getString("router") match {
|
||||
case "from-code" ⇒ NoRouter
|
||||
case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees)
|
||||
case "random" ⇒ RandomRouter(nrOfInstances, routees)
|
||||
case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees)
|
||||
case "scatter-gather" ⇒ ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within)
|
||||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees)
|
||||
case x ⇒ throw new ConfigurationException("unknown router type " + x + " for path " + key)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ object Props {
|
|||
import FaultHandlingStrategy._
|
||||
|
||||
final val defaultCreator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!")
|
||||
final val defaultDispatcher: MessageDispatcher = null
|
||||
final val defaultTimeout: Timeout = Timeout(Duration.MinusInf)
|
||||
final val defaultDecider: Decider = {
|
||||
case _: ActorInitializationException ⇒ Stop
|
||||
|
|
@ -125,7 +124,7 @@ object Props {
|
|||
*/
|
||||
case class Props(
|
||||
creator: () ⇒ Actor = Props.defaultCreator,
|
||||
@transient dispatcher: MessageDispatcher = Props.defaultDispatcher,
|
||||
dispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
timeout: Timeout = Props.defaultTimeout,
|
||||
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler,
|
||||
routerConfig: RouterConfig = Props.defaultRoutedProps) {
|
||||
|
|
@ -135,7 +134,7 @@ case class Props(
|
|||
*/
|
||||
def this() = this(
|
||||
creator = Props.defaultCreator,
|
||||
dispatcher = Props.defaultDispatcher,
|
||||
dispatcher = Dispatchers.DefaultDispatcherId,
|
||||
timeout = Props.defaultTimeout,
|
||||
faultHandler = Props.defaultFaultHandler)
|
||||
|
||||
|
|
@ -144,7 +143,7 @@ case class Props(
|
|||
*/
|
||||
def this(factory: UntypedActorFactory) = this(
|
||||
creator = () ⇒ factory.create(),
|
||||
dispatcher = Props.defaultDispatcher,
|
||||
dispatcher = Dispatchers.DefaultDispatcherId,
|
||||
timeout = Props.defaultTimeout,
|
||||
faultHandler = Props.defaultFaultHandler)
|
||||
|
||||
|
|
@ -153,7 +152,7 @@ case class Props(
|
|||
*/
|
||||
def this(actorClass: Class[_ <: Actor]) = this(
|
||||
creator = () ⇒ actorClass.newInstance,
|
||||
dispatcher = Props.defaultDispatcher,
|
||||
dispatcher = Dispatchers.DefaultDispatcherId,
|
||||
timeout = Props.defaultTimeout,
|
||||
faultHandler = Props.defaultFaultHandler,
|
||||
routerConfig = Props.defaultRoutedProps)
|
||||
|
|
@ -182,7 +181,7 @@ case class Props(
|
|||
/**
|
||||
* Returns a new Props with the specified dispatcher set.
|
||||
*/
|
||||
def withDispatcher(d: MessageDispatcher) = copy(dispatcher = d)
|
||||
def withDispatcher(d: String) = copy(dispatcher = d)
|
||||
|
||||
/**
|
||||
* Returns a new Props with the specified timeout set.
|
||||
|
|
|
|||
|
|
@ -16,6 +16,8 @@ import scala.annotation.tailrec
|
|||
import akka.event.EventStream
|
||||
import akka.actor.ActorSystem.Settings
|
||||
import com.typesafe.config.Config
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.util.ReflectiveAccess
|
||||
|
||||
final case class Envelope(val message: Any, val sender: ActorRef) {
|
||||
if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null")
|
||||
|
|
@ -100,6 +102,12 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
*/
|
||||
def name: String
|
||||
|
||||
/**
|
||||
* Identfier of this dispatcher, corresponds to the full key
|
||||
* of the dispatcher configuration.
|
||||
*/
|
||||
def id: String
|
||||
|
||||
/**
|
||||
* Attaches the specified actor instance to this dispatcher
|
||||
*/
|
||||
|
|
@ -262,15 +270,25 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
}
|
||||
|
||||
/**
|
||||
* Trait to be used for hooking in new dispatchers into Dispatchers.from(cfg: Config)
|
||||
* Base class to be used for hooking in new dispatchers into Dispatchers.
|
||||
*/
|
||||
abstract class MessageDispatcherConfigurator() {
|
||||
/**
|
||||
* Returns an instance of MessageDispatcher given a Configuration
|
||||
*/
|
||||
def configure(config: Config, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher
|
||||
abstract class MessageDispatcherConfigurator(val config: Config, val prerequisites: DispatcherPrerequisites) {
|
||||
|
||||
def mailboxType(config: Config, settings: Settings): MailboxType = {
|
||||
/**
|
||||
* Returns an instance of MessageDispatcher given the configuration.
|
||||
* Depending on the needs the implementation may return a new instance for
|
||||
* each invocation or return the same instance every time.
|
||||
*/
|
||||
def dispatcher(): MessageDispatcher
|
||||
|
||||
/**
|
||||
* Returns a factory for the [[akka.dispatch.Mailbox]] given the configuration.
|
||||
* Default implementation instantiate the [[akka.dispatch.MailboxType]] specified
|
||||
* as FQCN in mailboxType config property. If mailboxType is unspecified (empty)
|
||||
* then [[akka.dispatch.UnboundedMailbox]] is used when capacity is < 1,
|
||||
* otherwise [[akka.dispatch.BoundedMailbox]].
|
||||
*/
|
||||
def mailboxType(): MailboxType = {
|
||||
config.getString("mailboxType") match {
|
||||
case "" ⇒
|
||||
val capacity = config.getInt("mailbox-capacity")
|
||||
|
|
@ -279,13 +297,21 @@ abstract class MessageDispatcherConfigurator() {
|
|||
val duration = Duration(config.getNanoseconds("mailbox-push-timeout-time"), TimeUnit.NANOSECONDS)
|
||||
BoundedMailbox(capacity, duration)
|
||||
}
|
||||
case fqn ⇒ new CustomMailboxType(fqn)
|
||||
case fqcn ⇒
|
||||
val constructorSignature = Array[Class[_]](classOf[Config])
|
||||
ReflectiveAccess.createInstance[MailboxType](fqcn, constructorSignature, Array[AnyRef](config)) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
("Cannot instantiate MailboxType [%s], defined in [%s], " +
|
||||
"make sure it has constructor with a [com.typesafe.config.Config] parameter")
|
||||
.format(fqcn, config.getString("id")), exception)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def configureThreadPool(
|
||||
config: Config,
|
||||
settings: Settings,
|
||||
createDispatcher: ⇒ (ThreadPoolConfig) ⇒ MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
|
||||
import ThreadPoolConfigDispatcherBuilder.conf_?
|
||||
|
||||
|
|
|
|||
|
|
@ -23,8 +23,8 @@ import akka.util.Duration
|
|||
* Although the technique used in this implementation is commonly known as "work stealing", the actual implementation is probably
|
||||
* best described as "work donating" because the actor of which work is being stolen takes the initiative.
|
||||
* <p/>
|
||||
* The preferred way of creating dispatchers is to use
|
||||
* the {@link akka.dispatch.Dispatchers} factory object.
|
||||
* The preferred way of creating dispatchers is to define configuration of it and use the
|
||||
* the `lookup` method in [[akka.dispatch.Dispatchers]].
|
||||
*
|
||||
* @see akka.dispatch.BalancingDispatcher
|
||||
* @see akka.dispatch.Dispatchers
|
||||
|
|
@ -32,12 +32,13 @@ import akka.util.Duration
|
|||
class BalancingDispatcher(
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
_name: String,
|
||||
_id: String,
|
||||
throughput: Int,
|
||||
throughputDeadlineTime: Duration,
|
||||
mailboxType: MailboxType,
|
||||
config: ThreadPoolConfig,
|
||||
_shutdownTimeout: Duration)
|
||||
extends Dispatcher(_prerequisites, _name, throughput, throughputDeadlineTime, mailboxType, config, _shutdownTimeout) {
|
||||
extends Dispatcher(_prerequisites, _name, _id, throughput, throughputDeadlineTime, mailboxType, config, _shutdownTimeout) {
|
||||
|
||||
val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
|
||||
val rebalance = new AtomicBoolean(false)
|
||||
|
|
|
|||
|
|
@ -11,49 +11,11 @@ import akka.util.Duration
|
|||
import java.util.concurrent._
|
||||
|
||||
/**
|
||||
* Default settings are:
|
||||
* <pre/>
|
||||
* - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
|
||||
* - NR_START_THREADS = 16
|
||||
* - NR_MAX_THREADS = 128
|
||||
* - KEEP_ALIVE_TIME = 60000L // one minute
|
||||
* </pre>
|
||||
* <p/>
|
||||
* The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a
|
||||
* `BlockingQueue`.
|
||||
*
|
||||
* The dispatcher has a fluent builder interface to build up a thread pool to suite your use-case.
|
||||
* There is a default thread pool defined but make use of the builder if you need it. Here are some examples.
|
||||
* <p/>
|
||||
*
|
||||
* Scala API.
|
||||
* <p/>
|
||||
* Example usage:
|
||||
* <pre/>
|
||||
* val dispatcher = new Dispatcher("name")
|
||||
* dispatcher
|
||||
* .withNewThreadPoolWithBoundedBlockingQueue(100)
|
||||
* .setCorePoolSize(16)
|
||||
* .setMaxPoolSize(128)
|
||||
* .setKeepAliveTime(60 seconds)
|
||||
* .buildThreadPool
|
||||
* </pre>
|
||||
* <p/>
|
||||
*
|
||||
* Java API.
|
||||
* <p/>
|
||||
* Example usage:
|
||||
* <pre/>
|
||||
* Dispatcher dispatcher = new Dispatcher("name");
|
||||
* dispatcher
|
||||
* .withNewThreadPoolWithBoundedBlockingQueue(100)
|
||||
* .setCorePoolSize(16)
|
||||
* .setMaxPoolSize(128)
|
||||
* .setKeepAliveTime(60 seconds)
|
||||
* .buildThreadPool();
|
||||
* </pre>
|
||||
* <p/>
|
||||
*
|
||||
* But the preferred way of creating dispatchers is to use
|
||||
* the {@link akka.dispatch.Dispatchers} factory object.
|
||||
* The preferred way of creating dispatchers is to define configuration of it and use the
|
||||
* the `lookup` method in [[akka.dispatch.Dispatchers]].
|
||||
*
|
||||
* @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the
|
||||
* mailbox, without checking the mailboxes of other actors. Zero or negative means the dispatcher
|
||||
|
|
@ -63,6 +25,7 @@ import java.util.concurrent._
|
|||
class Dispatcher(
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
val name: String,
|
||||
val id: String,
|
||||
val throughput: Int,
|
||||
val throughputDeadlineTime: Duration,
|
||||
val mailboxType: MailboxType,
|
||||
|
|
|
|||
|
|
@ -6,7 +6,6 @@ package akka.dispatch
|
|||
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
import akka.actor.LocalActorRef
|
||||
import akka.actor.newUuid
|
||||
import akka.util.{ Duration, ReflectiveAccess }
|
||||
|
|
@ -17,6 +16,8 @@ import akka.actor.ActorSystem.Settings
|
|||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.config.ConfigurationException
|
||||
import akka.event.Logging.Warning
|
||||
import akka.actor.Props
|
||||
|
||||
trait DispatcherPrerequisites {
|
||||
def eventStream: EventStream
|
||||
|
|
@ -29,251 +30,195 @@ case class DefaultDispatcherPrerequisites(
|
|||
val deadLetterMailbox: Mailbox,
|
||||
val scheduler: Scheduler) extends DispatcherPrerequisites
|
||||
|
||||
object Dispatchers {
|
||||
/**
|
||||
* The id of the default dispatcher, also the full key of the
|
||||
* configuration of the default dispatcher.
|
||||
*/
|
||||
final val DefaultDispatcherId = "akka.actor.default-dispatcher"
|
||||
}
|
||||
|
||||
/**
|
||||
* It is recommended to define the dispatcher in configuration to allow for tuning
|
||||
* for different environments. Use the `lookup` or `newFromConfig` method to create
|
||||
* Dispatchers are to be defined in configuration to allow for tuning
|
||||
* for different environments. Use the `lookup` method to create
|
||||
* a dispatcher as specified in configuration.
|
||||
*
|
||||
* Scala API. Dispatcher factory.
|
||||
* <p/>
|
||||
* Example usage:
|
||||
* <pre/>
|
||||
* val dispatcher = Dispatchers.newDispatcher("name")
|
||||
* dispatcher
|
||||
* .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100)
|
||||
* .setCorePoolSize(16)
|
||||
* .setMaxPoolSize(128)
|
||||
* .setKeepAliveTime(60 seconds)
|
||||
* .build
|
||||
* </pre>
|
||||
* <p/>
|
||||
* Java API. Dispatcher factory.
|
||||
* <p/>
|
||||
* Example usage:
|
||||
* <pre/>
|
||||
* MessageDispatcher dispatcher = Dispatchers.newDispatcher("name");
|
||||
* dispatcher
|
||||
* .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100)
|
||||
* .setCorePoolSize(16)
|
||||
* .setMaxPoolSize(128)
|
||||
* .setKeepAliveTime(60 seconds)
|
||||
* .build();
|
||||
* </pre>
|
||||
* <p/>
|
||||
* Look in `akka.actor.default-dispatcher` section of the reference.conf
|
||||
* for documentation of dispatcher options.
|
||||
*/
|
||||
class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: DispatcherPrerequisites) {
|
||||
|
||||
val MailboxType: MailboxType =
|
||||
if (settings.MailboxCapacity < 1) UnboundedMailbox()
|
||||
else BoundedMailbox(settings.MailboxCapacity, settings.MailboxPushTimeout)
|
||||
import Dispatchers._
|
||||
|
||||
val defaultDispatcherConfig = settings.config.getConfig("akka.actor.default-dispatcher")
|
||||
val defaultDispatcherConfig: Config =
|
||||
idConfig(DefaultDispatcherId).withFallback(settings.config.getConfig(DefaultDispatcherId))
|
||||
|
||||
lazy val defaultGlobalDispatcher: MessageDispatcher =
|
||||
from(defaultDispatcherConfig) getOrElse {
|
||||
throw new ConfigurationException("Wrong configuration [akka.actor.default-dispatcher]")
|
||||
}
|
||||
/**
|
||||
* The one and only default dispatcher.
|
||||
*/
|
||||
def defaultGlobalDispatcher: MessageDispatcher = lookup(DefaultDispatcherId)
|
||||
|
||||
// FIXME: Dispatchers registered here are are not removed, see ticket #1494
|
||||
private val dispatchers = new ConcurrentHashMap[String, MessageDispatcher]
|
||||
// FIXME: Configurators registered here are are not removed, see ticket #1494
|
||||
private val dispatcherConfigurators = new ConcurrentHashMap[String, MessageDispatcherConfigurator]
|
||||
|
||||
/**
|
||||
* Returns a dispatcher as specified in configuration, or if not defined it uses
|
||||
* the default dispatcher. The same dispatcher instance is returned for subsequent
|
||||
* lookups.
|
||||
*/
|
||||
def lookup(key: String): MessageDispatcher = {
|
||||
dispatchers.get(key) match {
|
||||
def lookup(id: String): MessageDispatcher = lookupConfigurator(id).dispatcher()
|
||||
|
||||
private def lookupConfigurator(id: String): MessageDispatcherConfigurator = {
|
||||
dispatcherConfigurators.get(id) match {
|
||||
case null ⇒
|
||||
// It doesn't matter if we create a dispatcher that isn't used due to concurrent lookup.
|
||||
// It doesn't matter if we create a dispatcher configurator that isn't used due to concurrent lookup.
|
||||
// That shouldn't happen often and in case it does the actual ExecutorService isn't
|
||||
// created until used, i.e. cheap.
|
||||
val newDispatcher = newFromConfig(key)
|
||||
dispatchers.putIfAbsent(key, newDispatcher) match {
|
||||
case null ⇒ newDispatcher
|
||||
val newConfigurator =
|
||||
if (settings.config.hasPath(id)) {
|
||||
configuratorFrom(config(id))
|
||||
} else {
|
||||
// Note that the configurator of the default dispatcher will be registered for this id,
|
||||
// so this will only be logged once, which is crucial.
|
||||
prerequisites.eventStream.publish(Warning("Dispatchers",
|
||||
"Dispatcher [%s] not configured, using default-dispatcher".format(id)))
|
||||
lookupConfigurator(DefaultDispatcherId)
|
||||
}
|
||||
|
||||
dispatcherConfigurators.putIfAbsent(id, newConfigurator) match {
|
||||
case null ⇒ newConfigurator
|
||||
case existing ⇒ existing
|
||||
}
|
||||
|
||||
case existing ⇒ existing
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an thread based dispatcher serving a single actor through the same single thread.
|
||||
* <p/>
|
||||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
|
||||
new PinnedDispatcher(prerequisites, null, name, mailboxType, settings.DispatcherDefaultShutdown)
|
||||
|
||||
/**
|
||||
* Creates an thread based dispatcher serving a single actor through the same single thread.
|
||||
* <p/>
|
||||
* E.g. each actor consumes its own thread.
|
||||
*/
|
||||
def newPinnedDispatcher(name: String) =
|
||||
new PinnedDispatcher(prerequisites, null, name, MailboxType, settings.DispatcherDefaultShutdown)
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newDispatcher(name: String) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(prerequisites, name, settings.DispatcherThroughput,
|
||||
settings.DispatcherThroughputDeadlineTime, MailboxType, config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(prerequisites, name, throughput, settings.DispatcherThroughputDeadlineTime, mailboxType,
|
||||
config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newDispatcher(name: String, throughput: Int, throughputDeadline: Duration, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new Dispatcher(prerequisites, name, throughput, throughputDeadline, mailboxType, config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-sharing, serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newBalancingDispatcher(name: String) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(prerequisites, name, settings.DispatcherThroughput,
|
||||
settings.DispatcherThroughputDeadlineTime, MailboxType, config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-sharing, serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(prerequisites, name, throughput, settings.DispatcherThroughputDeadlineTime, MailboxType,
|
||||
config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-sharing, serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(prerequisites, name, throughput, settings.DispatcherThroughputDeadlineTime, mailboxType,
|
||||
config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a executor-based event-driven dispatcher, with work-sharing, serving multiple (millions) of actors through a thread pool.
|
||||
* <p/>
|
||||
* Has a fluent builder interface for configuring its semantics.
|
||||
*/
|
||||
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadline: Duration, mailboxType: MailboxType) =
|
||||
ThreadPoolConfigDispatcherBuilder(config ⇒
|
||||
new BalancingDispatcher(prerequisites, name, throughput, throughputDeadline, mailboxType,
|
||||
config, settings.DispatcherDefaultShutdown), ThreadPoolConfig())
|
||||
|
||||
/**
|
||||
* Creates a new dispatcher as specified in configuration
|
||||
* or if not defined it uses the supplied dispatcher.
|
||||
* Uses default values from default-dispatcher, i.e. all options doesn't need to be defined.
|
||||
*/
|
||||
def newFromConfig(key: String, default: ⇒ MessageDispatcher, cfg: Config): MessageDispatcher = {
|
||||
import scala.collection.JavaConverters._
|
||||
def simpleName = key.substring(key.lastIndexOf('.') + 1)
|
||||
cfg.hasPath(key) match {
|
||||
case false ⇒ default
|
||||
case true ⇒
|
||||
val conf = cfg.getConfig(key)
|
||||
val confWithName = conf.withFallback(ConfigFactory.parseMap(Map("name" -> simpleName).asJava))
|
||||
from(confWithName).getOrElse(throw new ConfigurationException("Wrong configuration [%s]".format(key)))
|
||||
}
|
||||
// FIXME #1563: Remove this method when dispatcher usage is rewritten in ActorModelSpec and CallingThreadDispatcherModelSpec
|
||||
private[akka] def register(id: String, dispatcherConfigurator: MessageDispatcherConfigurator): Unit = {
|
||||
dispatcherConfigurators.putIfAbsent(id, dispatcherConfigurator)
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new dispatcher as specified in configuration, or if not defined it uses
|
||||
* the default dispatcher.
|
||||
* Uses default configuration values from default-dispatcher, i.e. all options doesn't
|
||||
* need to be defined.
|
||||
*/
|
||||
def newFromConfig(key: String): MessageDispatcher = newFromConfig(key, defaultGlobalDispatcher, settings.config)
|
||||
private def config(id: String): Config = {
|
||||
import scala.collection.JavaConverters._
|
||||
def simpleName = id.substring(id.lastIndexOf('.') + 1)
|
||||
idConfig(id)
|
||||
.withFallback(settings.config.getConfig(id))
|
||||
.withFallback(ConfigFactory.parseMap(Map("name" -> simpleName).asJava))
|
||||
.withFallback(defaultDispatcherConfig)
|
||||
}
|
||||
|
||||
private def idConfig(id: String): Config = {
|
||||
import scala.collection.JavaConverters._
|
||||
ConfigFactory.parseMap(Map("id" -> id).asJava)
|
||||
}
|
||||
|
||||
/*
|
||||
* Creates of obtains a dispatcher from a ConfigMap according to the format below.
|
||||
* Uses default values from default-dispatcher.
|
||||
* Creates a dispatcher from a Config. Internal test purpose only.
|
||||
*
|
||||
* my-dispatcher {
|
||||
* type = "Dispatcher" # Must be one of the following
|
||||
* # Dispatcher, (BalancingDispatcher, only valid when all actors using it are of the same type),
|
||||
* # A FQCN to a class inheriting MessageDispatcherConfigurator with a no-arg visible constructor
|
||||
* name = "MyDispatcher" # Optional, will be a generated UUID if omitted
|
||||
* keep-alive-time = 60 # Keep alive time for threads in akka.time-unit
|
||||
* core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor)
|
||||
* max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor)
|
||||
* allow-core-timeout = on # Allow core threads to time out
|
||||
* throughput = 5 # Throughput for Dispatcher
|
||||
* }
|
||||
* ex: from(config.getConfig(identifier).get)
|
||||
* ex: from(config.getConfig(id))
|
||||
*
|
||||
* The Config must also contain a `id` property, which is the identifier of the dispatcher.
|
||||
*
|
||||
* Gotcha: Only configures the dispatcher if possible
|
||||
* Throws: IllegalArgumentException if the value of "type" is not valid
|
||||
* IllegalArgumentException if it cannot create the MessageDispatcherConfigurator
|
||||
*/
|
||||
def from(cfg: Config): Option[MessageDispatcher] = {
|
||||
val cfgWithFallback = cfg.withFallback(defaultDispatcherConfig)
|
||||
private[akka] def from(cfg: Config): MessageDispatcher = {
|
||||
configuratorFrom(cfg).dispatcher()
|
||||
}
|
||||
|
||||
val dispatcherConfigurator = cfgWithFallback.getString("type") match {
|
||||
case "Dispatcher" ⇒ Some(new DispatcherConfigurator())
|
||||
case "BalancingDispatcher" ⇒ Some(new BalancingDispatcherConfigurator())
|
||||
/*
|
||||
* Creates a MessageDispatcherConfigurator from a Config.
|
||||
*
|
||||
* The Config must also contain a `id` property, which is the identifier of the dispatcher.
|
||||
*
|
||||
* Throws: IllegalArgumentException if the value of "type" is not valid
|
||||
* IllegalArgumentException if it cannot create the MessageDispatcherConfigurator
|
||||
*/
|
||||
private def configuratorFrom(cfg: Config): MessageDispatcherConfigurator = {
|
||||
if (!cfg.hasPath("id")) throw new IllegalArgumentException("Missing dispatcher 'id' property in config: " + cfg.root.render)
|
||||
|
||||
cfg.getString("type") match {
|
||||
case "Dispatcher" ⇒ new DispatcherConfigurator(cfg, prerequisites)
|
||||
case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator(cfg, prerequisites)
|
||||
case "PinnedDispatcher" ⇒ new PinnedDispatcherConfigurator(cfg, prerequisites)
|
||||
case fqn ⇒
|
||||
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
|
||||
case Right(clazz) ⇒
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](clazz, Array[Class[_]](), Array[AnyRef]()) match {
|
||||
case Right(configurator) ⇒ Some(configurator)
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException(
|
||||
"Cannot instantiate MessageDispatcherConfigurator type [%s], make sure it has a default no-args constructor" format fqn, exception)
|
||||
}
|
||||
val constructorSignature = Array[Class[_]](classOf[Config], classOf[DispatcherPrerequisites])
|
||||
ReflectiveAccess.createInstance[MessageDispatcherConfigurator](fqn, constructorSignature, Array[AnyRef](cfg, prerequisites)) match {
|
||||
case Right(configurator) ⇒ configurator
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException("Unknown MessageDispatcherConfigurator type [%s]" format fqn, exception)
|
||||
throw new IllegalArgumentException(
|
||||
("Cannot instantiate MessageDispatcherConfigurator type [%s], defined in [%s], " +
|
||||
"make sure it has constructor with [com.typesafe.config.Config] and " +
|
||||
"[akka.dispatch.DispatcherPrerequisites] parameters")
|
||||
.format(fqn, cfg.getString("id")), exception)
|
||||
}
|
||||
}
|
||||
|
||||
dispatcherConfigurator map (_.configure(cfgWithFallback, settings, prerequisites))
|
||||
}
|
||||
}
|
||||
|
||||
class DispatcherConfigurator() extends MessageDispatcherConfigurator() {
|
||||
def configure(config: Config, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
|
||||
/**
|
||||
* Configurator for creating [[akka.dispatch.Dispatcher]].
|
||||
* Returns the same dispatcher instance for for each invocation
|
||||
* of the `dispatcher()` method.
|
||||
*/
|
||||
class DispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
|
||||
extends MessageDispatcherConfigurator(config, prerequisites) {
|
||||
|
||||
private val instance =
|
||||
configureThreadPool(config,
|
||||
settings,
|
||||
threadPoolConfig ⇒ new Dispatcher(prerequisites,
|
||||
config.getString("name"),
|
||||
config.getString("id"),
|
||||
config.getInt("throughput"),
|
||||
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
|
||||
mailboxType(config, settings),
|
||||
mailboxType,
|
||||
threadPoolConfig,
|
||||
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))).build
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the same dispatcher instance for each invocation
|
||||
*/
|
||||
override def dispatcher(): MessageDispatcher = instance
|
||||
}
|
||||
|
||||
class BalancingDispatcherConfigurator() extends MessageDispatcherConfigurator() {
|
||||
def configure(config: Config, settings: Settings, prerequisites: DispatcherPrerequisites): MessageDispatcher = {
|
||||
/**
|
||||
* Configurator for creating [[akka.dispatch.BalancingDispatcher]].
|
||||
* Returns the same dispatcher instance for for each invocation
|
||||
* of the `dispatcher()` method.
|
||||
*/
|
||||
class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
|
||||
extends MessageDispatcherConfigurator(config, prerequisites) {
|
||||
|
||||
private val instance =
|
||||
configureThreadPool(config,
|
||||
settings,
|
||||
threadPoolConfig ⇒ new BalancingDispatcher(prerequisites,
|
||||
config.getString("name"),
|
||||
config.getString("id"),
|
||||
config.getInt("throughput"),
|
||||
Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS),
|
||||
mailboxType(config, settings),
|
||||
mailboxType,
|
||||
threadPoolConfig,
|
||||
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))).build
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the same dispatcher instance for each invocation
|
||||
*/
|
||||
override def dispatcher(): MessageDispatcher = instance
|
||||
}
|
||||
|
||||
/**
|
||||
* Configurator for creating [[akka.dispatch.PinnedDispatcher]].
|
||||
* Returns new dispatcher instance for for each invocation
|
||||
* of the `dispatcher()` method.
|
||||
*/
|
||||
class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
|
||||
extends MessageDispatcherConfigurator(config, prerequisites) {
|
||||
/**
|
||||
* Creates new dispatcher for each invocation.
|
||||
*/
|
||||
override def dispatcher(): MessageDispatcher =
|
||||
new PinnedDispatcher(prerequisites, null, config.getString("name"), config.getString("id"), mailboxType,
|
||||
Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS))
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -37,6 +37,13 @@ object Mailbox {
|
|||
|
||||
/**
|
||||
* Custom mailbox implementations are implemented by extending this class.
|
||||
* E.g.
|
||||
* <pre<code>
|
||||
* class MyMailbox(owner: ActorContext) extends CustomMailbox(owner)
|
||||
* with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
|
||||
* val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||
* }
|
||||
* </code></pre>
|
||||
*/
|
||||
abstract class CustomMailbox(val actorContext: ActorContext) extends Mailbox(actorContext.asInstanceOf[ActorCell])
|
||||
|
||||
|
|
@ -373,29 +380,3 @@ case class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final va
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mailbox factory that creates instantiates the implementation from a
|
||||
* fully qualified class name. The implementation class must have
|
||||
* a constructor with a [[akka.actor.ActorContext]] parameter.
|
||||
* E.g.
|
||||
* <pre<code>
|
||||
* class MyMailbox(owner: ActorContext) extends CustomMailbox(owner)
|
||||
* with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
|
||||
* val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||
* }
|
||||
* </code></pre>
|
||||
*/
|
||||
class CustomMailboxType(mailboxFQN: String) extends MailboxType {
|
||||
|
||||
override def create(receiver: ActorContext): Mailbox = {
|
||||
val constructorSignature = Array[Class[_]](classOf[ActorContext])
|
||||
ReflectiveAccess.createInstance[Mailbox](mailboxFQN, constructorSignature, Array[AnyRef](receiver)) match {
|
||||
case Right(instance) ⇒ instance
|
||||
case Left(exception) ⇒
|
||||
throw new IllegalArgumentException("Cannot instantiate mailbox [%s] due to: %s".
|
||||
format(mailboxFQN, exception.toString))
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -14,15 +14,20 @@ import java.util.concurrent.TimeUnit
|
|||
|
||||
/**
|
||||
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
|
||||
*
|
||||
* The preferred way of creating dispatchers is to define configuration of it and use the
|
||||
* the `lookup` method in [[akka.dispatch.Dispatchers]].
|
||||
*/
|
||||
class PinnedDispatcher(
|
||||
_prerequisites: DispatcherPrerequisites,
|
||||
_actor: ActorCell,
|
||||
_name: String,
|
||||
_id: String,
|
||||
_mailboxType: MailboxType,
|
||||
_shutdownTimeout: Duration)
|
||||
extends Dispatcher(_prerequisites,
|
||||
_name,
|
||||
_id,
|
||||
Int.MaxValue,
|
||||
Duration.Zero,
|
||||
_mailboxType,
|
||||
|
|
|
|||
|
|
@ -93,7 +93,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒
|
|||
|
||||
protected[akka] var _delegates = Vector[ActorRef]()
|
||||
|
||||
val defaultProps: Props = Props.default.withDispatcher(this.context.dispatcher)
|
||||
val defaultProps: Props = Props.default.withDispatcher(this.context.dispatcher.id)
|
||||
|
||||
override def preStart() {
|
||||
resizeIfAppropriate()
|
||||
|
|
|
|||
|
|
@ -5,9 +5,8 @@ package akka.routing
|
|||
|
||||
import akka.actor._
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.util.Timeout
|
||||
import scala.collection.JavaConversions._
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.util.{ Duration, Timeout }
|
||||
|
||||
/**
|
||||
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
|
||||
|
|
@ -80,10 +79,6 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
*/
|
||||
trait RouterConfig {
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
||||
def createRoute(props: Props, actorContext: ActorContext, ref: RoutedActorRef): Route
|
||||
|
||||
def createActor(): Router = new Router {}
|
||||
|
|
@ -172,8 +167,6 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
|
|||
* Oxymoron style.
|
||||
*/
|
||||
case object NoRouter extends RouterConfig {
|
||||
def nrOfInstances: Int = 0
|
||||
def routees: Iterable[String] = Nil
|
||||
def createRoute(props: Props, actorContext: ActorContext, ref: RoutedActorRef): Route = null
|
||||
}
|
||||
|
||||
|
|
@ -211,6 +204,11 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
|||
}
|
||||
|
||||
trait RoundRobinLike { this: RouterConfig ⇒
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
||||
def createRoute(props: Props, context: ActorContext, ref: RoutedActorRef): Route = {
|
||||
createAndRegisterRoutees(props, context, nrOfInstances, routees)
|
||||
|
||||
|
|
@ -267,6 +265,10 @@ trait RandomLike { this: RouterConfig ⇒
|
|||
|
||||
import java.security.SecureRandom
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
||||
private val random = new ThreadLocal[SecureRandom] {
|
||||
override def initialValue = SecureRandom.getInstance("SHA1PRNG")
|
||||
}
|
||||
|
|
@ -322,6 +324,11 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
|||
}
|
||||
|
||||
trait BroadcastLike { this: RouterConfig ⇒
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
||||
def createRoute(props: Props, context: ActorContext, ref: RoutedActorRef): Route = {
|
||||
createAndRegisterRoutees(props, context, nrOfInstances, routees)
|
||||
|
||||
|
|
@ -335,7 +342,7 @@ trait BroadcastLike { this: RouterConfig ⇒
|
|||
}
|
||||
|
||||
object ScatterGatherFirstCompletedRouter {
|
||||
def apply(routees: Iterable[ActorRef]) = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString))
|
||||
def apply(routees: Iterable[ActorRef], within: Duration) = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||
}
|
||||
/**
|
||||
* Simple router that broadcasts the message to all routees, and replies with the first response.
|
||||
|
|
@ -348,33 +355,40 @@ object ScatterGatherFirstCompletedRouter {
|
|||
* if you provide either 'nrOfInstances' or 'routees' to during instantiation they will
|
||||
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
|
||||
*/
|
||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil)
|
||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration)
|
||||
extends RouterConfig with ScatterGatherFirstCompletedLike {
|
||||
|
||||
/**
|
||||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int) = {
|
||||
this(nrOfInstances = nr)
|
||||
def this(nr: Int, w: Duration) = {
|
||||
this(nrOfInstances = nr, within = w)
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
* Java API
|
||||
*/
|
||||
def this(t: java.util.Collection[String]) = {
|
||||
this(routees = collectionAsScalaIterable(t))
|
||||
def this(t: java.util.Collection[String], w: Duration) = {
|
||||
this(routees = collectionAsScalaIterable(t), within = w)
|
||||
}
|
||||
}
|
||||
|
||||
trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
||||
def within: Duration
|
||||
|
||||
def createRoute(props: Props, context: ActorContext, ref: RoutedActorRef): Route = {
|
||||
createAndRegisterRoutees(props, context, nrOfInstances, routees)
|
||||
|
||||
{
|
||||
case (sender, message) ⇒
|
||||
val asker = context.asInstanceOf[ActorCell].systemImpl.provider.ask(Timeout(5, TimeUnit.SECONDS)).get // FIXME, NO REALLY FIXME!
|
||||
val asker = context.asInstanceOf[ActorCell].systemImpl.provider.ask(Timeout(within)).get
|
||||
asker.result.pipeTo(sender)
|
||||
message match {
|
||||
case _ ⇒ toAll(asker, ref.routees)
|
||||
|
|
|
|||
22
akka-agent/src/main/resources/reference.conf
Normal file
22
akka-agent/src/main/resources/reference.conf
Normal file
|
|
@ -0,0 +1,22 @@
|
|||
####################################
|
||||
# Akka Agent Reference Config File #
|
||||
####################################
|
||||
|
||||
# This the reference config file has all the default settings.
|
||||
# Make your edits/overrides in your application.conf.
|
||||
|
||||
akka {
|
||||
agent {
|
||||
|
||||
# The dispatcher used for agent-send-off actor
|
||||
send-off-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
# The dispatcher used for agent-alter-off actor
|
||||
alter-off-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -153,8 +153,7 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
def sendOff(f: T ⇒ T): Unit = {
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val pinnedDispatcher = new PinnedDispatcher(system.dispatcherFactory.prerequisites, null, "agent-send-off", UnboundedMailbox(), system.settings.ActorTimeout.duration)
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.send-off-dispatcher"))
|
||||
threadBased ! Update(f)
|
||||
value
|
||||
})
|
||||
|
|
@ -171,8 +170,7 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
val result = Promise[T]()(system.dispatcher)
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val pinnedDispatcher = new PinnedDispatcher(system.dispatcherFactory.prerequisites, null, "agent-alter-off", UnboundedMailbox(), system.settings.ActorTimeout.duration)
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.alter-off-dispatcher"))
|
||||
result completeWith threadBased.?(Alter(f), timeout).asInstanceOf[Future[T]]
|
||||
value
|
||||
})
|
||||
|
|
|
|||
|
|
@ -1,513 +0,0 @@
|
|||
.. _stm-java:
|
||||
|
||||
Software Transactional Memory (Java)
|
||||
====================================
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Overview of STM
|
||||
---------------
|
||||
|
||||
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the Java heap into a transactional data set with begin/commit/rollback semantics. Very much like a regular database. It implements the first three letters in ACID; ACI:
|
||||
* (failure) Atomicity: all changes during the execution of a transaction make it, or none make it. This only counts for transactional datastructures.
|
||||
* Consistency: a transaction gets a consistent of reality (in Akka you get the Oracle version of the SERIALIZED isolation level).
|
||||
* Isolated: changes made by concurrent execution transactions are not visible to each other.
|
||||
|
||||
Generally, the STM is not needed that often when working with Akka. Some use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
- When you need to use the persistence modules.
|
||||
|
||||
Akka’s STM implements the concept in `Clojure’s <http://clojure.org/>`_ STM view on state in general. Please take the time to read `this excellent document <http://clojure.org/state>`_ and view `this presentation <http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey>`_ by Rich Hickey (the genius behind Clojure), since it forms the basis of Akka’s view on STM and state in general.
|
||||
|
||||
The STM is based on Transactional References (referred to as Refs). Refs are memory cells, holding an (arbitrary) immutable value, that implement CAS (Compare-And-Swap) semantics and are managed and enforced by the STM for coordinated changes across many Refs. They are implemented using the excellent `Multiverse STM <http://multiverse.codehaus.org/overview.html>`_.
|
||||
|
||||
Working with immutable collections can sometimes give bad performance due to extensive copying. Scala provides so-called persistent datastructures which makes working with immutable collections fast. They are immutable but with constant time access and modification. The use of structural sharing and an insert or update does not ruin the old structure, hence “persistent”. Makes working with immutable composite types fast. The persistent datastructures currently consist of a Map and Vector.
|
||||
|
||||
Simple example
|
||||
--------------
|
||||
|
||||
Here is a simple example of an incremental counter using STM. This shows creating a ``Ref``, a transactional reference, and then modifying it within a transaction, which is delimited by an ``Atomic`` anonymous inner class.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
|
||||
public int counter() {
|
||||
return new Atomic<Integer>() {
|
||||
public Integer atomically() {
|
||||
int inc = ref.get() + 1;
|
||||
ref.set(inc);
|
||||
return inc;
|
||||
}
|
||||
}.execute();
|
||||
}
|
||||
|
||||
counter();
|
||||
// -> 1
|
||||
|
||||
counter();
|
||||
// -> 2
|
||||
|
||||
|
||||
Ref
|
||||
---
|
||||
|
||||
Refs (transactional references) are mutable references to values and through the STM allow the safe sharing of mutable data. To ensure safety the value stored in a Ref should be immutable. The value referenced by a Ref can only be accessed or swapped within a transaction. Refs separate identity from value.
|
||||
|
||||
Creating a Ref
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
You can create a Ref with or without an initial value.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
// giving an initial value
|
||||
final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
|
||||
// specifying a type but no initial value
|
||||
final Ref<Integer> ref = new Ref<Integer>();
|
||||
|
||||
Accessing the value of a Ref
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Use ``get`` to access the value of a Ref. Note that if no initial value has been given then the value is initially ``null``.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
|
||||
Integer value = new Atomic<Integer>() {
|
||||
public Integer atomically() {
|
||||
return ref.get();
|
||||
}
|
||||
}.execute();
|
||||
// -> value = 0
|
||||
|
||||
Changing the value of a Ref
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
To set a new value for a Ref you can use ``set`` (or equivalently ``swap``), which sets the new value and returns the old value.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
return ref.set(5);
|
||||
}
|
||||
}.execute();
|
||||
|
||||
|
||||
Transactions
|
||||
------------
|
||||
|
||||
A transaction is delimited using an ``Atomic`` anonymous inner class.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
// ...
|
||||
}
|
||||
}.execute();
|
||||
|
||||
All changes made to transactional objects are isolated from other changes, all make it or non make it (so failure atomicity) and are consistent. With the AkkaSTM you automatically have the Oracle version of the SERIALIZED isolation level, lower isolation is not possible. To make it fully serialized, set the writeskew property that checks if a writeskew problem is allowed to happen.
|
||||
|
||||
Retries
|
||||
^^^^^^^
|
||||
|
||||
A transaction is automatically retried when it runs into some read or write conflict, until the operation completes, an exception (throwable) is thrown or when there are too many retries. When a read or writeconflict is encountered, the transaction uses a bounded exponential backoff to prevent cause more contention and give other transactions some room to complete.
|
||||
|
||||
If you are using non transactional resources in an atomic block, there could be problems because a transaction can be retried. If you are using print statements or logging, it could be that they are called more than once. So you need to be prepared to deal with this. One of the possible solutions is to work with a deferred or compensating task that is executed after the transaction aborts or commits.
|
||||
|
||||
Unexpected retries
|
||||
^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It can happen for the first few executions that you get a few failures of execution that lead to unexpected retries, even though there is not any read or writeconflict. The cause of this is that speculative transaction configuration/selection is used. There are transactions optimized for a single transactional object, for 1..n and for n to unlimited. So based on the execution of the transaction, the system learns; it begins with a cheap one and upgrades to more expensive ones. Once it has learned, it will reuse this knowledge. It can be activated/deactivated using the speculative property on the TransactionFactoryBuilder. In most cases it is best use the default value (enabled) so you get more out of performance.
|
||||
|
||||
Coordinated transactions and Transactors
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
If you need coordinated transactions across actors or threads then see :ref:`transactors-java`.
|
||||
|
||||
Configuring transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It's possible to configure transactions. The ``Atomic`` class can take a ``TransactionFactory``, which can determine properties of the transaction. A default transaction factory is used if none is specified. You can create a ``TransactionFactory`` with a ``TransactionFactoryBuilder``.
|
||||
|
||||
Configuring transactions with a ``TransactionFactory``:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
TransactionFactory txFactory = new TransactionFactoryBuilder()
|
||||
.setReadonly(true)
|
||||
.build();
|
||||
|
||||
new Atomic<Object>(txFactory) {
|
||||
public Object atomically() {
|
||||
// read only transaction
|
||||
return ...;
|
||||
}
|
||||
}.execute();
|
||||
|
||||
The following settings are possible on a TransactionFactory:
|
||||
|
||||
- familyName - Family name for transactions. Useful for debugging because the familyName is shown in exceptions, logging and in the future also will be used for profiling.
|
||||
- readonly - Sets transaction as readonly. Readonly transactions are cheaper and can be used to prevent modification to transactional objects.
|
||||
- maxRetries - The maximum number of times a transaction will retry.
|
||||
- timeout - The maximum time a transaction will block for.
|
||||
- trackReads - Whether all reads should be tracked. Needed for blocking operations. Readtracking makes a transaction more expensive, but makes subsequent reads cheaper and also lowers the chance of a readconflict.
|
||||
- writeSkew - Whether writeskew is allowed. Disable with care.
|
||||
- blockingAllowed - Whether explicit retries are allowed.
|
||||
- interruptible - Whether a blocking transaction can be interrupted if it is blocked.
|
||||
- speculative - Whether speculative configuration should be enabled.
|
||||
- quickRelease - Whether locks should be released as quickly as possible (before whole commit).
|
||||
- propagation - For controlling how nested transactions behave.
|
||||
- traceLevel - Transaction trace level.
|
||||
|
||||
You can also specify the default values for some of these options in :ref:`configuration`.
|
||||
|
||||
Transaction lifecycle listeners
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It's possible to have code that will only run on the successful commit of a transaction, or when a transaction aborts. You can do this by adding ``deferred`` or ``compensating`` blocks to a transaction.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
import static akka.stm.StmUtils.deferred;
|
||||
import static akka.stm.StmUtils.compensating;
|
||||
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
deferred(new Runnable() {
|
||||
public void run() {
|
||||
// executes when transaction commits
|
||||
}
|
||||
});
|
||||
compensating(new Runnable() {
|
||||
public void run() {
|
||||
// executes when transaction aborts
|
||||
}
|
||||
});
|
||||
// ...
|
||||
return something;
|
||||
}
|
||||
}.execute();
|
||||
|
||||
Blocking transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
You can block in a transaction until a condition is met by using an explicit ``retry``. To use ``retry`` you also need to configure the transaction to allow explicit retries.
|
||||
|
||||
Here is an example of using ``retry`` to block until an account has enough money for a withdrawal. This is also an example of using actors and STM together.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
public class Transfer {
|
||||
private final Ref<Double> from;
|
||||
private final Ref<Double> to;
|
||||
private final double amount;
|
||||
|
||||
public Transfer(Ref<Double> from, Ref<Double> to, double amount) {
|
||||
this.from = from;
|
||||
this.to = to;
|
||||
this.amount = amount;
|
||||
}
|
||||
|
||||
public Ref<Double> getFrom() { return from; }
|
||||
public Ref<Double> getTo() { return to; }
|
||||
public double getAmount() { return amount; }
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
import static akka.stm.StmUtils.retry;
|
||||
import akka.actor.*;
|
||||
import akka.util.FiniteDuration;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import akka.event.EventHandler;
|
||||
|
||||
public class Transferer extends UntypedActor {
|
||||
TransactionFactory txFactory = new TransactionFactoryBuilder()
|
||||
.setBlockingAllowed(true)
|
||||
.setTrackReads(true)
|
||||
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
|
||||
.build();
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Transfer) {
|
||||
Transfer transfer = (Transfer) message;
|
||||
final Ref<Double> from = transfer.getFrom();
|
||||
final Ref<Double> to = transfer.getTo();
|
||||
final double amount = transfer.getAmount();
|
||||
new Atomic(txFactory) {
|
||||
public Object atomically() {
|
||||
if (from.get() < amount) {
|
||||
EventHandler.info(this, "not enough money - retrying");
|
||||
retry();
|
||||
}
|
||||
EventHandler.info(this, "transferring");
|
||||
from.set(from.get() - amount);
|
||||
to.set(to.get() + amount);
|
||||
return null;
|
||||
}
|
||||
}.execute();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
import akka.actor.*;
|
||||
|
||||
public class Main {
|
||||
public static void main(String...args) throws Exception {
|
||||
final Ref<Double> account1 = new Ref<Double>(100.0);
|
||||
final Ref<Double> account2 = new Ref<Double>(100.0);
|
||||
|
||||
ActorRef transferer = Actors.actorOf(Transferer.class);
|
||||
|
||||
transferer.tell(new Transfer(account1, account2, 500.0));
|
||||
// Transferer: not enough money - retrying
|
||||
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
return account1.set(account1.get() + 2000);
|
||||
}
|
||||
}.execute();
|
||||
// Transferer: transferring
|
||||
|
||||
Thread.sleep(1000);
|
||||
|
||||
Double acc1 = new Atomic<Double>() {
|
||||
public Double atomically() {
|
||||
return account1.get();
|
||||
}
|
||||
}.execute();
|
||||
|
||||
Double acc2 = new Atomic<Double>() {
|
||||
public Double atomically() {
|
||||
return account2.get();
|
||||
}
|
||||
}.execute();
|
||||
|
||||
|
||||
|
||||
System.out.println("Account 1: " + acc1);
|
||||
// Account 1: 1600.0
|
||||
|
||||
System.out.println("Account 2: " + acc2);
|
||||
// Account 2: 600.0
|
||||
|
||||
transferer.stop();
|
||||
}
|
||||
}
|
||||
|
||||
Alternative blocking transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
You can also have two alternative blocking transactions, one of which can succeed first, with ``EitherOrElse``.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
public class Branch {
|
||||
private final Ref<Integer> left;
|
||||
private final Ref<Integer> right;
|
||||
private final double amount;
|
||||
|
||||
public Branch(Ref<Integer> left, Ref<Integer> right, int amount) {
|
||||
this.left = left;
|
||||
this.right = right;
|
||||
this.amount = amount;
|
||||
}
|
||||
|
||||
public Ref<Integer> getLeft() { return left; }
|
||||
|
||||
public Ref<Integer> getRight() { return right; }
|
||||
|
||||
public double getAmount() { return amount; }
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.actor.*;
|
||||
import akka.stm.*;
|
||||
import static akka.stm.StmUtils.retry;
|
||||
import akka.util.FiniteDuration;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import akka.event.EventHandler;
|
||||
|
||||
public class Brancher extends UntypedActor {
|
||||
TransactionFactory txFactory = new TransactionFactoryBuilder()
|
||||
.setBlockingAllowed(true)
|
||||
.setTrackReads(true)
|
||||
.setTimeout(new FiniteDuration(60, TimeUnit.SECONDS))
|
||||
.build();
|
||||
|
||||
public void onReceive(Object message) throws Exception {
|
||||
if (message instanceof Branch) {
|
||||
Branch branch = (Branch) message;
|
||||
final Ref<Integer> left = branch.getLeft();
|
||||
final Ref<Integer> right = branch.getRight();
|
||||
final double amount = branch.getAmount();
|
||||
new Atomic<Integer>(txFactory) {
|
||||
public Integer atomically() {
|
||||
return new EitherOrElse<Integer>() {
|
||||
public Integer either() {
|
||||
if (left.get() < amount) {
|
||||
EventHandler.info(this, "not enough on left - retrying");
|
||||
retry();
|
||||
}
|
||||
EventHandler.info(this, "going left");
|
||||
return left.get();
|
||||
}
|
||||
public Integer orElse() {
|
||||
if (right.get() < amount) {
|
||||
EventHandler.info(this, "not enough on right - retrying");
|
||||
retry();
|
||||
}
|
||||
EventHandler.info(this, "going right");
|
||||
return right.get();
|
||||
}
|
||||
}.execute();
|
||||
}
|
||||
}.execute();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
import akka.actor.*;
|
||||
|
||||
public class Main2 {
|
||||
public static void main(String...args) throws Exception {
|
||||
final Ref<Integer> left = new Ref<Integer>(100);
|
||||
final Ref<Integer> right = new Ref<Integer>(100);
|
||||
|
||||
ActorRef brancher = Actors.actorOf(Brancher.class);
|
||||
|
||||
brancher.tell(new Branch(left, right, 500));
|
||||
// not enough on left - retrying
|
||||
// not enough on right - retrying
|
||||
|
||||
Thread.sleep(1000);
|
||||
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
return right.set(right.get() + 1000);
|
||||
}
|
||||
}.execute();
|
||||
// going right
|
||||
|
||||
|
||||
|
||||
brancher.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Transactional datastructures
|
||||
----------------------------
|
||||
|
||||
Akka provides two datastructures that are managed by the STM.
|
||||
|
||||
- TransactionalMap
|
||||
- TransactionalVector
|
||||
|
||||
TransactionalMap and TransactionalVector look like regular mutable datastructures, they even implement the standard Scala 'Map' and 'RandomAccessSeq' interfaces, but they are implemented using persistent datastructures and managed references under the hood. Therefore they are safe to use in a concurrent environment. Underlying TransactionalMap is HashMap, an immutable Map but with near constant time access and modification operations. Similarly TransactionalVector uses a persistent Vector. See the Persistent Datastructures section below for more details.
|
||||
|
||||
Like managed references, TransactionalMap and TransactionalVector can only be modified inside the scope of an STM transaction.
|
||||
|
||||
Here is an example of creating and accessing a TransactionalMap:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
// assuming a User class
|
||||
|
||||
final TransactionalMap<String, User> users = new TransactionalMap<String, User>();
|
||||
|
||||
// fill users map (in a transaction)
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
users.put("bill", new User("bill"));
|
||||
users.put("mary", new User("mary"));
|
||||
users.put("john", new User("john"));
|
||||
return null;
|
||||
}
|
||||
}.execute();
|
||||
|
||||
// access users map (in a transaction)
|
||||
User user = new Atomic<User>() {
|
||||
public User atomically() {
|
||||
return users.get("bill").get();
|
||||
}
|
||||
}.execute();
|
||||
|
||||
Here is an example of creating and accessing a TransactionalVector:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.stm.*;
|
||||
|
||||
// assuming an Address class
|
||||
|
||||
final TransactionalVector<Address> addresses = new TransactionalVector<Address>();
|
||||
|
||||
// fill addresses vector (in a transaction)
|
||||
new Atomic() {
|
||||
public Object atomically() {
|
||||
addresses.add(new Address("somewhere"));
|
||||
addresses.add(new Address("somewhere else"));
|
||||
return null;
|
||||
}
|
||||
}.execute();
|
||||
|
||||
// access addresses vector (in a transaction)
|
||||
Address address = new Atomic<Address>() {
|
||||
public Address atomically() {
|
||||
return addresses.get(0);
|
||||
}
|
||||
}.execute();
|
||||
|
||||
|
||||
Persistent datastructures
|
||||
-------------------------
|
||||
|
||||
Akka's STM should only be used with immutable data. This can be costly if you have large datastructures and are using a naive copy-on-write. In order to make working with immutable datastructures fast enough Scala provides what are called Persistent Datastructures. There are currently two different ones:
|
||||
|
||||
- HashMap (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/HashMap.html>`__)
|
||||
- Vector (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/Vector.html>`__)
|
||||
|
||||
They are immutable and each update creates a completely new version but they are using clever structural sharing in order to make them almost as fast, for both read and update, as regular mutable datastructures.
|
||||
|
||||
This illustration is taken from Rich Hickey's presentation. Copyright Rich Hickey 2009.
|
||||
|
||||
.. image:: ../images/clojure-trees.png
|
||||
|
||||
|
||||
|
|
@ -1,269 +0,0 @@
|
|||
.. _transactors-java:
|
||||
|
||||
Transactors (Java)
|
||||
==================
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Why Transactors?
|
||||
----------------
|
||||
|
||||
Actors are excellent for solving problems where you have many independent processes that can work in isolation and only interact with other Actors through message passing. This model fits many problems. But the actor model is unfortunately a terrible model for implementing truly shared state. E.g. when you need to have consensus and a stable view of state across many components. The classic example is the bank account where clients can deposit and withdraw, in which each operation needs to be atomic. For detailed discussion on the topic see `this JavaOne presentation <http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
|
||||
|
||||
**STM** on the other hand is excellent for problems where you need consensus and a stable view of the state by providing compositional transactional shared state. Some of the really nice traits of STM are that transactions compose, and it raises the abstraction level from lock-based concurrency.
|
||||
|
||||
Akka's Transactors combine Actors and STM to provide the best of the Actor model (concurrency and asynchronous event-based programming) and STM (compositional transactional shared state) by providing transactional, compositional, asynchronous, event-based message flows.
|
||||
|
||||
If you need Durability then you should not use one of the in-memory data structures but one of the persistent ones.
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
- When you need to use the persistence modules.
|
||||
|
||||
Actors and STM
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
You can combine Actors and STM in several ways. An Actor may use STM internally so that particular changes are guaranteed to be atomic. Actors may also share transactional datastructures as the STM provides safe shared state across threads.
|
||||
|
||||
It's also possible to coordinate transactions across Actors or threads so that either the transactions in a set all commit successfully or they all fail. This is the focus of Transactors and the explicit support for coordinated transactions in this section.
|
||||
|
||||
|
||||
Coordinated transactions
|
||||
------------------------
|
||||
|
||||
Akka provides an explicit mechanism for coordinating transactions across actors. Under the hood it uses a ``CountDownCommitBarrier``, similar to a CountDownLatch.
|
||||
|
||||
Here is an example of coordinating two simple counter UntypedActors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
|
||||
public class Increment {
|
||||
private final ActorRef friend;
|
||||
|
||||
public Increment() {
|
||||
this.friend = null;
|
||||
}
|
||||
|
||||
public Increment(ActorRef friend) {
|
||||
this.friend = friend;
|
||||
}
|
||||
|
||||
public boolean hasFriend() {
|
||||
return friend != null;
|
||||
}
|
||||
|
||||
public ActorRef getFriend() {
|
||||
return friend;
|
||||
}
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.stm.Ref;
|
||||
import akka.transactor.Atomically;
|
||||
import akka.transactor.Coordinated;
|
||||
|
||||
public class Counter extends UntypedActor {
|
||||
private Ref<Integer> count = new Ref(0);
|
||||
|
||||
private void increment() {
|
||||
count.set(count.get() + 1);
|
||||
}
|
||||
|
||||
public void onReceive(Object incoming) throws Exception {
|
||||
if (incoming instanceof Coordinated) {
|
||||
Coordinated coordinated = (Coordinated) incoming;
|
||||
Object message = coordinated.getMessage();
|
||||
if (message instanceof Increment) {
|
||||
Increment increment = (Increment) message;
|
||||
if (increment.hasFriend()) {
|
||||
increment.getFriend().tell(coordinated.coordinate(new Increment()));
|
||||
}
|
||||
coordinated.atomic(new Atomically() {
|
||||
public void atomically() {
|
||||
increment();
|
||||
}
|
||||
});
|
||||
}
|
||||
} else if (incoming.equals("GetCount")) {
|
||||
getContext().reply(count.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
ActorRef counter1 = actorOf(Counter.class);
|
||||
ActorRef counter2 = actorOf(Counter.class);
|
||||
|
||||
counter1.tell(new Coordinated(new Increment(counter2)));
|
||||
|
||||
To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
Coordinated coordinated = new Coordinated();
|
||||
|
||||
To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
actor.tell(new Coordinated(new Message()));
|
||||
|
||||
To include another actor in the same coordinated transaction that you've created or received, use the ``coordinate`` method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
actor.tell(coordinated.coordinate(new Message()));
|
||||
|
||||
To enter the coordinated transaction use the atomic method of the coordinated object. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object the same as used normally in the STM (just don't execute it - the coordination will do that).
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
coordinated.atomic(new Atomically() {
|
||||
public void atomically() {
|
||||
// do something in a transaction
|
||||
}
|
||||
});
|
||||
|
||||
The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail.
|
||||
|
||||
|
||||
UntypedTransactor
|
||||
-----------------
|
||||
|
||||
UntypedTransactors are untyped actors that provide a general pattern for coordinating transactions, using the explicit coordination described above.
|
||||
|
||||
Here's an example of a simple untyped transactor that will join a coordinated transaction:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.transactor.UntypedTransactor;
|
||||
import akka.stm.Ref;
|
||||
|
||||
public class Counter extends UntypedTransactor {
|
||||
Ref<Integer> count = new Ref<Integer>(0);
|
||||
|
||||
@Override
|
||||
public void atomically(Object message) {
|
||||
if (message instanceof Increment) {
|
||||
count.set(count.get() + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own ``Coordinated`` (but in this particular case wouldn't be coordinating transactions with any other transactors).
|
||||
|
||||
To coordinate with other transactors override the ``coordinate`` method. The ``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of ``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods to easily coordinate with other transactors.
|
||||
|
||||
Example of coordinating an increment, similar to the explicitly coordinated example:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.transactor.UntypedTransactor;
|
||||
import akka.transactor.SendTo;
|
||||
import akka.stm.Ref;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
public class Counter extends UntypedTransactor {
|
||||
Ref<Integer> count = new Ref<Integer>(0);
|
||||
|
||||
@Override
|
||||
public Set<SendTo> coordinate(Object message) {
|
||||
if (message instanceof Increment) {
|
||||
Increment increment = (Increment) message;
|
||||
if (increment.hasFriend())
|
||||
return include(increment.getFriend(), new Increment());
|
||||
}
|
||||
return nobody();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void atomically(Object message) {
|
||||
if (message instanceof Increment) {
|
||||
count.set(count.get() + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. They do not execute within the transaction.
|
||||
|
||||
To completely bypass coordinated transactions override the ``normally`` method. Any message matched by ``normally`` will not be matched by the other methods, and will not be involved in coordinated transactions. In this method you can implement normal actor behavior, or use the normal STM atomic for local transactions.
|
||||
|
||||
|
||||
Coordinating Typed Actors
|
||||
-------------------------
|
||||
|
||||
It's also possible to use coordinated transactions with typed actors. You can explicitly pass around ``Coordinated`` objects, or use built-in support with the ``@Coordinated`` annotation and the ``Coordination.coordinate`` method.
|
||||
|
||||
To specify a method should use coordinated transactions add the ``@Coordinated`` annotation. **Note**: the ``@Coordinated`` annotation will only work with void (one-way) methods.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
public interface Counter {
|
||||
@Coordinated public void increment();
|
||||
public Integer get();
|
||||
}
|
||||
|
||||
To coordinate transactions use a ``coordinate`` block. This accepts either an ``akka.transactor.Atomically`` object, or an ``Atomic`` object liked used in the STM (but don't execute it). The first boolean parameter specifies whether or not to wait for the transactions to complete.
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
Coordination.coordinate(true, new Atomically() {
|
||||
public void atomically() {
|
||||
counter1.increment();
|
||||
counter2.increment();
|
||||
}
|
||||
});
|
||||
|
||||
Here's an example of using ``@Coordinated`` with a TypedActor to coordinate increments:
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.transactor.annotation.Coordinated;
|
||||
|
||||
public interface Counter {
|
||||
@Coordinated public void increment();
|
||||
public Integer get();
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
import akka.actor.TypedActor;
|
||||
import akka.stm.Ref;
|
||||
|
||||
public class CounterImpl extends TypedActor implements Counter {
|
||||
private Ref<Integer> count = new Ref<Integer>(0);
|
||||
|
||||
public void increment() {
|
||||
count.set(count.get() + 1);
|
||||
}
|
||||
|
||||
public Integer get() {
|
||||
return count.get();
|
||||
}
|
||||
}
|
||||
|
||||
.. code-block:: java
|
||||
|
||||
Counter counter1 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
|
||||
Counter counter2 = (Counter) TypedActor.newInstance(Counter.class, CounterImpl.class);
|
||||
|
||||
Coordination.coordinate(true, new Atomically() {
|
||||
public void atomically() {
|
||||
counter1.increment();
|
||||
counter2.increment();
|
||||
}
|
||||
});
|
||||
|
||||
TypedActor.stop(counter1);
|
||||
TypedActor.stop(counter2);
|
||||
|
||||
|
|
@ -1,537 +0,0 @@
|
|||
|
||||
.. _stm-scala:
|
||||
|
||||
#######################################
|
||||
Software Transactional Memory (Scala)
|
||||
#######################################
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Overview of STM
|
||||
===============
|
||||
|
||||
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the
|
||||
Java heap into a transactional data set with begin/commit/rollback
|
||||
semantics. Very much like a regular database. It implements the first three
|
||||
letters in ACID; ACI:
|
||||
|
||||
* Atomic
|
||||
* Consistent
|
||||
* Isolated
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some
|
||||
use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating
|
||||
their **internal local** state but need them to do that atomically in one big
|
||||
transaction. Might not be often, but when you do need this then you are
|
||||
screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
- When you need to use the persistence modules.
|
||||
|
||||
Akka’s STM implements the concept in `Clojure's <clojure>`_ STM view on state in
|
||||
general. Please take the time to read `this excellent document <clojure-state>`_
|
||||
and view `this presentation <clojure-presentation>`_ by Rich Hickey (the genius
|
||||
behind Clojure), since it forms the basis of Akka’s view on STM and state in
|
||||
general.
|
||||
|
||||
.. _clojure: http://clojure.org/
|
||||
.. _clojure-state: http://clojure.org/state
|
||||
.. _clojure-presentation: http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey
|
||||
|
||||
The STM is based on Transactional References (referred to as Refs). Refs are
|
||||
memory cells, holding an (arbitrary) immutable value, that implement CAS
|
||||
(Compare-And-Swap) semantics and are managed and enforced by the STM for
|
||||
coordinated changes across many Refs. They are implemented using the excellent
|
||||
`Multiverse STM <multiverse>`_.
|
||||
|
||||
.. _multiverse: http://multiverse.codehaus.org/overview.html
|
||||
|
||||
Working with immutable collections can sometimes give bad performance due to
|
||||
extensive copying. Scala provides so-called persistent datastructures which
|
||||
makes working with immutable collections fast. They are immutable but with
|
||||
constant time access and modification. They use structural sharing and an insert
|
||||
or update does not ruin the old structure, hence “persistent”. Makes working
|
||||
with immutable composite types fast. The persistent datastructures currently
|
||||
consist of a Map and Vector.
|
||||
|
||||
|
||||
Simple example
|
||||
==============
|
||||
|
||||
Here is a simple example of an incremental counter using STM. This shows
|
||||
creating a ``Ref``, a transactional reference, and then modifying it within a
|
||||
transaction, which is delimited by ``atomic``.
|
||||
|
||||
.. includecode:: code/StmDocSpec.scala#simple
|
||||
|
||||
|
||||
Ref
|
||||
---
|
||||
|
||||
Refs (transactional references) are mutable references to values and through the STM allow the safe sharing of mutable data. Refs separate identity from value. To ensure safety the value stored in a Ref should be immutable (they can of course contain refs themselves). The value referenced by a Ref can only be accessed or swapped within a transaction. If a transaction is not available, the call will be executed in its own transaction (the call will be atomic). This is a different approach than the Clojure Refs, where a missing transaction results in an error.
|
||||
|
||||
Creating a Ref
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
You can create a Ref with or without an initial value.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
// giving an initial value
|
||||
val ref = Ref(0)
|
||||
|
||||
// specifying a type but no initial value
|
||||
val ref = Ref[Int]
|
||||
|
||||
Accessing the value of a Ref
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Use ``get`` to access the value of a Ref. Note that if no initial value has been given then the value is initially ``null``.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val ref = Ref(0)
|
||||
|
||||
atomic {
|
||||
ref.get
|
||||
}
|
||||
// -> 0
|
||||
|
||||
If there is a chance that the value of a Ref is null then you can use ``opt``, which will create an Option, either Some(value) or None, or you can provide a default value with ``getOrElse``. You can also check for null using ``isNull``.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val ref = Ref[Int]
|
||||
|
||||
atomic {
|
||||
ref.opt // -> None
|
||||
ref.getOrElse(0) // -> 0
|
||||
ref.isNull // -> true
|
||||
}
|
||||
|
||||
Changing the value of a Ref
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
To set a new value for a Ref you can use ``set`` (or equivalently ``swap``), which sets the new value and returns the old value.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val ref = Ref(0)
|
||||
|
||||
atomic {
|
||||
ref.set(5)
|
||||
}
|
||||
// -> 0
|
||||
|
||||
atomic {
|
||||
ref.get
|
||||
}
|
||||
// -> 5
|
||||
|
||||
You can also use ``alter`` which accepts a function that takes the old value and creates a new value of the same type.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val ref = Ref(0)
|
||||
|
||||
atomic {
|
||||
ref alter (_ + 5)
|
||||
}
|
||||
// -> 5
|
||||
|
||||
val inc = (i: Int) => i + 1
|
||||
|
||||
atomic {
|
||||
ref alter inc
|
||||
}
|
||||
// -> 6
|
||||
|
||||
Refs in for-comprehensions
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Ref is monadic and can be used in for-comprehensions.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val ref = Ref(1)
|
||||
|
||||
atomic {
|
||||
for (value <- ref) {
|
||||
// do something with value
|
||||
}
|
||||
}
|
||||
|
||||
val anotherRef = Ref(3)
|
||||
|
||||
atomic {
|
||||
for {
|
||||
value1 <- ref
|
||||
value2 <- anotherRef
|
||||
} yield (value1 + value2)
|
||||
}
|
||||
// -> Ref(4)
|
||||
|
||||
val emptyRef = Ref[Int]
|
||||
|
||||
atomic {
|
||||
for {
|
||||
value1 <- ref
|
||||
value2 <- emptyRef
|
||||
} yield (value1 + value2)
|
||||
}
|
||||
// -> Ref[Int]
|
||||
|
||||
|
||||
Transactions
|
||||
------------
|
||||
|
||||
A transaction is delimited using ``atomic``.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
atomic {
|
||||
// ...
|
||||
}
|
||||
|
||||
All changes made to transactional objects are isolated from other changes, all make it or non make it (so failure atomicity) and are consistent. With the AkkaSTM you automatically have the Oracle version of the SERIALIZED isolation level, lower isolation is not possible. To make it fully serialized, set the writeskew property that checks if a writeskew problem is allowed to happen.
|
||||
|
||||
Retries
|
||||
^^^^^^^
|
||||
|
||||
A transaction is automatically retried when it runs into some read or write conflict, until the operation completes, an exception (throwable) is thrown or when there are too many retries. When a read or writeconflict is encountered, the transaction uses a bounded exponential backoff to prevent cause more contention and give other transactions some room to complete.
|
||||
|
||||
If you are using non transactional resources in an atomic block, there could be problems because a transaction can be retried. If you are using print statements or logging, it could be that they are called more than once. So you need to be prepared to deal with this. One of the possible solutions is to work with a deferred or compensating task that is executed after the transaction aborts or commits.
|
||||
|
||||
Unexpected retries
|
||||
^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It can happen for the first few executions that you get a few failures of execution that lead to unexpected retries, even though there is not any read or writeconflict. The cause of this is that speculative transaction configuration/selection is used. There are transactions optimized for a single transactional object, for 1..n and for n to unlimited. So based on the execution of the transaction, the system learns; it begins with a cheap one and upgrades to more expensive ones. Once it has learned, it will reuse this knowledge. It can be activated/deactivated using the speculative property on the TransactionFactory. In most cases it is best use the default value (enabled) so you get more out of performance.
|
||||
|
||||
Coordinated transactions and Transactors
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
If you need coordinated transactions across actors or threads then see :ref:`transactors-scala`.
|
||||
|
||||
Configuring transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It's possible to configure transactions. The ``atomic`` method can take an implicit or explicit ``TransactionFactory``, which can determine properties of the transaction. A default transaction factory is used if none is specified explicitly or there is no implicit ``TransactionFactory`` in scope.
|
||||
|
||||
Configuring transactions with an **implicit** ``TransactionFactory``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
implicit val txFactory = TransactionFactory(readonly = true)
|
||||
|
||||
atomic {
|
||||
// read only transaction
|
||||
}
|
||||
|
||||
Configuring transactions with an **explicit** ``TransactionFactory``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
val txFactory = TransactionFactory(readonly = true)
|
||||
|
||||
atomic(txFactory) {
|
||||
// read only transaction
|
||||
}
|
||||
|
||||
The following settings are possible on a TransactionFactory:
|
||||
|
||||
- ``familyName`` - Family name for transactions. Useful for debugging.
|
||||
- ``readonly`` - Sets transaction as readonly. Readonly transactions are cheaper.
|
||||
- ``maxRetries`` - The maximum number of times a transaction will retry.
|
||||
- ``timeout`` - The maximum time a transaction will block for.
|
||||
- ``trackReads`` - Whether all reads should be tracked. Needed for blocking operations.
|
||||
- ``writeSkew`` - Whether writeskew is allowed. Disable with care.
|
||||
- ``blockingAllowed`` - Whether explicit retries are allowed.
|
||||
- ``interruptible`` - Whether a blocking transaction can be interrupted.
|
||||
- ``speculative`` - Whether speculative configuration should be enabled.
|
||||
- ``quickRelease`` - Whether locks should be released as quickly as possible (before whole commit).
|
||||
- ``propagation`` - For controlling how nested transactions behave.
|
||||
- ``traceLevel`` - Transaction trace level.
|
||||
|
||||
You can also specify the default values for some of these options in the :ref:`configuration`.
|
||||
|
||||
You can also determine at which level a transaction factory is shared or not shared, which affects the way in which the STM can optimise transactions.
|
||||
|
||||
Here is a shared transaction factory for all instances of an actor.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.actor._
|
||||
import akka.stm._
|
||||
|
||||
object MyActor {
|
||||
implicit val txFactory = TransactionFactory(readonly = true)
|
||||
}
|
||||
|
||||
class MyActor extends Actor {
|
||||
import MyActor.txFactory
|
||||
|
||||
def receive = {
|
||||
case message: String =>
|
||||
atomic {
|
||||
// read only transaction
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Here's a similar example with an individual transaction factory for each instance of an actor.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.actor._
|
||||
import akka.stm._
|
||||
|
||||
class MyActor extends Actor {
|
||||
implicit val txFactory = TransactionFactory(readonly = true)
|
||||
|
||||
def receive = {
|
||||
case message: String =>
|
||||
atomic {
|
||||
// read only transaction
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Transaction lifecycle listeners
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
It's possible to have code that will only run on the successful commit of a transaction, or when a transaction aborts. You can do this by adding ``deferred`` or ``compensating`` blocks to a transaction.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
atomic {
|
||||
deferred {
|
||||
// executes when transaction commits
|
||||
}
|
||||
compensating {
|
||||
// executes when transaction aborts
|
||||
}
|
||||
}
|
||||
|
||||
Blocking transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
You can block in a transaction until a condition is met by using an explicit ``retry``. To use ``retry`` you also need to configure the transaction to allow explicit retries.
|
||||
|
||||
Here is an example of using ``retry`` to block until an account has enough money for a withdrawal. This is also an example of using actors and STM together.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
import akka.actor._
|
||||
import akka.util.duration._
|
||||
import akka.event.EventHandler
|
||||
|
||||
type Account = Ref[Double]
|
||||
|
||||
case class Transfer(from: Account, to: Account, amount: Double)
|
||||
|
||||
class Transferer extends Actor {
|
||||
implicit val txFactory = TransactionFactory(blockingAllowed = true, trackReads = true, timeout = 60 seconds)
|
||||
|
||||
def receive = {
|
||||
case Transfer(from, to, amount) =>
|
||||
atomic {
|
||||
if (from.get < amount) {
|
||||
EventHandler.info(this, "not enough money - retrying")
|
||||
retry
|
||||
}
|
||||
EventHandler.info(this, "transferring")
|
||||
from alter (_ - amount)
|
||||
to alter (_ + amount)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val account1 = Ref(100.0)
|
||||
val account2 = Ref(100.0)
|
||||
|
||||
val transferer = Actor.actorOf(new Transferer)
|
||||
|
||||
transferer ! Transfer(account1, account2, 500.0)
|
||||
// INFO Transferer: not enough money - retrying
|
||||
|
||||
atomic { account1 alter (_ + 2000) }
|
||||
// INFO Transferer: transferring
|
||||
|
||||
atomic { account1.get }
|
||||
// -> 1600.0
|
||||
|
||||
atomic { account2.get }
|
||||
// -> 600.0
|
||||
|
||||
transferer.stop()
|
||||
|
||||
Alternative blocking transactions
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
You can also have two alternative blocking transactions, one of which can succeed first, with ``either-orElse``.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
import akka.actor._
|
||||
import akka.util.duration._
|
||||
import akka.event.EventHandler
|
||||
|
||||
case class Branch(left: Ref[Int], right: Ref[Int], amount: Int)
|
||||
|
||||
class Brancher extends Actor {
|
||||
implicit val txFactory = TransactionFactory(blockingAllowed = true, trackReads = true, timeout = 60 seconds)
|
||||
|
||||
def receive = {
|
||||
case Branch(left, right, amount) =>
|
||||
atomic {
|
||||
either {
|
||||
if (left.get < amount) {
|
||||
EventHandler.info(this, "not enough on left - retrying")
|
||||
retry
|
||||
}
|
||||
log.info("going left")
|
||||
} orElse {
|
||||
if (right.get < amount) {
|
||||
EventHandler.info(this, "not enough on right - retrying")
|
||||
retry
|
||||
}
|
||||
log.info("going right")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val ref1 = Ref(0)
|
||||
val ref2 = Ref(0)
|
||||
|
||||
val brancher = Actor.actorOf(new Brancher)
|
||||
|
||||
brancher ! Branch(ref1, ref2, 1)
|
||||
// INFO Brancher: not enough on left - retrying
|
||||
// INFO Brancher: not enough on right - retrying
|
||||
|
||||
atomic { ref2 alter (_ + 1) }
|
||||
// INFO Brancher: not enough on left - retrying
|
||||
// INFO Brancher: going right
|
||||
|
||||
brancher.stop()
|
||||
|
||||
|
||||
Transactional datastructures
|
||||
----------------------------
|
||||
|
||||
Akka provides two datastructures that are managed by the STM.
|
||||
|
||||
- ``TransactionalMap``
|
||||
- ``TransactionalVector``
|
||||
|
||||
``TransactionalMap`` and ``TransactionalVector`` look like regular mutable datastructures, they even implement the standard Scala 'Map' and 'RandomAccessSeq' interfaces, but they are implemented using persistent datastructures and managed references under the hood. Therefore they are safe to use in a concurrent environment. Underlying TransactionalMap is HashMap, an immutable Map but with near constant time access and modification operations. Similarly ``TransactionalVector`` uses a persistent Vector. See the Persistent Datastructures section below for more details.
|
||||
|
||||
Like managed references, ``TransactionalMap`` and ``TransactionalVector`` can only be modified inside the scope of an STM transaction.
|
||||
|
||||
*IMPORTANT*: There have been some problems reported when using transactional datastructures with 'lazy' initialization. Avoid that.
|
||||
|
||||
Here is how you create these transactional datastructures:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
// assuming something like
|
||||
case class User(name: String)
|
||||
case class Address(location: String)
|
||||
|
||||
// using initial values
|
||||
val map = TransactionalMap("bill" -> User("bill"))
|
||||
val vector = TransactionalVector(Address("somewhere"))
|
||||
|
||||
// specifying types
|
||||
val map = TransactionalMap[String, User]
|
||||
val vector = TransactionalVector[Address]
|
||||
|
||||
``TransactionalMap`` and ``TransactionalVector`` wrap persistent datastructures with transactional references and provide a standard Scala interface. This makes them convenient to use.
|
||||
|
||||
Here is an example of using a ``Ref`` and a ``HashMap`` directly:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
import scala.collection.immutable.HashMap
|
||||
|
||||
case class User(name: String)
|
||||
|
||||
val ref = Ref(HashMap[String, User]())
|
||||
|
||||
atomic {
|
||||
val users = ref.get
|
||||
val newUsers = users + ("bill" -> User("bill")) // creates a new HashMap
|
||||
ref.swap(newUsers)
|
||||
}
|
||||
|
||||
atomic {
|
||||
ref.get.apply("bill")
|
||||
}
|
||||
// -> User("bill")
|
||||
|
||||
Here is the same example using ``TransactionalMap``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.stm._
|
||||
|
||||
case class User(name: String)
|
||||
|
||||
val users = TransactionalMap[String, User]
|
||||
|
||||
atomic {
|
||||
users += "bill" -> User("bill")
|
||||
}
|
||||
|
||||
atomic {
|
||||
users("bill")
|
||||
}
|
||||
// -> User("bill")
|
||||
|
||||
|
||||
Persistent datastructures
|
||||
-------------------------
|
||||
|
||||
Akka's STM should only be used with immutable data. This can be costly if you have large datastructures and are using a naive copy-on-write. In order to make working with immutable datastructures fast enough Scala provides what are called Persistent Datastructures. There are currently two different ones:
|
||||
|
||||
* ``HashMap`` (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/HashMap.html>`__)
|
||||
* ``Vector`` (`scaladoc <http://www.scala-lang.org/api/current/scala/collection/immutable/Vector.html>`__)
|
||||
|
||||
They are immutable and each update creates a completely new version but they are using clever structural sharing in order to make them almost as fast, for both read and update, as regular mutable datastructures.
|
||||
|
||||
This illustration is taken from Rich Hickey's presentation. Copyright Rich Hickey 2009.
|
||||
|
||||
.. image:: ../images/clojure-trees.png
|
||||
|
||||
|
||||
Ants simulation sample
|
||||
----------------------
|
||||
|
||||
One fun and very enlightening visual demo of STM, actors and transactional references is the `Ant simulation sample <http://github.com/jboner/akka/tree/master/akka-samples/akka-sample-ants/>`_. I encourage you to run it and read through the code since it's a good example of using actors with STM.
|
||||
|
|
@ -1,248 +0,0 @@
|
|||
.. _transactors-scala:
|
||||
|
||||
Transactors (Scala)
|
||||
===================
|
||||
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
Why Transactors?
|
||||
----------------
|
||||
|
||||
Actors are excellent for solving problems where you have many independent processes that can work in isolation and only interact with other Actors through message passing. This model fits many problems. But the actor model is unfortunately a terrible model for implementing truly shared state. E.g. when you need to have consensus and a stable view of state across many components. The classic example is the bank account where clients can deposit and withdraw, in which each operation needs to be atomic. For detailed discussion on the topic see `this JavaOne presentation <http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
|
||||
|
||||
**STM** on the other hand is excellent for problems where you need consensus and a stable view of the state by providing compositional transactional shared state. Some of the really nice traits of STM are that transactions compose, and it raises the abstraction level from lock-based concurrency.
|
||||
|
||||
Akka's Transactors combine Actors and STM to provide the best of the Actor model (concurrency and asynchronous event-based programming) and STM (compositional transactional shared state) by providing transactional, compositional, asynchronous, event-based message flows.
|
||||
|
||||
If you need Durability then you should not use one of the in-memory data structures but one of the persistent ones.
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating their **internal local** state but need them to do that atomically in one big transaction. Might not often, but when you do need this then you are screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
- When you need to use the persistence modules.
|
||||
|
||||
Actors and STM
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
You can combine Actors and STM in several ways. An Actor may use STM internally so that particular changes are guaranteed to be atomic. Actors may also share transactional datastructures as the STM provides safe shared state across threads.
|
||||
|
||||
It's also possible to coordinate transactions across Actors or threads so that either the transactions in a set all commit successfully or they all fail. This is the focus of Transactors and the explicit support for coordinated transactions in this section.
|
||||
|
||||
|
||||
Coordinated transactions
|
||||
------------------------
|
||||
|
||||
Akka provides an explicit mechanism for coordinating transactions across Actors. Under the hood it uses a ``CountDownCommitBarrier``, similar to a CountDownLatch.
|
||||
|
||||
Here is an example of coordinating two simple counter Actors so that they both increment together in coordinated transactions. If one of them was to fail to increment, the other would also fail.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.transactor.Coordinated
|
||||
import akka.stm.Ref
|
||||
import akka.actor.{Actor, ActorRef}
|
||||
|
||||
case class Increment(friend: Option[ActorRef] = None)
|
||||
case object GetCount
|
||||
|
||||
class Counter extends Actor {
|
||||
val count = Ref(0)
|
||||
|
||||
def receive = {
|
||||
case coordinated @ Coordinated(Increment(friend)) => {
|
||||
friend foreach (_ ! coordinated(Increment()))
|
||||
coordinated atomic {
|
||||
count alter (_ + 1)
|
||||
}
|
||||
}
|
||||
case GetCount => self.reply(count.get)
|
||||
}
|
||||
}
|
||||
|
||||
val counter1 = Actor.actorOf[Counter]
|
||||
val counter2 = Actor.actorOf[Counter]
|
||||
|
||||
counter1 ! Coordinated(Increment(Some(counter2)))
|
||||
|
||||
...
|
||||
|
||||
(counter1 ? GetCount).as[Int] // Some(1)
|
||||
|
||||
counter1.stop()
|
||||
counter2.stop()
|
||||
|
||||
To start a new coordinated transaction that you will also participate in, just create a ``Coordinated`` object:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val coordinated = Coordinated()
|
||||
|
||||
To start a coordinated transaction that you won't participate in yourself you can create a ``Coordinated`` object with a message and send it directly to an actor. The recipient of the message will be the first member of the coordination set:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
actor ! Coordinated(Message)
|
||||
|
||||
To receive a coordinated message in an actor simply match it in a case statement:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
def receive = {
|
||||
case coordinated @ Coordinated(Message) => ...
|
||||
}
|
||||
|
||||
To include another actor in the same coordinated transaction that you've created or received, use the apply method on that object. This will increment the number of parties involved by one and create a new ``Coordinated`` object to be sent.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
actor ! coordinated(Message)
|
||||
|
||||
To enter the coordinated transaction use the atomic method of the coordinated object:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
coordinated atomic {
|
||||
// do something in transaction ...
|
||||
}
|
||||
|
||||
The coordinated transaction will wait for the other transactions before committing. If any of the coordinated transactions fail then they all fail.
|
||||
|
||||
|
||||
Transactor
|
||||
----------
|
||||
|
||||
Transactors are actors that provide a general pattern for coordinating transactions, using the explicit coordination described above.
|
||||
|
||||
Here's an example of a simple transactor that will join a coordinated transaction:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.transactor.Transactor
|
||||
import akka.stm.Ref
|
||||
|
||||
case object Increment
|
||||
|
||||
class Counter extends Transactor {
|
||||
val count = Ref(0)
|
||||
|
||||
override def atomically = {
|
||||
case Increment => count alter (_ + 1)
|
||||
}
|
||||
}
|
||||
|
||||
You could send this Counter transactor a ``Coordinated(Increment)`` message. If you were to send it just an ``Increment`` message it will create its own ``Coordinated`` (but in this particular case wouldn't be coordinating transactions with any other transactors).
|
||||
|
||||
To coordinate with other transactors override the ``coordinate`` method. The ``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of ``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods to easily coordinate with other transactors. The ``include`` method will send on the same message that was received to other transactors. The ``sendTo`` method allows you to specify both the actor to send to, and the message to send.
|
||||
|
||||
Example of coordinating an increment:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.transactor.Transactor
|
||||
import akka.stm.Ref
|
||||
import akka.actor.ActorRef
|
||||
|
||||
case object Increment
|
||||
|
||||
class FriendlyCounter(friend: ActorRef) extends Transactor {
|
||||
val count = Ref(0)
|
||||
|
||||
override def coordinate = {
|
||||
case Increment => include(friend)
|
||||
}
|
||||
|
||||
override def atomically = {
|
||||
case Increment => count alter (_ + 1)
|
||||
}
|
||||
}
|
||||
|
||||
Using ``include`` to include more than one transactor:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
override def coordinate = {
|
||||
case Message => include(actor1, actor2, actor3)
|
||||
}
|
||||
|
||||
Using ``sendTo`` to coordinate transactions but pass-on a different message than the one that was received:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
override def coordinate = {
|
||||
case Message => sendTo(someActor -> SomeOtherMessage)
|
||||
case SomeMessage => sendTo(actor1 -> Message1, actor2 -> Message2)
|
||||
}
|
||||
|
||||
To execute directly before or after the coordinated transaction, override the ``before`` and ``after`` methods. These methods also expect partial functions like the receive method. They do not execute within the transaction.
|
||||
|
||||
To completely bypass coordinated transactions override the ``normally`` method. Any message matched by ``normally`` will not be matched by the other methods, and will not be involved in coordinated transactions. In this method you can implement normal actor behavior, or use the normal STM atomic for local transactions.
|
||||
|
||||
|
||||
Coordinating Typed Actors
|
||||
-------------------------
|
||||
|
||||
It's also possible to use coordinated transactions with typed actors. You can explicitly pass around ``Coordinated`` objects, or use built-in support with the ``@Coordinated`` annotation and the ``Coordination.coordinate`` method.
|
||||
|
||||
To specify a method should use coordinated transactions add the ``@Coordinated`` annotation. **Note**: the ``@Coordinated`` annotation only works with methods that return Unit (one-way methods).
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
trait Counter {
|
||||
@Coordinated def increment()
|
||||
def get: Int
|
||||
}
|
||||
|
||||
To coordinate transactions use a ``coordinate`` block:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
coordinate {
|
||||
counter1.increment()
|
||||
counter2.increment()
|
||||
}
|
||||
|
||||
Here's an example of using ``@Coordinated`` with a TypedActor to coordinate increments.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.actor.TypedActor
|
||||
import akka.stm.Ref
|
||||
import akka.transactor.annotation.Coordinated
|
||||
import akka.transactor.Coordination._
|
||||
|
||||
trait Counter {
|
||||
@Coordinated def increment()
|
||||
def get: Int
|
||||
}
|
||||
|
||||
class CounterImpl extends TypedActor with Counter {
|
||||
val ref = Ref(0)
|
||||
def increment() { ref alter (_ + 1) }
|
||||
def get = ref.get
|
||||
}
|
||||
|
||||
...
|
||||
|
||||
val counter1 = TypedActor.newInstance(classOf[Counter], classOf[CounterImpl])
|
||||
val counter2 = TypedActor.newInstance(classOf[Counter], classOf[CounterImpl])
|
||||
|
||||
coordinate {
|
||||
counter1.increment()
|
||||
counter2.increment()
|
||||
}
|
||||
|
||||
TypedActor.stop(counter1)
|
||||
TypedActor.stop(counter2)
|
||||
|
||||
The ``coordinate`` block will wait for the transactions to complete. If you do not want to wait then you can specify this explicitly:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
coordinate(wait = false) {
|
||||
counter1.increment()
|
||||
counter2.increment()
|
||||
}
|
||||
|
||||
|
|
@ -96,8 +96,7 @@ public class UntypedActorDocTestBase {
|
|||
public void propsActorOf() {
|
||||
ActorSystem system = ActorSystem.create("MySystem");
|
||||
//#creating-props
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory().lookup("my-dispatcher");
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher),
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher("my-dispatcher"),
|
||||
"myactor");
|
||||
//#creating-props
|
||||
myActor.tell("test");
|
||||
|
|
|
|||
|
|
@ -23,7 +23,6 @@ import akka.japi.Function;
|
|||
//#import-function
|
||||
|
||||
//#import-timeout
|
||||
import akka.util.Duration;
|
||||
import akka.util.Timeout;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
//#import-timeout
|
||||
|
|
@ -86,7 +85,7 @@ public class AgentDocTest {
|
|||
//#send-off
|
||||
|
||||
//#read-await
|
||||
Integer result = agent.await(new Timeout(Duration.create(5, SECONDS)));
|
||||
Integer result = agent.await(new Timeout(5, SECONDS));
|
||||
//#read-await
|
||||
|
||||
assertEquals(result, new Integer(14));
|
||||
|
|
|
|||
|
|
@ -14,13 +14,18 @@ import akka.dispatch.MessageDispatcher;
|
|||
import akka.actor.UntypedActor;
|
||||
import akka.actor.UntypedActorFactory;
|
||||
import akka.actor.Actors;
|
||||
import akka.dispatch.PriorityGenerator;
|
||||
import akka.dispatch.UnboundedPriorityMailbox;
|
||||
import akka.event.Logging;
|
||||
import akka.event.LoggingAdapter;
|
||||
|
||||
//#imports-prio
|
||||
|
||||
//#imports-prio-mailbox
|
||||
import akka.dispatch.PriorityGenerator;
|
||||
import akka.dispatch.UnboundedPriorityMailbox;
|
||||
import com.typesafe.config.Config;
|
||||
|
||||
//#imports-prio-mailbox
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
|
@ -52,10 +57,9 @@ public class DispatcherDocTestBase {
|
|||
@Test
|
||||
public void defineDispatcher() {
|
||||
//#defining-dispatcher
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory().lookup("my-dispatcher");
|
||||
ActorRef myActor1 = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher),
|
||||
ActorRef myActor1 = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher("my-dispatcher"),
|
||||
"myactor1");
|
||||
ActorRef myActor2 = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher),
|
||||
ActorRef myActor2 = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher("my-dispatcher"),
|
||||
"myactor2");
|
||||
//#defining-dispatcher
|
||||
}
|
||||
|
|
@ -64,31 +68,14 @@ public class DispatcherDocTestBase {
|
|||
public void definePinnedDispatcher() {
|
||||
//#defining-pinned-dispatcher
|
||||
String name = "myactor";
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory().newPinnedDispatcher(name);
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class).withDispatcher(dispatcher), name);
|
||||
ActorRef myActor = system.actorOf(new Props().withCreator(MyUntypedActor.class)
|
||||
.withDispatcher("myactor-dispatcher"), name);
|
||||
//#defining-pinned-dispatcher
|
||||
}
|
||||
|
||||
@Test
|
||||
public void priorityDispatcher() throws Exception {
|
||||
//#prio-dispatcher
|
||||
PriorityGenerator generator = new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important
|
||||
@Override
|
||||
public int gen(Object message) {
|
||||
if (message.equals("highpriority"))
|
||||
return 0; // 'highpriority messages should be treated first if possible
|
||||
else if (message.equals("lowpriority"))
|
||||
return 100; // 'lowpriority messages should be treated last if possible
|
||||
else if (message.equals(Actors.poisonPill()))
|
||||
return 1000; // PoisonPill when no other left
|
||||
else
|
||||
return 50; // We default to 50
|
||||
}
|
||||
};
|
||||
|
||||
// We create a new Priority dispatcher and seed it with the priority generator
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory()
|
||||
.newDispatcher("foo", 5, new UnboundedPriorityMailbox(generator)).build();
|
||||
|
||||
ActorRef myActor = system.actorOf( // We create a new Actor that just prints out what it processes
|
||||
new Props().withCreator(new UntypedActorFactory() {
|
||||
|
|
@ -111,7 +98,7 @@ public class DispatcherDocTestBase {
|
|||
}
|
||||
};
|
||||
}
|
||||
}).withDispatcher(dispatcher));
|
||||
}).withDispatcher("prio-dispatcher-java"));
|
||||
|
||||
/*
|
||||
Logs:
|
||||
|
|
@ -131,4 +118,27 @@ public class DispatcherDocTestBase {
|
|||
Thread.sleep(100);
|
||||
}
|
||||
}
|
||||
|
||||
//#prio-mailbox
|
||||
public static class PrioMailbox extends UnboundedPriorityMailbox {
|
||||
|
||||
static final PriorityGenerator generator = new PriorityGenerator() { // Create a new PriorityGenerator, lower prio means more important
|
||||
@Override
|
||||
public int gen(Object message) {
|
||||
if (message.equals("highpriority"))
|
||||
return 0; // 'highpriority messages should be treated first if possible
|
||||
else if (message.equals("lowpriority"))
|
||||
return 100; // 'lowpriority messages should be treated last if possible
|
||||
else if (message.equals(Actors.poisonPill()))
|
||||
return 1000; // PoisonPill when no other left
|
||||
else
|
||||
return 50; // We default to 50
|
||||
}
|
||||
};
|
||||
|
||||
public PrioMailbox(Config config) {
|
||||
super(generator);
|
||||
}
|
||||
}
|
||||
//#prio-mailbox
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,40 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
//#class
|
||||
import akka.actor.*;
|
||||
import akka.transactor.*;
|
||||
import scala.concurrent.stm.*;
|
||||
|
||||
public class CoordinatedCounter extends UntypedActor {
|
||||
private Ref<Integer> count = Stm.ref(0);
|
||||
|
||||
private void increment(InTxn txn) {
|
||||
Integer newValue = count.get(txn) + 1;
|
||||
count.set(newValue, txn);
|
||||
}
|
||||
|
||||
public void onReceive(Object incoming) throws Exception {
|
||||
if (incoming instanceof Coordinated) {
|
||||
Coordinated coordinated = (Coordinated) incoming;
|
||||
Object message = coordinated.getMessage();
|
||||
if (message instanceof Increment) {
|
||||
Increment increment = (Increment) message;
|
||||
if (increment.hasFriend()) {
|
||||
increment.getFriend().tell(coordinated.coordinate(new Increment()));
|
||||
}
|
||||
coordinated.atomic(new Atomically() {
|
||||
public void atomically(InTxn txn) {
|
||||
increment(txn);
|
||||
}
|
||||
});
|
||||
}
|
||||
} else if ("GetCount".equals(incoming)) {
|
||||
getSender().tell(count.single().get());
|
||||
}
|
||||
}
|
||||
}
|
||||
//#class
|
||||
27
akka-docs/java/code/akka/docs/transactor/Coordinator.java
Normal file
27
akka-docs/java/code/akka/docs/transactor/Coordinator.java
Normal file
|
|
@ -0,0 +1,27 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
import akka.actor.*;
|
||||
import akka.transactor.*;
|
||||
import scala.concurrent.stm.*;
|
||||
|
||||
public class Coordinator extends UntypedActor {
|
||||
public void onReceive(Object incoming) throws Exception {
|
||||
if (incoming instanceof Coordinated) {
|
||||
Coordinated coordinated = (Coordinated) incoming;
|
||||
Object message = coordinated.getMessage();
|
||||
if (message instanceof Message) {
|
||||
//#coordinated-atomic
|
||||
coordinated.atomic(new Atomically() {
|
||||
public void atomically(InTxn txn) {
|
||||
// do something in the coordinated transaction ...
|
||||
}
|
||||
});
|
||||
//#coordinated-atomic
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
28
akka-docs/java/code/akka/docs/transactor/Counter.java
Normal file
28
akka-docs/java/code/akka/docs/transactor/Counter.java
Normal file
|
|
@ -0,0 +1,28 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
//#class
|
||||
import akka.transactor.*;
|
||||
import scala.concurrent.stm.*;
|
||||
|
||||
public class Counter extends UntypedTransactor {
|
||||
Ref<Integer> count = Stm.ref(0);
|
||||
|
||||
public void atomically(InTxn txn, Object message) {
|
||||
if (message instanceof Increment) {
|
||||
Integer newValue = count.get(txn) + 1;
|
||||
count.set(newValue, txn);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public boolean normally(Object message) {
|
||||
if ("GetCount".equals(message)) {
|
||||
getSender().tell(count.single().get());
|
||||
return true;
|
||||
} else return false;
|
||||
}
|
||||
}
|
||||
//#class
|
||||
|
|
@ -1,13 +1,17 @@
|
|||
package akka.transactor.example;
|
||||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
import akka.transactor.UntypedTransactor;
|
||||
import akka.transactor.SendTo;
|
||||
import akka.stm.Ref;
|
||||
package akka.docs.transactor;
|
||||
|
||||
//#class
|
||||
import akka.actor.*;
|
||||
import akka.transactor.*;
|
||||
import java.util.Set;
|
||||
import scala.concurrent.stm.*;
|
||||
|
||||
public class UntypedCounter extends UntypedTransactor {
|
||||
Ref<Integer> count = new Ref<Integer>(0);
|
||||
public class FriendlyCounter extends UntypedTransactor {
|
||||
Ref<Integer> count = Stm.ref(0);
|
||||
|
||||
@Override public Set<SendTo> coordinate(Object message) {
|
||||
if (message instanceof Increment) {
|
||||
|
|
@ -18,16 +22,18 @@ public class UntypedCounter extends UntypedTransactor {
|
|||
return nobody();
|
||||
}
|
||||
|
||||
public void atomically(Object message) {
|
||||
public void atomically(InTxn txn, Object message) {
|
||||
if (message instanceof Increment) {
|
||||
count.set(count.get() + 1);
|
||||
Integer newValue = count.get(txn) + 1;
|
||||
count.set(newValue, txn);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public boolean normally(Object message) {
|
||||
if ("GetCount".equals(message)) {
|
||||
getSender().tell(count.get());
|
||||
getSender().tell(count.single().get());
|
||||
return true;
|
||||
} else return false;
|
||||
}
|
||||
}
|
||||
//#class
|
||||
|
|
@ -1,5 +1,10 @@
|
|||
package akka.transactor.example;
|
||||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
//#class
|
||||
import akka.actor.ActorRef;
|
||||
|
||||
public class Increment {
|
||||
|
|
@ -19,3 +24,4 @@ public class Increment {
|
|||
return friend;
|
||||
}
|
||||
}
|
||||
//#class
|
||||
7
akka-docs/java/code/akka/docs/transactor/Message.java
Normal file
7
akka-docs/java/code/akka/docs/transactor/Message.java
Normal file
|
|
@ -0,0 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
public class Message {}
|
||||
|
|
@ -0,0 +1,11 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor
|
||||
|
||||
import org.scalatest.junit.JUnitWrapperSuite
|
||||
|
||||
class TransactorDocJavaSpec extends JUnitWrapperSuite(
|
||||
"akka.docs.transactor.TransactorDocTest",
|
||||
Thread.currentThread.getContextClassLoader)
|
||||
|
|
@ -0,0 +1,99 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import org.junit.Test;
|
||||
|
||||
//#imports
|
||||
import akka.actor.*;
|
||||
import akka.dispatch.Await;
|
||||
import akka.transactor.Coordinated;
|
||||
import akka.util.Duration;
|
||||
import akka.util.Timeout;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
//#imports
|
||||
|
||||
public class TransactorDocTest {
|
||||
|
||||
@Test
|
||||
public void coordinatedExample() {
|
||||
//#coordinated-example
|
||||
ActorSystem system = ActorSystem.create("CoordinatedExample");
|
||||
|
||||
ActorRef counter1 = system.actorOf(new Props(CoordinatedCounter.class));
|
||||
ActorRef counter2 = system.actorOf(new Props(CoordinatedCounter.class));
|
||||
|
||||
Timeout timeout = new Timeout(5, SECONDS);
|
||||
|
||||
counter1.tell(new Coordinated(new Increment(counter2), timeout));
|
||||
|
||||
Integer count = (Integer) Await.result(counter1.ask("GetCount", timeout), timeout.duration());
|
||||
//#coordinated-example
|
||||
|
||||
assertEquals(count, new Integer(1));
|
||||
|
||||
system.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void coordinatedApi() {
|
||||
//#create-coordinated
|
||||
Timeout timeout = new Timeout(5, SECONDS);
|
||||
Coordinated coordinated = new Coordinated(timeout);
|
||||
//#create-coordinated
|
||||
|
||||
ActorSystem system = ActorSystem.create("CoordinatedApi");
|
||||
ActorRef actor = system.actorOf(new Props(Coordinator.class));
|
||||
|
||||
//#send-coordinated
|
||||
actor.tell(new Coordinated(new Message(), timeout));
|
||||
//#send-coordinated
|
||||
|
||||
//#include-coordinated
|
||||
actor.tell(coordinated.coordinate(new Message()));
|
||||
//#include-coordinated
|
||||
|
||||
coordinated.await();
|
||||
|
||||
system.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void counterTransactor() {
|
||||
ActorSystem system = ActorSystem.create("CounterTransactor");
|
||||
ActorRef counter = system.actorOf(new Props(Counter.class));
|
||||
|
||||
Timeout timeout = new Timeout(5, SECONDS);
|
||||
Coordinated coordinated = new Coordinated(timeout);
|
||||
counter.tell(coordinated.coordinate(new Increment()));
|
||||
coordinated.await();
|
||||
|
||||
Integer count = (Integer) Await.result(counter.ask("GetCount", timeout), timeout.duration());
|
||||
assertEquals(count, new Integer(1));
|
||||
|
||||
system.shutdown();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void friendlyCounterTransactor() {
|
||||
ActorSystem system = ActorSystem.create("FriendlyCounterTransactor");
|
||||
ActorRef friend = system.actorOf(new Props(Counter.class));
|
||||
ActorRef friendlyCounter = system.actorOf(new Props(FriendlyCounter.class));
|
||||
|
||||
Timeout timeout = new Timeout(5, SECONDS);
|
||||
Coordinated coordinated = new Coordinated(timeout);
|
||||
friendlyCounter.tell(coordinated.coordinate(new Increment(friend)));
|
||||
coordinated.await();
|
||||
|
||||
Integer count1 = (Integer) Await.result(friendlyCounter.ask("GetCount", timeout), timeout.duration());
|
||||
assertEquals(count1, new Integer(1));
|
||||
|
||||
Integer count2 = (Integer) Await.result(friend.ask("GetCount", timeout), timeout.duration());
|
||||
assertEquals(count2, new Integer(1));
|
||||
|
||||
system.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
@ -27,7 +27,8 @@ See below for details on which ones are available and how they can be configured
|
|||
Setting the dispatcher
|
||||
----------------------
|
||||
|
||||
You specify the dispatcher to use when creating an actor.
|
||||
You specify the id of the dispatcher to use when creating an actor. The id corresponds to the :ref:`configuration` key
|
||||
of the dispatcher settings.
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java
|
||||
:include: imports,defining-dispatcher
|
||||
|
|
@ -44,18 +45,15 @@ There are 4 different types of message dispatchers:
|
|||
|
||||
It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments.
|
||||
|
||||
Example of a custom event-based dispatcher, which can be fetched with ``system.dispatcherFactory().lookup("my-dispatcher")``
|
||||
Example of a custom event-based dispatcher, which can be used with
|
||||
``new Props().withCreator(MyUntypedActor.class).withDispatcher("my-dispatcher")``
|
||||
as in the example above:
|
||||
|
||||
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config
|
||||
|
||||
Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined.
|
||||
|
||||
.. warning::
|
||||
|
||||
Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e.
|
||||
``dispatcherFactory`` of the ``ActorSystem``. These methods will probably be changed or removed before
|
||||
2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup.
|
||||
Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined. See
|
||||
:ref:`configuration` for the default values of the ``default-dispatcher``. You can also override
|
||||
the values for the ``default-dispatcher`` in your configuration.
|
||||
|
||||
Let's now walk through the different dispatchers in more detail.
|
||||
|
||||
|
|
@ -118,7 +116,19 @@ Sometimes it's useful to be able to specify priority order of messages, that is
|
|||
an UnboundedPriorityMailbox or BoundedPriorityMailbox with a ``java.util.Comparator[Envelope]`` or use a
|
||||
``akka.dispatch.PriorityGenerator`` (recommended).
|
||||
|
||||
Creating a Dispatcher using PriorityGenerator:
|
||||
Creating a Dispatcher with a mailbox using PriorityGenerator:
|
||||
|
||||
Config:
|
||||
|
||||
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala
|
||||
:include: prio-dispatcher-config-java
|
||||
|
||||
Priority mailbox:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java
|
||||
:include: imports-prio-mailbox,prio-mailbox
|
||||
|
||||
Usage:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocTestBase.java
|
||||
:include: imports-prio,prio-dispatcher
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@ Java API
|
|||
routing
|
||||
remoting
|
||||
serialization
|
||||
stm
|
||||
agents
|
||||
extending-akka
|
||||
transactors
|
||||
extending-akka
|
||||
|
|
|
|||
|
|
@ -1,4 +1,3 @@
|
|||
|
||||
.. _remoting-java:
|
||||
|
||||
#####################
|
||||
|
|
@ -25,14 +24,16 @@ First of all you have to change the actor provider from ``LocalActorRefProvider`
|
|||
After that you must also add the following settings::
|
||||
|
||||
akka {
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
remote {
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA)
|
||||
port = 2552
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA)
|
||||
port = 2552
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -42,8 +43,19 @@ reference file for more information:
|
|||
|
||||
* `reference.conf of akka-remote <https://github.com/jboner/akka/blob/master/akka-remote/src/main/resources/reference.conf#L39>`_
|
||||
|
||||
Using Remote Actors
|
||||
^^^^^^^^^^^^^^^^^^^
|
||||
Looking up Remote Actors
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
``actorFor(path)`` will obtain an ``ActorRef`` to an Actor on a remote node::
|
||||
|
||||
ActorRef actor = context.actorFor("akka://app@10.0.0.1:2552/user/serviceA/retrieval");
|
||||
|
||||
As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node::
|
||||
|
||||
akka://<actorsystemname>@<hostname>:<port>/<actor path>
|
||||
|
||||
Creating Actors Remotely
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The configuration below instructs the system to deploy the actor "retrieval” on the specific host "app@10.0.0.1".
|
||||
The "app" in this case refers to the name of the ``ActorSystem``::
|
||||
|
|
|
|||
60
akka-docs/java/stm.rst
Normal file
60
akka-docs/java/stm.rst
Normal file
|
|
@ -0,0 +1,60 @@
|
|||
|
||||
.. _stm-java:
|
||||
|
||||
#####################################
|
||||
Software Transactional Memory (Java)
|
||||
#####################################
|
||||
|
||||
|
||||
Overview of STM
|
||||
===============
|
||||
|
||||
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the
|
||||
Java heap into a transactional data set with begin/commit/rollback
|
||||
semantics. Very much like a regular database. It implements the first three
|
||||
letters in `ACID`_; ACI:
|
||||
|
||||
* Atomic
|
||||
* Consistent
|
||||
* Isolated
|
||||
|
||||
.. _ACID: http://en.wikipedia.org/wiki/ACID
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some
|
||||
use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating
|
||||
their **internal local** state but need them to do that atomically in one big
|
||||
transaction. Might not be often, but when you do need this then you are
|
||||
screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
|
||||
The use of STM in Akka is inspired by the concepts and views in `Clojure`_\'s
|
||||
STM. Please take the time to read `this excellent document`_ about state in
|
||||
clojure and view `this presentation`_ by Rich Hickey (the genius behind
|
||||
Clojure).
|
||||
|
||||
.. _Clojure: http://clojure.org/
|
||||
.. _this excellent document: http://clojure.org/state
|
||||
.. _this presentation: http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey
|
||||
|
||||
|
||||
Scala STM
|
||||
=========
|
||||
|
||||
The STM supported in Akka is `ScalaSTM`_ which will be soon included in the
|
||||
Scala standard library.
|
||||
|
||||
.. _ScalaSTM: http://nbronson.github.com/scala-stm/
|
||||
|
||||
The STM is based on Transactional References (referred to as Refs). Refs are
|
||||
memory cells, holding an (arbitrary) immutable value, that implement CAS
|
||||
(Compare-And-Swap) semantics and are managed and enforced by the STM for
|
||||
coordinated changes across many Refs.
|
||||
|
||||
|
||||
Integration with Actors
|
||||
=======================
|
||||
|
||||
In Akka we've also integrated Actors and STM in :ref:`agents-java` and
|
||||
:ref:`transactors-java`.
|
||||
|
|
@ -1,6 +1,157 @@
|
|||
.. _transactors-java:
|
||||
|
||||
Transactors (Java)
|
||||
==================
|
||||
####################
|
||||
Transactors (Java)
|
||||
####################
|
||||
|
||||
The Akka Transactors module has not been migrated to Akka 2.0-SNAPSHOT yet.
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
|
||||
Why Transactors?
|
||||
================
|
||||
|
||||
Actors are excellent for solving problems where you have many independent
|
||||
processes that can work in isolation and only interact with other Actors through
|
||||
message passing. This model fits many problems. But the actor model is
|
||||
unfortunately a terrible model for implementing truly shared state. E.g. when
|
||||
you need to have consensus and a stable view of state across many
|
||||
components. The classic example is the bank account where clients can deposit
|
||||
and withdraw, in which each operation needs to be atomic. For detailed
|
||||
discussion on the topic see `this JavaOne presentation
|
||||
<http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
|
||||
|
||||
STM on the other hand is excellent for problems where you need consensus and a
|
||||
stable view of the state by providing compositional transactional shared
|
||||
state. Some of the really nice traits of STM are that transactions compose, and
|
||||
it raises the abstraction level from lock-based concurrency.
|
||||
|
||||
Akka's Transactors combine Actors and STM to provide the best of the Actor model
|
||||
(concurrency and asynchronous event-based programming) and STM (compositional
|
||||
transactional shared state) by providing transactional, compositional,
|
||||
asynchronous, event-based message flows.
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some
|
||||
use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating
|
||||
their **internal local** state but need them to do that atomically in one big
|
||||
transaction. Might not be often but when you do need this then you are
|
||||
screwed without it.
|
||||
|
||||
- When you want to share a datastructure across actors.
|
||||
|
||||
|
||||
Actors and STM
|
||||
==============
|
||||
|
||||
You can combine Actors and STM in several ways. An Actor may use STM internally
|
||||
so that particular changes are guaranteed to be atomic. Actors may also share
|
||||
transactional datastructures as the STM provides safe shared state across
|
||||
threads.
|
||||
|
||||
It's also possible to coordinate transactions across Actors or threads so that
|
||||
either the transactions in a set all commit successfully or they all fail. This
|
||||
is the focus of Transactors and the explicit support for coordinated
|
||||
transactions in this section.
|
||||
|
||||
|
||||
Coordinated transactions
|
||||
========================
|
||||
|
||||
Akka provides an explicit mechanism for coordinating transactions across
|
||||
actors. Under the hood it uses a ``CommitBarrier``, similar to a CountDownLatch.
|
||||
|
||||
Here is an example of coordinating two simple counter UntypedActors so that they
|
||||
both increment together in coordinated transactions. If one of them was to fail
|
||||
to increment, the other would also fail.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/Increment.java#class
|
||||
:language: java
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/CoordinatedCounter.java#class
|
||||
:language: java
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#imports
|
||||
:language: java
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#coordinated-example
|
||||
:language: java
|
||||
|
||||
To start a new coordinated transaction that you will also participate in, create
|
||||
a ``Coordinated`` object, passing in a ``Timeout``:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#create-coordinated
|
||||
:language: java
|
||||
|
||||
To start a coordinated transaction that you won't participate in yourself you
|
||||
can create a ``Coordinated`` object with a message and send it directly to an
|
||||
actor. The recipient of the message will be the first member of the coordination
|
||||
set:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#send-coordinated
|
||||
:language: java
|
||||
|
||||
To include another actor in the same coordinated transaction that you've created
|
||||
or received, use the ``coordinate`` method on that object. This will increment
|
||||
the number of parties involved by one and create a new ``Coordinated`` object to
|
||||
be sent.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocTest.java#include-coordinated
|
||||
:language: java
|
||||
|
||||
To enter the coordinated transaction use the atomic method of the coordinated
|
||||
object, passing in an ``akka.transactor.Atomically`` object.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/Coordinator.java#coordinated-atomic
|
||||
:language: java
|
||||
|
||||
The coordinated transaction will wait for the other transactions before
|
||||
committing. If any of the coordinated transactions fail then they all fail.
|
||||
|
||||
.. note::
|
||||
|
||||
The same actor should not be added to a coordinated transaction more than
|
||||
once. The transaction will not be able to complete as an actor only processes
|
||||
a single message at a time. When processing the first message the coordinated
|
||||
transaction will wait for the commit barrier, which in turn needs the second
|
||||
message to be received to proceed.
|
||||
|
||||
|
||||
UntypedTransactor
|
||||
=================
|
||||
|
||||
UntypedTransactors are untyped actors that provide a general pattern for
|
||||
coordinating transactions, using the explicit coordination described above.
|
||||
|
||||
Here's an example of a simple untyped transactor that will join a coordinated
|
||||
transaction:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/Counter.java#class
|
||||
:language: java
|
||||
|
||||
You could send this Counter transactor a ``Coordinated(Increment)`` message. If
|
||||
you were to send it just an ``Increment`` message it will create its own
|
||||
``Coordinated`` (but in this particular case wouldn't be coordinating
|
||||
transactions with any other transactors).
|
||||
|
||||
To coordinate with other transactors override the ``coordinate`` method. The
|
||||
``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of
|
||||
``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods
|
||||
to easily coordinate with other transactors.
|
||||
|
||||
Here's an example of coordinating an increment, using an untyped transactor,
|
||||
similar to the explicitly coordinated example above.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/FriendlyCounter.java#class
|
||||
:language: java
|
||||
|
||||
To execute directly before or after the coordinated transaction, override the
|
||||
``before`` and ``after`` methods. They do not execute within the transaction.
|
||||
|
||||
To completely bypass coordinated transactions override the ``normally``
|
||||
method. Any message matched by ``normally`` will not be matched by the other
|
||||
methods, and will not be involved in coordinated transactions. In this method
|
||||
you can implement normal actor behavior, or use the normal STM atomic for local
|
||||
transactions.
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ object DurableMailboxDocSpec {
|
|||
val config = """
|
||||
//#dispatcher-config
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.FileBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.FileBasedMailboxType
|
||||
}
|
||||
//#dispatcher-config
|
||||
"""
|
||||
|
|
@ -33,8 +33,7 @@ class DurableMailboxDocSpec extends AkkaSpec(DurableMailboxDocSpec.config) {
|
|||
|
||||
"configuration of dispatcher with durable mailbox" in {
|
||||
//#dispatcher-config-use
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher")
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor")
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor")
|
||||
//#dispatcher-config-use
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.docs.actor.mailbox;
|
||||
|
||||
//#imports
|
||||
import akka.dispatch.MessageDispatcher;
|
||||
import akka.actor.UntypedActorFactory;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.Props;
|
||||
|
|
@ -40,12 +39,12 @@ public class DurableMailboxDocTestBase {
|
|||
@Test
|
||||
public void configDefinedDispatcher() {
|
||||
//#dispatcher-config-use
|
||||
MessageDispatcher dispatcher = system.dispatcherFactory().lookup("my-dispatcher");
|
||||
ActorRef myActor = system.actorOf(new Props().withDispatcher(dispatcher).withCreator(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new MyUntypedActor();
|
||||
}
|
||||
}), "myactor");
|
||||
ActorRef myActor = system.actorOf(
|
||||
new Props().withDispatcher("my-dispatcher").withCreator(new UntypedActorFactory() {
|
||||
public UntypedActor create() {
|
||||
return new MyUntypedActor();
|
||||
}
|
||||
}), "myactor");
|
||||
//#dispatcher-config-use
|
||||
myActor.tell("test");
|
||||
}
|
||||
|
|
|
|||
|
|
@ -99,7 +99,7 @@ You configure durable mailboxes through the dispatcher, as described in
|
|||
Config::
|
||||
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.FileBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.FileBasedMailboxType
|
||||
}
|
||||
|
||||
You can also configure and tune the file-based durable mailbox. This is done in
|
||||
|
|
@ -124,7 +124,7 @@ You configure durable mailboxes through the dispatcher, as described in
|
|||
Config::
|
||||
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.RedisBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.RedisBasedMailboxType
|
||||
}
|
||||
|
||||
You also need to configure the IP and port for the Redis server. This is done in
|
||||
|
|
@ -150,7 +150,7 @@ You configure durable mailboxes through the dispatcher, as described in
|
|||
Config::
|
||||
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.ZooKeeperBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.ZooKeeperBasedMailboxType
|
||||
}
|
||||
|
||||
You also need to configure ZooKeeper server addresses, timeouts, etc. This is
|
||||
|
|
@ -173,7 +173,7 @@ You configure durable mailboxes through the dispatcher, as described in
|
|||
Config::
|
||||
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.BeanstalkBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.BeanstalkBasedMailboxType
|
||||
}
|
||||
|
||||
You also need to configure the IP, and port, and so on, for the Beanstalk
|
||||
|
|
@ -202,7 +202,7 @@ You configure durable mailboxes through the dispatcher, as described in
|
|||
Config::
|
||||
|
||||
my-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.MongoBasedMailbox
|
||||
mailboxType = akka.actor.mailbox.MongoBasedMailboxType
|
||||
}
|
||||
|
||||
You will need to configure the URI for the MongoDB server, using the URI Format specified in the
|
||||
|
|
|
|||
|
|
@ -194,7 +194,6 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
}
|
||||
|
||||
"creating a Props config" in {
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher")
|
||||
//#creating-props-config
|
||||
import akka.actor.Props
|
||||
val props1 = Props()
|
||||
|
|
@ -202,10 +201,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
val props3 = Props(new MyActor)
|
||||
val props4 = Props(
|
||||
creator = { () ⇒ new MyActor },
|
||||
dispatcher = dispatcher,
|
||||
dispatcher = "my-dispatcher",
|
||||
timeout = Timeout(100))
|
||||
val props5 = props1.withCreator(new MyActor)
|
||||
val props6 = props5.withDispatcher(dispatcher)
|
||||
val props6 = props5.withDispatcher("my-dispatcher")
|
||||
val props7 = props6.withTimeout(Timeout(100))
|
||||
//#creating-props-config
|
||||
}
|
||||
|
|
@ -213,8 +212,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
|
|||
"creating actor with Props" in {
|
||||
//#creating-props
|
||||
import akka.actor.Props
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher")
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor")
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor")
|
||||
//#creating-props
|
||||
|
||||
system.stop(myActor)
|
||||
|
|
|
|||
|
|
@ -6,14 +6,15 @@ package akka.docs.dispatcher
|
|||
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
|
||||
import org.scalatest.matchers.MustMatchers
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.dispatch.PriorityGenerator
|
||||
import akka.actor.Props
|
||||
import akka.actor.Actor
|
||||
import akka.dispatch.UnboundedPriorityMailbox
|
||||
import akka.event.Logging
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.util.duration._
|
||||
import akka.actor.PoisonPill
|
||||
import akka.dispatch.MessageDispatcherConfigurator
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.DispatcherPrerequisites
|
||||
|
||||
object DispatcherDocSpec {
|
||||
val config = """
|
||||
|
|
@ -53,8 +54,36 @@ object DispatcherDocSpec {
|
|||
type = BalancingDispatcher
|
||||
}
|
||||
//#my-balancing-config
|
||||
|
||||
//#prio-dispatcher-config
|
||||
prio-dispatcher {
|
||||
mailboxType = "akka.docs.dispatcher.DispatcherDocSpec$PrioMailbox"
|
||||
}
|
||||
//#prio-dispatcher-config
|
||||
|
||||
//#prio-dispatcher-config-java
|
||||
prio-dispatcher-java {
|
||||
mailboxType = "akka.docs.dispatcher.DispatcherDocTestBase$PrioMailbox"
|
||||
}
|
||||
//#prio-dispatcher-config-java
|
||||
"""
|
||||
|
||||
//#prio-mailbox
|
||||
import akka.dispatch.PriorityGenerator
|
||||
import akka.dispatch.UnboundedPriorityMailbox
|
||||
import com.typesafe.config.Config
|
||||
|
||||
val generator = PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important
|
||||
case 'highpriority ⇒ 0 // 'highpriority messages should be treated first if possible
|
||||
case 'lowpriority ⇒ 100 // 'lowpriority messages should be treated last if possible
|
||||
case PoisonPill ⇒ 1000 // PoisonPill when no other left
|
||||
case otherwise ⇒ 50 // We default to 50
|
||||
}
|
||||
|
||||
// We create a new Priority dispatcher and seed it with the priority generator
|
||||
class PrioMailbox(config: Config) extends UnboundedPriorityMailbox(generator)
|
||||
//#prio-mailbox
|
||||
|
||||
class MyActor extends Actor {
|
||||
def receive = {
|
||||
case x ⇒
|
||||
|
|
@ -69,35 +98,23 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
|||
"defining dispatcher" in {
|
||||
//#defining-dispatcher
|
||||
import akka.actor.Props
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher")
|
||||
val myActor1 = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor1")
|
||||
val myActor2 = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name = "myactor2")
|
||||
val myActor1 = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor1")
|
||||
val myActor2 = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor2")
|
||||
//#defining-dispatcher
|
||||
}
|
||||
|
||||
"defining dispatcher with bounded queue" in {
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher-bounded-queue")
|
||||
val dispatcher = system.dispatchers.lookup("my-dispatcher-bounded-queue")
|
||||
}
|
||||
|
||||
"defining pinned dispatcher" in {
|
||||
//#defining-pinned-dispatcher
|
||||
val name = "myactor"
|
||||
val dispatcher = system.dispatcherFactory.newPinnedDispatcher(name)
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher(dispatcher), name)
|
||||
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), name = "myactor")
|
||||
//#defining-pinned-dispatcher
|
||||
}
|
||||
|
||||
"defining priority dispatcher" in {
|
||||
//#prio-dispatcher
|
||||
val gen = PriorityGenerator { // Create a new PriorityGenerator, lower prio means more important
|
||||
case 'highpriority ⇒ 0 // 'highpriority messages should be treated first if possible
|
||||
case 'lowpriority ⇒ 100 // 'lowpriority messages should be treated last if possible
|
||||
case PoisonPill ⇒ 1000 // PoisonPill when no other left
|
||||
case otherwise ⇒ 50 // We default to 50
|
||||
}
|
||||
|
||||
// We create a new Priority dispatcher and seed it with the priority generator
|
||||
val dispatcher = system.dispatcherFactory.newDispatcher("foo", 5, UnboundedPriorityMailbox(gen)).build
|
||||
|
||||
val a = system.actorOf( // We create a new Actor that just prints out what it processes
|
||||
Props(new Actor {
|
||||
|
|
@ -115,7 +132,7 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
|||
def receive = {
|
||||
case x ⇒ log.info(x.toString)
|
||||
}
|
||||
}).withDispatcher(dispatcher))
|
||||
}).withDispatcher("prio-dispatcher"))
|
||||
|
||||
/*
|
||||
Logs:
|
||||
|
|
@ -133,7 +150,7 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
|||
}
|
||||
|
||||
"defining balancing dispatcher" in {
|
||||
val dispatcher = system.dispatcherFactory.lookup("my-balancing-dispatcher")
|
||||
val dispatcher = system.dispatchers.lookup("my-balancing-dispatcher")
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -68,7 +68,7 @@ class ParentActor extends Actor {
|
|||
case "sgfcr" ⇒
|
||||
//#scatterGatherFirstCompletedRouter
|
||||
val scatterGatherFirstCompletedRouter = context.actorOf(
|
||||
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter()),
|
||||
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(within = 2 seconds)),
|
||||
"router")
|
||||
implicit val timeout = context.system.settings.ActorTimeout
|
||||
val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10)
|
||||
|
|
|
|||
|
|
@ -181,8 +181,8 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
val actor = system.actorOf(Props[MyDoubleEcho])
|
||||
actor ! (probe1.ref, probe2.ref)
|
||||
actor ! "hello"
|
||||
probe1.expectMsg(50 millis, "hello")
|
||||
probe2.expectMsg(50 millis, "hello")
|
||||
probe1.expectMsg(500 millis, "hello")
|
||||
probe2.expectMsg(500 millis, "hello")
|
||||
//#test-probe
|
||||
|
||||
//#test-special-probe
|
||||
|
|
@ -227,8 +227,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
"demonstrate " in {
|
||||
//#calling-thread-dispatcher
|
||||
import akka.testkit.CallingThreadDispatcher
|
||||
val dispatcher = new CallingThreadDispatcher(system.dispatcherFactory.prerequisites)
|
||||
val ref = system.actorOf(Props[MyActor].withDispatcher(dispatcher))
|
||||
val ref = system.actorOf(Props[MyActor].withDispatcher(CallingThreadDispatcher.Id))
|
||||
//#calling-thread-dispatcher
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,230 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.docs.transactor
|
||||
|
||||
import akka.actor._
|
||||
import akka.transactor._
|
||||
import akka.util.duration._
|
||||
import akka.util.Timeout
|
||||
import akka.testkit._
|
||||
import scala.concurrent.stm._
|
||||
|
||||
object CoordinatedExample {
|
||||
//#coordinated-example
|
||||
import akka.actor._
|
||||
import akka.transactor._
|
||||
import scala.concurrent.stm._
|
||||
|
||||
case class Increment(friend: Option[ActorRef] = None)
|
||||
case object GetCount
|
||||
|
||||
class Counter extends Actor {
|
||||
val count = Ref(0)
|
||||
|
||||
def receive = {
|
||||
case coordinated @ Coordinated(Increment(friend)) ⇒ {
|
||||
friend foreach (_ ! coordinated(Increment()))
|
||||
coordinated atomic { implicit t ⇒
|
||||
count transform (_ + 1)
|
||||
}
|
||||
}
|
||||
case GetCount ⇒ sender ! count.single.get
|
||||
}
|
||||
}
|
||||
//#coordinated-example
|
||||
}
|
||||
|
||||
object CoordinatedApi {
|
||||
case object Message
|
||||
|
||||
class Coordinator extends Actor {
|
||||
//#receive-coordinated
|
||||
def receive = {
|
||||
case coordinated @ Coordinated(Message) ⇒ {
|
||||
//#coordinated-atomic
|
||||
coordinated atomic { implicit t ⇒
|
||||
// do something in the coordinated transaction ...
|
||||
}
|
||||
//#coordinated-atomic
|
||||
}
|
||||
}
|
||||
//#receive-coordinated
|
||||
}
|
||||
}
|
||||
|
||||
object CounterExample {
|
||||
//#counter-example
|
||||
import akka.transactor._
|
||||
import scala.concurrent.stm._
|
||||
|
||||
case object Increment
|
||||
|
||||
class Counter extends Transactor {
|
||||
val count = Ref(0)
|
||||
|
||||
def atomically = implicit txn ⇒ {
|
||||
case Increment ⇒ count transform (_ + 1)
|
||||
}
|
||||
}
|
||||
//#counter-example
|
||||
}
|
||||
|
||||
object FriendlyCounterExample {
|
||||
//#friendly-counter-example
|
||||
import akka.actor._
|
||||
import akka.transactor._
|
||||
import scala.concurrent.stm._
|
||||
|
||||
case object Increment
|
||||
|
||||
class FriendlyCounter(friend: ActorRef) extends Transactor {
|
||||
val count = Ref(0)
|
||||
|
||||
override def coordinate = {
|
||||
case Increment ⇒ include(friend)
|
||||
}
|
||||
|
||||
def atomically = implicit txn ⇒ {
|
||||
case Increment ⇒ count transform (_ + 1)
|
||||
}
|
||||
}
|
||||
//#friendly-counter-example
|
||||
|
||||
class Friend extends Transactor {
|
||||
val count = Ref(0)
|
||||
|
||||
def atomically = implicit txn ⇒ {
|
||||
case Increment ⇒ count transform (_ + 1)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Only checked for compilation
|
||||
object TransactorCoordinate {
|
||||
case object Message
|
||||
case object SomeMessage
|
||||
case object SomeOtherMessage
|
||||
case object OtherMessage
|
||||
case object Message1
|
||||
case object Message2
|
||||
|
||||
class TestCoordinateInclude(actor1: ActorRef, actor2: ActorRef, actor3: ActorRef) extends Transactor {
|
||||
//#coordinate-include
|
||||
override def coordinate = {
|
||||
case Message ⇒ include(actor1, actor2, actor3)
|
||||
}
|
||||
//#coordinate-include
|
||||
|
||||
def atomically = txn ⇒ doNothing
|
||||
}
|
||||
|
||||
class TestCoordinateSendTo(someActor: ActorRef, actor1: ActorRef, actor2: ActorRef) extends Transactor {
|
||||
//#coordinate-sendto
|
||||
override def coordinate = {
|
||||
case SomeMessage ⇒ sendTo(someActor -> SomeOtherMessage)
|
||||
case OtherMessage ⇒ sendTo(actor1 -> Message1, actor2 -> Message2)
|
||||
}
|
||||
//#coordinate-sendto
|
||||
|
||||
def atomically = txn ⇒ doNothing
|
||||
}
|
||||
}
|
||||
|
||||
class TransactorDocSpec extends AkkaSpec {
|
||||
|
||||
"coordinated example" in {
|
||||
import CoordinatedExample._
|
||||
|
||||
//#run-coordinated-example
|
||||
import akka.dispatch.Await
|
||||
import akka.util.duration._
|
||||
import akka.util.Timeout
|
||||
|
||||
val system = ActorSystem("app")
|
||||
|
||||
val counter1 = system.actorOf(Props[Counter], name = "counter1")
|
||||
val counter2 = system.actorOf(Props[Counter], name = "counter2")
|
||||
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
|
||||
counter1 ! Coordinated(Increment(Some(counter2)))
|
||||
|
||||
val count = Await.result(counter1 ? GetCount, timeout.duration)
|
||||
|
||||
// count == 1
|
||||
//#run-coordinated-example
|
||||
|
||||
count must be === 1
|
||||
|
||||
system.shutdown()
|
||||
}
|
||||
|
||||
"coordinated api" in {
|
||||
import CoordinatedApi._
|
||||
|
||||
//#implicit-timeout
|
||||
import akka.util.duration._
|
||||
import akka.util.Timeout
|
||||
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
//#implicit-timeout
|
||||
|
||||
//#create-coordinated
|
||||
val coordinated = Coordinated()
|
||||
//#create-coordinated
|
||||
|
||||
val system = ActorSystem("coordinated")
|
||||
val actor = system.actorOf(Props[Coordinator], name = "coordinator")
|
||||
|
||||
//#send-coordinated
|
||||
actor ! Coordinated(Message)
|
||||
//#send-coordinated
|
||||
|
||||
//#include-coordinated
|
||||
actor ! coordinated(Message)
|
||||
//#include-coordinated
|
||||
|
||||
coordinated.await()
|
||||
|
||||
system.shutdown()
|
||||
}
|
||||
|
||||
"counter transactor" in {
|
||||
import CounterExample._
|
||||
|
||||
val system = ActorSystem("transactors")
|
||||
|
||||
lazy val underlyingCounter = new Counter
|
||||
val counter = system.actorOf(Props(underlyingCounter), name = "counter")
|
||||
val coordinated = Coordinated()(Timeout(5 seconds))
|
||||
counter ! coordinated(Increment)
|
||||
coordinated.await()
|
||||
|
||||
underlyingCounter.count.single.get must be === 1
|
||||
|
||||
system.shutdown()
|
||||
}
|
||||
|
||||
"friendly counter transactor" in {
|
||||
import FriendlyCounterExample._
|
||||
|
||||
val system = ActorSystem("transactors")
|
||||
|
||||
lazy val underlyingFriend = new Friend
|
||||
val friend = system.actorOf(Props(underlyingFriend), name = "friend")
|
||||
|
||||
lazy val underlyingFriendlyCounter = new FriendlyCounter(friend)
|
||||
val friendlyCounter = system.actorOf(Props(underlyingFriendlyCounter), name = "friendly")
|
||||
|
||||
val coordinated = Coordinated()(Timeout(5 seconds))
|
||||
friendlyCounter ! coordinated(Increment)
|
||||
coordinated.await()
|
||||
|
||||
underlyingFriendlyCounter.count.single.get must be === 1
|
||||
underlyingFriend.count.single.get must be === 1
|
||||
|
||||
system.shutdown()
|
||||
}
|
||||
}
|
||||
|
|
@ -27,7 +27,8 @@ See below for details on which ones are available and how they can be configured
|
|||
Setting the dispatcher
|
||||
----------------------
|
||||
|
||||
You specify the dispatcher to use when creating an actor.
|
||||
You specify the id of the dispatcher to use when creating an actor. The id corresponds to the :ref:`configuration` key
|
||||
of the dispatcher settings.
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala
|
||||
:include: imports,defining-dispatcher
|
||||
|
|
@ -44,18 +45,14 @@ There are 4 different types of message dispatchers:
|
|||
|
||||
It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments.
|
||||
|
||||
Example of a custom event-based dispatcher, which can be fetched with ``system.dispatcherFactory.lookup("my-dispatcher")``
|
||||
Example of a custom event-based dispatcher, which can be used with ``Props[MyActor].withDispatcher("my-dispatcher")``
|
||||
as in the example above:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config
|
||||
|
||||
Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined.
|
||||
|
||||
.. warning::
|
||||
|
||||
Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e.
|
||||
``dispatcherFactory`` of the ``ActorSystem``. These methods will probably be changed or removed before
|
||||
2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup.
|
||||
Default values are taken from ``default-dispatcher``, i.e. all options doesn't need to be defined. See
|
||||
:ref:`configuration` for the default values of the ``default-dispatcher``. You can also override
|
||||
the values for the ``default-dispatcher`` in your configuration.
|
||||
|
||||
Let's now walk through the different dispatchers in more detail.
|
||||
|
||||
|
|
@ -118,9 +115,22 @@ Sometimes it's useful to be able to specify priority order of messages, that is
|
|||
an UnboundedPriorityMailbox or BoundedPriorityMailbox with a ``java.util.Comparator[Envelope]`` or use a
|
||||
``akka.dispatch.PriorityGenerator`` (recommended).
|
||||
|
||||
Creating a Dispatcher using PriorityGenerator:
|
||||
Creating a Dispatcher with a mailbox using PriorityGenerator:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#prio-dispatcher
|
||||
Config:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala
|
||||
:include: prio-dispatcher-config
|
||||
|
||||
Priority mailbox:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala
|
||||
:include: prio-mailbox
|
||||
|
||||
Usage:
|
||||
|
||||
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala
|
||||
:include: prio-dispatcher
|
||||
|
||||
Work-sharing event-based
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
|
|
|||
|
|
@ -18,7 +18,8 @@ Scala API
|
|||
remoting
|
||||
serialization
|
||||
fsm
|
||||
stm
|
||||
agents
|
||||
transactors
|
||||
testing
|
||||
extending-akka
|
||||
transactors
|
||||
|
|
|
|||
|
|
@ -1,4 +1,3 @@
|
|||
|
||||
.. _remoting-scala:
|
||||
|
||||
#################
|
||||
|
|
@ -29,14 +28,16 @@ First of all you have to change the actor provider from ``LocalActorRefProvider`
|
|||
After that you must also add the following settings::
|
||||
|
||||
akka {
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
remote {
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
hostname = ""
|
||||
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA)
|
||||
port = 2552
|
||||
# The default remote server port clients should connect to.
|
||||
# Default is 2552 (AKKA)
|
||||
port = 2552
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -46,8 +47,19 @@ reference file for more information:
|
|||
|
||||
* `reference.conf of akka-remote <https://github.com/jboner/akka/blob/master/akka-remote/src/main/resources/reference.conf#L39>`_
|
||||
|
||||
Using Remote Actors
|
||||
^^^^^^^^^^^^^^^^^^^
|
||||
Looking up Remote Actors
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
``actorFor(path)`` will obtain an ``ActorRef`` to an Actor on a remote node::
|
||||
|
||||
val actor = context.actorFor("akka://app@10.0.0.1:2552/user/serviceA/retrieval")
|
||||
|
||||
As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node::
|
||||
|
||||
akka://<actorsystemname>@<hostname>:<port>/<actor path>
|
||||
|
||||
Creating Actors Remotely
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The configuration below instructs the system to deploy the actor "retrieval” on the specific host "app@10.0.0.1".
|
||||
The "app" in this case refers to the name of the ``ActorSystem``::
|
||||
|
|
@ -67,14 +79,6 @@ actor created above you would do the following::
|
|||
|
||||
val actor = context.actorFor("/serviceA/retrieval")
|
||||
|
||||
This will obtain an ``ActorRef`` on a remote node::
|
||||
|
||||
val actor = context.actorFor("akka://app@10.0.0.1:2552/user/serviceA/retrieval")
|
||||
|
||||
As you can see from the example above the following pattern is used to find an ``ActorRef`` on a remote node::
|
||||
|
||||
akka://<actorsystemname>@<hostname>:<port>/<actor path>
|
||||
|
||||
Serialization
|
||||
^^^^^^^^^^^^^
|
||||
|
||||
|
|
|
|||
75
akka-docs/scala/stm.rst
Normal file
75
akka-docs/scala/stm.rst
Normal file
|
|
@ -0,0 +1,75 @@
|
|||
|
||||
.. _stm-scala:
|
||||
|
||||
#######################################
|
||||
Software Transactional Memory (Scala)
|
||||
#######################################
|
||||
|
||||
|
||||
Overview of STM
|
||||
===============
|
||||
|
||||
An `STM <http://en.wikipedia.org/wiki/Software_transactional_memory>`_ turns the
|
||||
Java heap into a transactional data set with begin/commit/rollback
|
||||
semantics. Very much like a regular database. It implements the first three
|
||||
letters in `ACID`_; ACI:
|
||||
|
||||
* Atomic
|
||||
* Consistent
|
||||
* Isolated
|
||||
|
||||
.. _ACID: http://en.wikipedia.org/wiki/ACID
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some
|
||||
use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating
|
||||
their **internal local** state but need them to do that atomically in one big
|
||||
transaction. Might not be often, but when you do need this then you are
|
||||
screwed without it.
|
||||
- When you want to share a datastructure across actors.
|
||||
|
||||
The use of STM in Akka is inspired by the concepts and views in `Clojure`_\'s
|
||||
STM. Please take the time to read `this excellent document`_ about state in
|
||||
clojure and view `this presentation`_ by Rich Hickey (the genius behind
|
||||
Clojure).
|
||||
|
||||
.. _Clojure: http://clojure.org/
|
||||
.. _this excellent document: http://clojure.org/state
|
||||
.. _this presentation: http://www.infoq.com/presentations/Value-Identity-State-Rich-Hickey
|
||||
|
||||
|
||||
Scala STM
|
||||
=========
|
||||
|
||||
The STM supported in Akka is `ScalaSTM`_ which will be soon included in the
|
||||
Scala standard library.
|
||||
|
||||
.. _ScalaSTM: http://nbronson.github.com/scala-stm/
|
||||
|
||||
The STM is based on Transactional References (referred to as Refs). Refs are
|
||||
memory cells, holding an (arbitrary) immutable value, that implement CAS
|
||||
(Compare-And-Swap) semantics and are managed and enforced by the STM for
|
||||
coordinated changes across many Refs.
|
||||
|
||||
|
||||
Persistent Datastructures
|
||||
=========================
|
||||
|
||||
Working with immutable collections can sometimes give bad performance due to
|
||||
extensive copying. Scala provides so-called persistent datastructures which
|
||||
makes working with immutable collections fast. They are immutable but with
|
||||
constant time access and modification. They use structural sharing and an insert
|
||||
or update does not ruin the old structure, hence "persistent". Makes working
|
||||
with immutable composite types fast. The persistent datastructures currently
|
||||
consist of a `Map`_ and `Vector`_.
|
||||
|
||||
.. _Map: http://www.scala-lang.org/api/current/index.html#scala.collection.immutable.Map
|
||||
.. _Vector: http://www.scala-lang.org/api/current/index.html#scala.collection.immutable.Vector
|
||||
|
||||
|
||||
Integration with Actors
|
||||
=======================
|
||||
|
||||
In Akka we've also integrated Actors and STM in :ref:`agents-scala` and
|
||||
:ref:`transactors-scala`.
|
||||
|
|
@ -1,6 +1,167 @@
|
|||
.. _transactors-scala:
|
||||
|
||||
Transactors (Scala)
|
||||
===================
|
||||
#####################
|
||||
Transactors (Scala)
|
||||
#####################
|
||||
|
||||
The Akka Transactors module has not been migrated to Akka 2.0-SNAPSHOT yet.
|
||||
.. sidebar:: Contents
|
||||
|
||||
.. contents:: :local:
|
||||
|
||||
|
||||
Why Transactors?
|
||||
================
|
||||
|
||||
Actors are excellent for solving problems where you have many independent
|
||||
processes that can work in isolation and only interact with other Actors through
|
||||
message passing. This model fits many problems. But the actor model is
|
||||
unfortunately a terrible model for implementing truly shared state. E.g. when
|
||||
you need to have consensus and a stable view of state across many
|
||||
components. The classic example is the bank account where clients can deposit
|
||||
and withdraw, in which each operation needs to be atomic. For detailed
|
||||
discussion on the topic see `this JavaOne presentation
|
||||
<http://www.slideshare.net/jboner/state-youre-doing-it-wrong-javaone-2009>`_.
|
||||
|
||||
STM on the other hand is excellent for problems where you need consensus and a
|
||||
stable view of the state by providing compositional transactional shared
|
||||
state. Some of the really nice traits of STM are that transactions compose, and
|
||||
it raises the abstraction level from lock-based concurrency.
|
||||
|
||||
Akka's Transactors combine Actors and STM to provide the best of the Actor model
|
||||
(concurrency and asynchronous event-based programming) and STM (compositional
|
||||
transactional shared state) by providing transactional, compositional,
|
||||
asynchronous, event-based message flows.
|
||||
|
||||
Generally, the STM is not needed very often when working with Akka. Some
|
||||
use-cases (that we can think of) are:
|
||||
|
||||
- When you really need composable message flows across many actors updating
|
||||
their **internal local** state but need them to do that atomically in one big
|
||||
transaction. Might not be often but when you do need this then you are
|
||||
screwed without it.
|
||||
|
||||
- When you want to share a datastructure across actors.
|
||||
|
||||
|
||||
Actors and STM
|
||||
==============
|
||||
|
||||
You can combine Actors and STM in several ways. An Actor may use STM internally
|
||||
so that particular changes are guaranteed to be atomic. Actors may also share
|
||||
transactional datastructures as the STM provides safe shared state across
|
||||
threads.
|
||||
|
||||
It's also possible to coordinate transactions across Actors or threads so that
|
||||
either the transactions in a set all commit successfully or they all fail. This
|
||||
is the focus of Transactors and the explicit support for coordinated
|
||||
transactions in this section.
|
||||
|
||||
|
||||
Coordinated transactions
|
||||
========================
|
||||
|
||||
Akka provides an explicit mechanism for coordinating transactions across
|
||||
Actors. Under the hood it uses a ``CommitBarrier``, similar to a CountDownLatch.
|
||||
|
||||
Here is an example of coordinating two simple counter Actors so that they both
|
||||
increment together in coordinated transactions. If one of them was to fail to
|
||||
increment, the other would also fail.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinated-example
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#run-coordinated-example
|
||||
|
||||
Note that creating a ``Coordinated`` object requires a ``Timeout`` to be
|
||||
specified for the coordinated transaction. This can be done implicitly, by
|
||||
having an implicit ``Timeout`` in scope, or explicitly, by passing the timeout
|
||||
when creating a a ``Coordinated`` object. Here's an example of specifying an
|
||||
implicit timeout:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#implicit-timeout
|
||||
|
||||
To start a new coordinated transaction that you will also participate in, just
|
||||
create a ``Coordinated`` object (this assumes an implicit timeout):
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#create-coordinated
|
||||
|
||||
To start a coordinated transaction that you won't participate in yourself you
|
||||
can create a ``Coordinated`` object with a message and send it directly to an
|
||||
actor. The recipient of the message will be the first member of the coordination
|
||||
set:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#send-coordinated
|
||||
|
||||
To receive a coordinated message in an actor simply match it in a case
|
||||
statement:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#receive-coordinated
|
||||
:exclude: coordinated-atomic
|
||||
|
||||
To include another actor in the same coordinated transaction that you've created
|
||||
or received, use the apply method on that object. This will increment the number
|
||||
of parties involved by one and create a new ``Coordinated`` object to be sent.
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#include-coordinated
|
||||
|
||||
To enter the coordinated transaction use the atomic method of the coordinated
|
||||
object:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinated-atomic
|
||||
|
||||
The coordinated transaction will wait for the other transactions before
|
||||
committing. If any of the coordinated transactions fail then they all fail.
|
||||
|
||||
.. note::
|
||||
|
||||
The same actor should not be added to a coordinated transaction more than
|
||||
once. The transaction will not be able to complete as an actor only processes
|
||||
a single message at a time. When processing the first message the coordinated
|
||||
transaction will wait for the commit barrier, which in turn needs the second
|
||||
message to be received to proceed.
|
||||
|
||||
|
||||
Transactor
|
||||
==========
|
||||
|
||||
Transactors are actors that provide a general pattern for coordinating
|
||||
transactions, using the explicit coordination described above.
|
||||
|
||||
Here's an example of a simple transactor that will join a coordinated
|
||||
transaction:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#counter-example
|
||||
|
||||
You could send this Counter transactor a ``Coordinated(Increment)`` message. If
|
||||
you were to send it just an ``Increment`` message it will create its own
|
||||
``Coordinated`` (but in this particular case wouldn't be coordinating
|
||||
transactions with any other transactors).
|
||||
|
||||
To coordinate with other transactors override the ``coordinate`` method. The
|
||||
``coordinate`` method maps a message to a set of ``SendTo`` objects, pairs of
|
||||
``ActorRef`` and a message. You can use the ``include`` and ``sendTo`` methods
|
||||
to easily coordinate with other transactors. The ``include`` method will send on
|
||||
the same message that was received to other transactors. The ``sendTo`` method
|
||||
allows you to specify both the actor to send to, and the message to send.
|
||||
|
||||
Example of coordinating an increment:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#friendly-counter-example
|
||||
|
||||
Using ``include`` to include more than one transactor:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinate-include
|
||||
|
||||
Using ``sendTo`` to coordinate transactions but pass-on a different message than
|
||||
the one that was received:
|
||||
|
||||
.. includecode:: code/akka/docs/transactor/TransactorDocSpec.scala#coordinate-sendto
|
||||
|
||||
To execute directly before or after the coordinated transaction, override the
|
||||
``before`` and ``after`` methods. These methods also expect partial functions
|
||||
like the receive method. They do not execute within the transaction.
|
||||
|
||||
To completely bypass coordinated transactions override the ``normally``
|
||||
method. Any message matched by ``normally`` will not be matched by the other
|
||||
methods, and will not be involved in coordinated transactions. In this method
|
||||
you can implement normal actor behavior, or use the normal STM atomic for local
|
||||
transactions.
|
||||
|
|
|
|||
|
|
@ -13,9 +13,15 @@ import akka.actor.ActorContext
|
|||
import akka.dispatch.Envelope
|
||||
import akka.event.Logging
|
||||
import akka.actor.ActorRef
|
||||
import akka.dispatch.MailboxType
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class BeanstalkBasedMailboxException(message: String) extends AkkaException(message) {}
|
||||
|
||||
class BeanstalkBasedMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new BeanstalkBasedMailbox(owner)
|
||||
}
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -1,7 +1,13 @@
|
|||
package akka.actor.mailbox
|
||||
|
||||
import akka.dispatch.CustomMailboxType
|
||||
object BeanstalkBasedMailboxSpec {
|
||||
val config = """
|
||||
Beanstalkd-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.BeanstalkBasedMailboxType
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class BeanstalkBasedMailboxSpec extends DurableMailboxSpec("Beanstalkd",
|
||||
new CustomMailboxType("akka.actor.mailbox.BeanstalkBasedMailbox"))
|
||||
class BeanstalkBasedMailboxSpec extends DurableMailboxSpec("Beanstalkd", BeanstalkBasedMailboxSpec.config)
|
||||
|
|
|
|||
|
|
@ -9,6 +9,12 @@ import akka.actor.ActorContext
|
|||
import akka.dispatch.Envelope
|
||||
import akka.event.Logging
|
||||
import akka.actor.ActorRef
|
||||
import akka.dispatch.MailboxType
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class FileBasedMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new FileBasedMailbox(owner)
|
||||
}
|
||||
|
||||
class FileBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
|
|
|
|||
|
|
@ -1,11 +1,18 @@
|
|||
package akka.actor.mailbox
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import akka.dispatch.CustomMailboxType
|
||||
|
||||
object FileBasedMailboxSpec {
|
||||
val config = """
|
||||
File-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.FileBasedMailboxType
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class FileBasedMailboxSpec extends DurableMailboxSpec("File",
|
||||
new CustomMailboxType("akka.actor.mailbox.FileBasedMailbox")) {
|
||||
class FileBasedMailboxSpec extends DurableMailboxSpec("File", FileBasedMailboxSpec.config) {
|
||||
|
||||
def clean {
|
||||
val queuePath = FileBasedMailboxExtension(system).QueuePath
|
||||
|
|
|
|||
|
|
@ -23,7 +23,6 @@ import akka.remote.RemoteActorRefProvider
|
|||
import akka.remote.netty.NettyRemoteServer
|
||||
import akka.serialization.Serialization
|
||||
import com.typesafe.config.Config
|
||||
import akka.dispatch.CustomMailboxType
|
||||
|
||||
private[akka] object DurableExecutableMailboxConfig {
|
||||
val Name = "[\\.\\/\\$\\s]".r
|
||||
|
|
|
|||
|
|
@ -24,13 +24,15 @@ object DurableMailboxSpecActorFactory {
|
|||
|
||||
}
|
||||
|
||||
abstract class DurableMailboxSpec(val backendName: String, val mailboxType: MailboxType) extends AkkaSpec with BeforeAndAfterEach {
|
||||
/**
|
||||
* Subclass must define dispatcher in the supplied config for the specific backend.
|
||||
* The id of the dispatcher must be the same as the `<backendName>-dispatcher`.
|
||||
*/
|
||||
abstract class DurableMailboxSpec(val backendName: String, config: String) extends AkkaSpec(config) with BeforeAndAfterEach {
|
||||
import DurableMailboxSpecActorFactory._
|
||||
|
||||
implicit val dispatcher = system.dispatcherFactory.newDispatcher(backendName, throughput = 1, mailboxType = mailboxType).build
|
||||
|
||||
def createMailboxTestActor(id: String)(implicit dispatcher: MessageDispatcher): ActorRef =
|
||||
system.actorOf(Props(new MailboxTestActor).withDispatcher(dispatcher))
|
||||
def createMailboxTestActor(id: String): ActorRef =
|
||||
system.actorOf(Props(new MailboxTestActor).withDispatcher(backendName + "-dispatcher"))
|
||||
|
||||
"A " + backendName + " based mailbox backed actor" must {
|
||||
|
||||
|
|
|
|||
|
|
@ -12,9 +12,15 @@ import akka.event.Logging
|
|||
import akka.actor.ActorRef
|
||||
import akka.dispatch.{ Await, Promise, Envelope, DefaultPromise }
|
||||
import java.util.concurrent.TimeoutException
|
||||
import akka.dispatch.MailboxType
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class MongoBasedMailboxException(message: String) extends AkkaException(message)
|
||||
|
||||
class MongoBasedMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new MongoBasedMailbox(owner)
|
||||
}
|
||||
|
||||
/**
|
||||
* A "naive" durable mailbox which uses findAndRemove; it's possible if the actor crashes
|
||||
* after consuming a message that the message could be lost.
|
||||
|
|
|
|||
|
|
@ -8,11 +8,18 @@ import akka.actor._
|
|||
import akka.actor.Actor._
|
||||
import java.util.concurrent.CountDownLatch
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.CustomMailboxType
|
||||
|
||||
object MongoBasedMailboxSpec {
|
||||
val config = """
|
||||
mongodb-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.MongoBasedMailboxType
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb",
|
||||
new CustomMailboxType("akka.actor.mailbox.MongoBasedMailbox")) {
|
||||
class MongoBasedMailboxSpec extends DurableMailboxSpec("mongodb", MongoBasedMailboxSpec.config) {
|
||||
|
||||
import org.apache.log4j.{ Logger, Level }
|
||||
import com.mongodb.async._
|
||||
|
|
|
|||
|
|
@ -10,9 +10,15 @@ import akka.actor.ActorContext
|
|||
import akka.dispatch.Envelope
|
||||
import akka.event.Logging
|
||||
import akka.actor.ActorRef
|
||||
import akka.dispatch.MailboxType
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class RedisBasedMailboxException(message: String) extends AkkaException(message)
|
||||
|
||||
class RedisBasedMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new RedisBasedMailbox(owner)
|
||||
}
|
||||
|
||||
class RedisBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
private val settings = RedisBasedMailboxExtension(owner.system)
|
||||
|
|
|
|||
|
|
@ -1,6 +1,13 @@
|
|||
package akka.actor.mailbox
|
||||
import akka.dispatch.CustomMailboxType
|
||||
|
||||
object RedisBasedMailboxSpec {
|
||||
val config = """
|
||||
Redis-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.RedisBasedMailboxType
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RedisBasedMailboxSpec extends DurableMailboxSpec("Redis",
|
||||
new CustomMailboxType("akka.actor.mailbox.RedisBasedMailbox"))
|
||||
class RedisBasedMailboxSpec extends DurableMailboxSpec("Redis", RedisBasedMailboxSpec.config)
|
||||
|
|
|
|||
|
|
@ -14,9 +14,15 @@ import akka.dispatch.Envelope
|
|||
import akka.event.Logging
|
||||
import akka.cluster.zookeeper.ZooKeeperQueue
|
||||
import akka.actor.ActorRef
|
||||
import akka.dispatch.MailboxType
|
||||
import com.typesafe.config.Config
|
||||
|
||||
class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message)
|
||||
|
||||
class ZooKeeperBasedMailboxType(config: Config) extends MailboxType {
|
||||
override def create(owner: ActorContext) = new ZooKeeperBasedMailbox(owner)
|
||||
}
|
||||
|
||||
class ZooKeeperBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) with DurableMessageSerialization {
|
||||
|
||||
private val settings = ZooKeeperBasedMailboxExtension(owner.system)
|
||||
|
|
|
|||
|
|
@ -4,12 +4,19 @@ import akka.actor.{ Actor, LocalActorRef }
|
|||
import akka.cluster.zookeeper._
|
||||
import org.I0Itec.zkclient._
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.dispatch.CustomMailboxType
|
||||
import akka.actor.ActorRef
|
||||
|
||||
object ZooKeeperBasedMailboxSpec {
|
||||
val config = """
|
||||
ZooKeeper-dispatcher {
|
||||
mailboxType = akka.actor.mailbox.ZooKeeperBasedMailboxType
|
||||
throughput = 1
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ZooKeeperBasedMailboxSpec extends DurableMailboxSpec("ZooKeeper",
|
||||
new CustomMailboxType("akka.actor.mailbox.ZooKeeperBasedMailbox")) {
|
||||
class ZooKeeperBasedMailboxSpec extends DurableMailboxSpec("ZooKeeper", ZooKeeperBasedMailboxSpec.config) {
|
||||
|
||||
val dataPath = "_akka_cluster/data"
|
||||
val logPath = "_akka_cluster/log"
|
||||
|
|
|
|||
|
|
@ -75,6 +75,11 @@ akka {
|
|||
name = ComputeGridDispatcher
|
||||
}
|
||||
|
||||
# The dispatcher used for the system actor "network-event-sender"
|
||||
network-event-sender-dispatcher {
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
|
||||
server {
|
||||
# The hostname or ip to bind the remoting to,
|
||||
# InetAddress.getLocalHost.getHostAddress is used if empty
|
||||
|
|
|
|||
|
|
@ -62,8 +62,7 @@ class NetworkEventStream(system: ActorSystemImpl) {
|
|||
|
||||
// FIXME: check that this supervision is correct, ticket #1408
|
||||
private[akka] val sender =
|
||||
system.systemActorOf(Props[Channel].copy(dispatcher = system.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")),
|
||||
"network-event-sender")
|
||||
system.systemActorOf(Props[Channel].withDispatcher("akka.remote.network-event-sender-dispatcher"), "network-event-sender")
|
||||
|
||||
/**
|
||||
* Registers a network event stream listener (asyncronously).
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ class Remote(val settings: ActorSystem.Settings, val remoteSettings: RemoteSetti
|
|||
|
||||
_provider = provider
|
||||
_serialization = SerializationExtension(system)
|
||||
_computeGridDispatcher = system.dispatcherFactory.newFromConfig("akka.remote.compute-grid-dispatcher")
|
||||
_computeGridDispatcher = system.dispatchers.lookup("akka.remote.compute-grid-dispatcher")
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log)
|
||||
_eventStream = new NetworkEventStream(system)
|
||||
_server = {
|
||||
|
|
|
|||
|
|
@ -26,10 +26,10 @@ class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings
|
|||
if (nodes.isEmpty || deploy.routing == NoRouter) d
|
||||
else {
|
||||
val r = deploy.routing match {
|
||||
case RoundRobinRouter(x, _) ⇒ RemoteRoundRobinRouter(x, nodes)
|
||||
case RandomRouter(x, _) ⇒ RemoteRandomRouter(x, nodes)
|
||||
case BroadcastRouter(x, _) ⇒ RemoteBroadcastRouter(x, nodes)
|
||||
case ScatterGatherFirstCompletedRouter(x, _) ⇒ RemoteScatterGatherFirstCompletedRouter(x, nodes)
|
||||
case RoundRobinRouter(x, _) ⇒ RemoteRoundRobinRouter(x, nodes)
|
||||
case RandomRouter(x, _) ⇒ RemoteRandomRouter(x, nodes)
|
||||
case BroadcastRouter(x, _) ⇒ RemoteBroadcastRouter(x, nodes)
|
||||
case ScatterGatherFirstCompletedRouter(x, _, w) ⇒ RemoteScatterGatherFirstCompletedRouter(x, nodes, w)
|
||||
}
|
||||
Some(deploy.copy(routing = r))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -155,8 +155,8 @@ case class RemoteClientError[T <: ParsedTransportAddress](
|
|||
override def toString =
|
||||
"RemoteClientError@" +
|
||||
remoteAddress +
|
||||
": ErrorMessage[" +
|
||||
(if (cause ne null) cause.getMessage else "no message") +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
|
|
@ -203,8 +203,8 @@ case class RemoteClientWriteFailed[T <: ParsedTransportAddress](
|
|||
remoteAddress +
|
||||
": MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] ErrorMessage[" +
|
||||
(if (cause ne null) cause.getMessage else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
|
|
@ -234,8 +234,8 @@ case class RemoteServerError[T <: ParsedTransportAddress](
|
|||
override def toString =
|
||||
"RemoteServerError@" +
|
||||
remote.name +
|
||||
": ErrorMessage[" +
|
||||
(if (cause ne null) cause.getMessage else "no message") +
|
||||
": Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
|
|
@ -288,8 +288,8 @@ case class RemoteServerWriteFailed[T <: ParsedTransportAddress](
|
|||
remoteAddress +
|
||||
"] MessageClass[" +
|
||||
(if (request ne null) request.getClass.getName else "no message") +
|
||||
"] ErrorMessage[" +
|
||||
(if (cause ne null) cause.getMessage else "no message") +
|
||||
"] Error[" +
|
||||
(if (cause ne null) cause.getClass.getName + ": " + cause.getMessage else "unknown") +
|
||||
"]"
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -6,9 +6,9 @@ package akka.routing
|
|||
import akka.actor._
|
||||
import akka.remote._
|
||||
import scala.collection.JavaConverters._
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.config.ConfigurationException
|
||||
import akka.util.Duration
|
||||
|
||||
trait RemoteRouterConfig extends RouterConfig {
|
||||
override protected def createRoutees(props: Props, context: ActorContext, nrOfInstances: Int, routees: Iterable[String]): Vector[ActorRef] = (nrOfInstances, routees) match {
|
||||
|
|
@ -99,12 +99,12 @@ case class RemoteBroadcastRouter(nrOfInstances: Int, routees: Iterable[String])
|
|||
* if you provide either 'nrOfInstances' or 'routees' to during instantiation they will
|
||||
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
|
||||
*/
|
||||
case class RemoteScatterGatherFirstCompletedRouter(nrOfInstances: Int, routees: Iterable[String])
|
||||
case class RemoteScatterGatherFirstCompletedRouter(nrOfInstances: Int, routees: Iterable[String], within: Duration)
|
||||
extends RemoteRouterConfig with ScatterGatherFirstCompletedLike {
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
* Java API
|
||||
*/
|
||||
def this(n: Int, t: java.util.Collection[String]) = this(n, t.asScala)
|
||||
def this(n: Int, t: java.util.Collection[String], w: Duration) = this(n, t.asScala, w)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,25 +0,0 @@
|
|||
##################################
|
||||
# Akka STM Reference Config File #
|
||||
##################################
|
||||
|
||||
# This the reference config file has all the default settings.
|
||||
# Make your edits/overrides in your application.conf.
|
||||
|
||||
akka {
|
||||
|
||||
stm {
|
||||
# Should global transactions be fair or non-fair (non fair yield better performance)
|
||||
fair = on
|
||||
max-retries = 1000
|
||||
# Default timeout for blocking transactions and transaction set
|
||||
timeout = 5s
|
||||
write-skew = on
|
||||
blocking-allowed = off
|
||||
interruptible = off
|
||||
speculative = on
|
||||
quick-release = on
|
||||
propagation = "requires"
|
||||
trace-level = "none"
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,40 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
/**
|
||||
* Java-friendly atomic blocks.
|
||||
*
|
||||
* Example usage ''(Java)''
|
||||
*
|
||||
* {{{
|
||||
* import akka.stm.*;
|
||||
*
|
||||
* final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
*
|
||||
* new Atomic() {
|
||||
* public Object atomically() {
|
||||
* return ref.set(1);
|
||||
* }
|
||||
* }.execute();
|
||||
*
|
||||
* // To configure transactions pass a TransactionFactory
|
||||
*
|
||||
* TransactionFactory txFactory = new TransactionFactoryBuilder()
|
||||
* .setReadonly(true)
|
||||
* .build();
|
||||
*
|
||||
* Integer value = new Atomic<Integer>(txFactory) {
|
||||
* public Integer atomically() {
|
||||
* return ref.get();
|
||||
* }
|
||||
* }.execute();
|
||||
* }}}
|
||||
*/
|
||||
abstract class Atomic[T](val factory: TransactionFactory) {
|
||||
def this() = this(DefaultTransactionFactory)
|
||||
def atomically: T
|
||||
def execute: T = atomic(factory)(atomically)
|
||||
}
|
||||
|
|
@ -1,131 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import akka.actor.{ newUuid, Uuid }
|
||||
|
||||
import org.multiverse.transactional.refs.BasicRef
|
||||
|
||||
/**
|
||||
* Common trait for all the transactional objects.
|
||||
*/
|
||||
trait Transactional extends Serializable {
|
||||
val uuid: String
|
||||
}
|
||||
|
||||
/**
|
||||
* Transactional managed reference. See the companion class for more information.
|
||||
*/
|
||||
object Ref {
|
||||
def apply[T]() = new Ref[T]()
|
||||
|
||||
def apply[T](initialValue: T) = new Ref[T](initialValue)
|
||||
|
||||
/**
|
||||
* An implicit conversion that converts a Ref to an Iterable value.
|
||||
*/
|
||||
implicit def ref2Iterable[T](ref: Ref[T]): Iterable[T] = ref.toList
|
||||
}
|
||||
|
||||
/**
|
||||
* Refs (transactional references) are mutable references to values and through
|
||||
* the STM allow the safe sharing of mutable data. Refs separate identity from value.
|
||||
* To ensure safety the value stored in a Ref should be immutable (they can also
|
||||
* contain refs themselves). The value referenced by a Ref can only be accessed
|
||||
* or swapped within a transaction. If a transaction is not available, the call will
|
||||
* be executed in its own transaction.
|
||||
* <br/><br/>
|
||||
*
|
||||
* Creating a Ref ''(Scala)''
|
||||
*
|
||||
* {{{
|
||||
* import akka.stm._
|
||||
*
|
||||
* // giving an initial value
|
||||
* val ref = Ref(0)
|
||||
*
|
||||
* // specifying a type but no initial value
|
||||
* val ref = Ref[Int]
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* Creating a Ref ''(Java)''
|
||||
*
|
||||
* {{{
|
||||
* import akka.stm.*;
|
||||
*
|
||||
* // giving an initial value
|
||||
* final Ref<Integer> ref = new Ref<Integer>(0);
|
||||
*
|
||||
* // specifying a type but no initial value
|
||||
* final Ref<Integer> ref = new Ref<Integer>();
|
||||
* }}}
|
||||
*/
|
||||
class Ref[T](initialValue: T) extends BasicRef[T](initialValue) with Transactional {
|
||||
self ⇒
|
||||
|
||||
def this() = this(null.asInstanceOf[T])
|
||||
|
||||
val uuid = newUuid.toString
|
||||
|
||||
def apply() = get
|
||||
|
||||
def update(newValue: T) = set(newValue)
|
||||
|
||||
def swap(newValue: T) = set(newValue)
|
||||
|
||||
def alter(f: T ⇒ T): T = {
|
||||
val value = f(get)
|
||||
set(value)
|
||||
value
|
||||
}
|
||||
|
||||
def opt: Option[T] = Option(get)
|
||||
|
||||
def getOrWait: T = getOrAwait
|
||||
|
||||
def getOrElse(default: ⇒ T): T =
|
||||
if (isNull) default else get
|
||||
|
||||
def isDefined: Boolean = !isNull
|
||||
|
||||
def isEmpty: Boolean = isNull
|
||||
|
||||
def map[B](f: T ⇒ B): Ref[B] =
|
||||
if (isEmpty) Ref[B] else Ref(f(get))
|
||||
|
||||
def flatMap[B](f: T ⇒ Ref[B]): Ref[B] =
|
||||
if (isEmpty) Ref[B] else f(get)
|
||||
|
||||
def filter(p: T ⇒ Boolean): Ref[T] =
|
||||
if (isDefined && p(get)) Ref(get) else Ref[T]
|
||||
|
||||
/**
|
||||
* Necessary to keep from being implicitly converted to Iterable in for comprehensions.
|
||||
*/
|
||||
def withFilter(p: T ⇒ Boolean): WithFilter = new WithFilter(p)
|
||||
|
||||
class WithFilter(p: T ⇒ Boolean) {
|
||||
def map[B](f: T ⇒ B): Ref[B] = self filter p map f
|
||||
def flatMap[B](f: T ⇒ Ref[B]): Ref[B] = self filter p flatMap f
|
||||
def foreach[U](f: T ⇒ U): Unit = self filter p foreach f
|
||||
def withFilter(q: T ⇒ Boolean): WithFilter = new WithFilter(x ⇒ p(x) && q(x))
|
||||
}
|
||||
|
||||
def foreach[U](f: T ⇒ U): Unit =
|
||||
if (isDefined) f(get)
|
||||
|
||||
def elements: Iterator[T] =
|
||||
if (isEmpty) Iterator.empty else Iterator(get)
|
||||
|
||||
def toList: List[T] =
|
||||
if (isEmpty) List() else List(get)
|
||||
|
||||
def toRight[X](left: ⇒ X) =
|
||||
if (isEmpty) Left(left) else Right(get)
|
||||
|
||||
def toLeft[X](right: ⇒ X) =
|
||||
if (isEmpty) Right(right) else Left(get)
|
||||
}
|
||||
|
|
@ -1,177 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import org.multiverse.api.{ StmUtils ⇒ MultiverseStmUtils }
|
||||
import org.multiverse.api.{ Transaction ⇒ MultiverseTransaction }
|
||||
import org.multiverse.templates.{ TransactionalCallable, OrElseTemplate }
|
||||
|
||||
object Stm {
|
||||
/**
|
||||
* Check whether there is an active Multiverse transaction.
|
||||
*/
|
||||
def activeTransaction() = {
|
||||
val tx = org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
|
||||
(tx ne null) && !tx.getStatus.isDead
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines the atomic block for local transactions. Automatically imported with:
|
||||
*
|
||||
* {{{
|
||||
* import akka.stm._
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* If you need to coordinate transactions across actors see [[akka.stm.Coordinated]].
|
||||
* <br/><br/>
|
||||
*
|
||||
* Example of using the atomic block ''(Scala)''
|
||||
*
|
||||
* {{{
|
||||
* atomic {
|
||||
* // do something within a transaction
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* @see [[akka.stm.Atomic]] for creating atomic blocks in Java.
|
||||
* @see [[akka.stm.StmUtil]] for useful methods to combine with `atomic`
|
||||
*/
|
||||
trait Stm {
|
||||
val DefaultTransactionFactory = TransactionFactory(DefaultTransactionConfig, "DefaultTransaction")
|
||||
|
||||
def atomic[T](body: ⇒ T)(implicit factory: TransactionFactory = DefaultTransactionFactory): T =
|
||||
atomic(factory)(body)
|
||||
|
||||
def atomic[T](factory: TransactionFactory)(body: ⇒ T): T = {
|
||||
factory.boilerplate.execute(new TransactionalCallable[T]() {
|
||||
def call(mtx: MultiverseTransaction): T = body
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stm utility methods for scheduling transaction lifecycle tasks and for blocking transactions.
|
||||
* Automatically imported with:
|
||||
*
|
||||
* {{{
|
||||
* import akka.stm._
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* Schedule a deferred task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction commits.
|
||||
*
|
||||
* {{{
|
||||
* atomic {
|
||||
* deferred {
|
||||
* // executes when transaction successfully commits
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* Schedule a compensating task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction aborts.
|
||||
*
|
||||
* {{{
|
||||
* atomic {
|
||||
* compensating {
|
||||
* // executes when transaction aborts
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* STM retry for blocking transactions (use within an atomic).
|
||||
* Can be used to wait for a condition.
|
||||
*
|
||||
* {{{
|
||||
* atomic {
|
||||
* if (!someCondition) retry
|
||||
* // ...
|
||||
* }
|
||||
* }}}
|
||||
* <br/>
|
||||
*
|
||||
* Use either-orElse to combine two blocking transactions.
|
||||
*
|
||||
* {{{
|
||||
* atomic {
|
||||
* either {
|
||||
* // ...
|
||||
* } orElse {
|
||||
* // ...
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
* <br/>
|
||||
*/
|
||||
trait StmUtil {
|
||||
/**
|
||||
* Schedule a deferred task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction commits.
|
||||
*/
|
||||
def deferred[T](body: ⇒ T): Unit =
|
||||
MultiverseStmUtils.scheduleDeferredTask(new Runnable { def run = body })
|
||||
|
||||
/**
|
||||
* Schedule a compensating task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction aborts.
|
||||
*/
|
||||
def compensating[T](body: ⇒ T): Unit =
|
||||
MultiverseStmUtils.scheduleCompensatingTask(new Runnable { def run = body })
|
||||
|
||||
/**
|
||||
* STM retry for blocking transactions (use within an atomic).
|
||||
* Can be used to wait for a condition.
|
||||
*/
|
||||
def retry() = MultiverseStmUtils.retry
|
||||
|
||||
/**
|
||||
* Use either-orElse to combine two blocking transactions.
|
||||
*/
|
||||
def either[T](firstBody: ⇒ T) = new {
|
||||
def orElse(secondBody: ⇒ T) = new OrElseTemplate[T] {
|
||||
def either(mtx: MultiverseTransaction) = firstBody
|
||||
def orelse(mtx: MultiverseTransaction) = secondBody
|
||||
}.execute()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stm utility methods for using from Java.
|
||||
*/
|
||||
object StmUtils {
|
||||
/**
|
||||
* Schedule a deferred task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction commits.
|
||||
*/
|
||||
def deferred(runnable: Runnable): Unit = MultiverseStmUtils.scheduleDeferredTask(runnable)
|
||||
|
||||
/**
|
||||
* Schedule a compensating task on the thread local transaction (use within an atomic).
|
||||
* This is executed when the transaction aborts.
|
||||
*/
|
||||
def compensating(runnable: Runnable): Unit = MultiverseStmUtils.scheduleCompensatingTask(runnable)
|
||||
|
||||
/**
|
||||
* STM retry for blocking transactions (use within an atomic).
|
||||
* Can be used to wait for a condition.
|
||||
*/
|
||||
def retry = MultiverseStmUtils.retry
|
||||
}
|
||||
|
||||
/**
|
||||
* Use EitherOrElse to combine two blocking transactions (from Java).
|
||||
*/
|
||||
abstract class EitherOrElse[T] extends OrElseTemplate[T] {
|
||||
def either(mtx: MultiverseTransaction) = either
|
||||
def orelse(mtx: MultiverseTransaction) = orElse
|
||||
|
||||
def either: T
|
||||
def orElse: T
|
||||
}
|
||||
|
|
@ -1,202 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import java.lang.{ Boolean ⇒ JBoolean }
|
||||
|
||||
import akka.util.Duration
|
||||
|
||||
import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance
|
||||
import org.multiverse.stms.alpha.AlphaStm
|
||||
import org.multiverse.templates.TransactionBoilerplate
|
||||
import org.multiverse.api.PropagationLevel
|
||||
import org.multiverse.api.{ TraceLevel ⇒ MTraceLevel }
|
||||
|
||||
/**
|
||||
* For configuring multiverse transactions.
|
||||
*/
|
||||
object TransactionConfig {
|
||||
object Default {
|
||||
// note: null values are so that we can default to Multiverse inference when not set
|
||||
val FamilyName = "DefaultTransaction"
|
||||
val Readonly = null.asInstanceOf[JBoolean]
|
||||
val MaxRetries = 1000
|
||||
val Timeout = Duration(5, "seconds")
|
||||
val TrackReads = null.asInstanceOf[JBoolean]
|
||||
val WriteSkew = true
|
||||
val BlockingAllowed = false
|
||||
val Interruptible = false
|
||||
val Speculative = true
|
||||
val QuickRelease = true
|
||||
val Propagation = PropagationLevel.Requires
|
||||
val TraceLevel = MTraceLevel.none
|
||||
}
|
||||
|
||||
/**
|
||||
* For configuring multiverse transactions.
|
||||
*
|
||||
* @param familyName Family name for transactions. Useful for debugging.
|
||||
* @param readonly Sets transaction as readonly. Readonly transactions are cheaper.
|
||||
* @param maxRetries The maximum number of times a transaction will retry.
|
||||
* @param timeout The maximum time a transaction will block for.
|
||||
* @param trackReads Whether all reads should be tracked. Needed for blocking operations.
|
||||
* @param writeSkew Whether writeskew is allowed. Disable with care.
|
||||
* @param blockingAllowed Whether explicit retries are allowed.
|
||||
* @param interruptible Whether a blocking transaction can be interrupted.
|
||||
* @param speculative Whether speculative configuration should be enabled.
|
||||
* @param quickRelease Whether locks should be released as quickly as possible (before whole commit).
|
||||
* @param propagation For controlling how nested transactions behave.
|
||||
* @param traceLevel Transaction trace level.
|
||||
*/
|
||||
def apply(familyName: String = Default.FamilyName,
|
||||
readonly: JBoolean = Default.Readonly,
|
||||
maxRetries: Int = Default.MaxRetries,
|
||||
timeout: Duration = Default.Timeout,
|
||||
trackReads: JBoolean = Default.TrackReads,
|
||||
writeSkew: Boolean = Default.WriteSkew,
|
||||
blockingAllowed: Boolean = Default.BlockingAllowed,
|
||||
interruptible: Boolean = Default.Interruptible,
|
||||
speculative: Boolean = Default.Speculative,
|
||||
quickRelease: Boolean = Default.QuickRelease,
|
||||
propagation: PropagationLevel = Default.Propagation,
|
||||
traceLevel: MTraceLevel = Default.TraceLevel) = {
|
||||
new TransactionConfig(familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
|
||||
interruptible, speculative, quickRelease, propagation, traceLevel)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For configuring multiverse transactions.
|
||||
*
|
||||
* <p>familyName - Family name for transactions. Useful for debugging.
|
||||
* <p>readonly - Sets transaction as readonly. Readonly transactions are cheaper.
|
||||
* <p>maxRetries - The maximum number of times a transaction will retry.
|
||||
* <p>timeout - The maximum time a transaction will block for.
|
||||
* <p>trackReads - Whether all reads should be tracked. Needed for blocking operations.
|
||||
* <p>writeSkew - Whether writeskew is allowed. Disable with care.
|
||||
* <p>blockingAllowed - Whether explicit retries are allowed.
|
||||
* <p>interruptible - Whether a blocking transaction can be interrupted.
|
||||
* <p>speculative - Whether speculative configuration should be enabled.
|
||||
* <p>quickRelease - Whether locks should be released as quickly as possible (before whole commit).
|
||||
* <p>propagation - For controlling how nested transactions behave.
|
||||
* <p>traceLevel - Transaction trace level.
|
||||
*/
|
||||
class TransactionConfig(val familyName: String = TransactionConfig.Default.FamilyName,
|
||||
val readonly: JBoolean = TransactionConfig.Default.Readonly,
|
||||
val maxRetries: Int = TransactionConfig.Default.MaxRetries,
|
||||
val timeout: Duration = TransactionConfig.Default.Timeout,
|
||||
val trackReads: JBoolean = TransactionConfig.Default.TrackReads,
|
||||
val writeSkew: Boolean = TransactionConfig.Default.WriteSkew,
|
||||
val blockingAllowed: Boolean = TransactionConfig.Default.BlockingAllowed,
|
||||
val interruptible: Boolean = TransactionConfig.Default.Interruptible,
|
||||
val speculative: Boolean = TransactionConfig.Default.Speculative,
|
||||
val quickRelease: Boolean = TransactionConfig.Default.QuickRelease,
|
||||
val propagation: PropagationLevel = TransactionConfig.Default.Propagation,
|
||||
val traceLevel: MTraceLevel = TransactionConfig.Default.TraceLevel)
|
||||
|
||||
object DefaultTransactionConfig extends TransactionConfig
|
||||
|
||||
/**
|
||||
* Wrapper for transaction config, factory, and boilerplate. Used by atomic.
|
||||
*/
|
||||
object TransactionFactory {
|
||||
def apply(config: TransactionConfig) = new TransactionFactory(config)
|
||||
|
||||
def apply(config: TransactionConfig, defaultName: String) = new TransactionFactory(config, defaultName)
|
||||
|
||||
def apply(familyName: String = TransactionConfig.Default.FamilyName,
|
||||
readonly: JBoolean = TransactionConfig.Default.Readonly,
|
||||
maxRetries: Int = TransactionConfig.Default.MaxRetries,
|
||||
timeout: Duration = TransactionConfig.Default.Timeout,
|
||||
trackReads: JBoolean = TransactionConfig.Default.TrackReads,
|
||||
writeSkew: Boolean = TransactionConfig.Default.WriteSkew,
|
||||
blockingAllowed: Boolean = TransactionConfig.Default.BlockingAllowed,
|
||||
interruptible: Boolean = TransactionConfig.Default.Interruptible,
|
||||
speculative: Boolean = TransactionConfig.Default.Speculative,
|
||||
quickRelease: Boolean = TransactionConfig.Default.QuickRelease,
|
||||
propagation: PropagationLevel = TransactionConfig.Default.Propagation,
|
||||
traceLevel: MTraceLevel = TransactionConfig.Default.TraceLevel) = {
|
||||
val config = new TransactionConfig(
|
||||
familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
|
||||
interruptible, speculative, quickRelease, propagation, traceLevel)
|
||||
new TransactionFactory(config)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper for transaction config, factory, and boilerplate. Used by atomic.
|
||||
* Can be passed to atomic implicitly or explicitly.
|
||||
*
|
||||
* {{{
|
||||
* implicit val txFactory = TransactionFactory(readonly = true)
|
||||
* ...
|
||||
* atomic {
|
||||
* // do something within a readonly transaction
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* Can be created at different levels as needed. For example: as an implicit object
|
||||
* used throughout a package, as a static implicit val within a singleton object and
|
||||
* imported where needed, or as an implicit val within each instance of a class.
|
||||
*
|
||||
* If no explicit transaction factory is passed to atomic and there is no implicit
|
||||
* transaction factory in scope, then a default transaction factory is used.
|
||||
*
|
||||
* @see [[akka.stm.TransactionConfig]] for configuration options.
|
||||
*/
|
||||
class TransactionFactory(
|
||||
val config: TransactionConfig = DefaultTransactionConfig,
|
||||
defaultName: String = TransactionConfig.Default.FamilyName) { self ⇒
|
||||
|
||||
// use the config family name if it's been set, otherwise defaultName - used by actors to set class name as default
|
||||
val familyName = if (config.familyName != TransactionConfig.Default.FamilyName) config.familyName else defaultName
|
||||
|
||||
val factory = {
|
||||
var builder = (getGlobalStmInstance().asInstanceOf[AlphaStm].getTransactionFactoryBuilder()
|
||||
.setFamilyName(familyName)
|
||||
.setMaxRetries(config.maxRetries)
|
||||
.setTimeoutNs(config.timeout.toNanos)
|
||||
.setWriteSkewAllowed(config.writeSkew)
|
||||
.setExplicitRetryAllowed(config.blockingAllowed)
|
||||
.setInterruptible(config.interruptible)
|
||||
.setSpeculativeConfigurationEnabled(config.speculative)
|
||||
.setQuickReleaseEnabled(config.quickRelease)
|
||||
.setPropagationLevel(config.propagation)
|
||||
.setTraceLevel(config.traceLevel))
|
||||
|
||||
if (config.readonly ne null) {
|
||||
builder = builder.setReadonly(config.readonly.booleanValue)
|
||||
} // otherwise default to Multiverse inference
|
||||
|
||||
if (config.trackReads ne null) {
|
||||
builder = builder.setReadTrackingEnabled(config.trackReads.booleanValue)
|
||||
} // otherwise default to Multiverse inference
|
||||
|
||||
builder.build()
|
||||
}
|
||||
|
||||
val boilerplate = new TransactionBoilerplate(factory)
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping to Multiverse PropagationLevel.
|
||||
*/
|
||||
object Propagation {
|
||||
val RequiresNew = PropagationLevel.RequiresNew
|
||||
val Mandatory = PropagationLevel.Mandatory
|
||||
val Requires = PropagationLevel.Requires
|
||||
val Supports = PropagationLevel.Supports
|
||||
val Never = PropagationLevel.Never
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping to Multiverse TraceLevel.
|
||||
*/
|
||||
object TraceLevel {
|
||||
val None = MTraceLevel.none
|
||||
val Coarse = MTraceLevel.course // mispelling?
|
||||
val Course = MTraceLevel.course
|
||||
val Fine = MTraceLevel.fine
|
||||
}
|
||||
|
|
@ -1,85 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import java.lang.{ Boolean ⇒ JBoolean }
|
||||
|
||||
import akka.util.Duration
|
||||
|
||||
import org.multiverse.api.{ TraceLevel ⇒ MTraceLevel }
|
||||
import org.multiverse.api.PropagationLevel
|
||||
|
||||
/**
|
||||
* For more easily creating TransactionConfig from Java.
|
||||
*/
|
||||
class TransactionConfigBuilder {
|
||||
var familyName: String = TransactionConfig.Default.FamilyName
|
||||
var readonly: JBoolean = TransactionConfig.Default.Readonly
|
||||
var maxRetries: Int = TransactionConfig.Default.MaxRetries
|
||||
var timeout: Duration = TransactionConfig.Default.Timeout
|
||||
var trackReads: JBoolean = TransactionConfig.Default.TrackReads
|
||||
var writeSkew: Boolean = TransactionConfig.Default.WriteSkew
|
||||
var blockingAllowed: Boolean = TransactionConfig.Default.BlockingAllowed
|
||||
var interruptible: Boolean = TransactionConfig.Default.Interruptible
|
||||
var speculative: Boolean = TransactionConfig.Default.Speculative
|
||||
var quickRelease: Boolean = TransactionConfig.Default.QuickRelease
|
||||
var propagation: PropagationLevel = TransactionConfig.Default.Propagation
|
||||
var traceLevel: MTraceLevel = TransactionConfig.Default.TraceLevel
|
||||
|
||||
def setFamilyName(familyName: String) = { this.familyName = familyName; this }
|
||||
def setReadonly(readonly: JBoolean) = { this.readonly = readonly; this }
|
||||
def setMaxRetries(maxRetries: Int) = { this.maxRetries = maxRetries; this }
|
||||
def setTimeout(timeout: Duration) = { this.timeout = timeout; this }
|
||||
def setTrackReads(trackReads: JBoolean) = { this.trackReads = trackReads; this }
|
||||
def setWriteSkew(writeSkew: Boolean) = { this.writeSkew = writeSkew; this }
|
||||
def setBlockingAllowed(blockingAllowed: Boolean) = { this.blockingAllowed = blockingAllowed; this }
|
||||
def setInterruptible(interruptible: Boolean) = { this.interruptible = interruptible; this }
|
||||
def setSpeculative(speculative: Boolean) = { this.speculative = speculative; this }
|
||||
def setQuickRelease(quickRelease: Boolean) = { this.quickRelease = quickRelease; this }
|
||||
def setPropagation(propagation: PropagationLevel) = { this.propagation = propagation; this }
|
||||
def setTraceLevel(traceLevel: MTraceLevel) = { this.traceLevel = traceLevel; this }
|
||||
|
||||
def build() = new TransactionConfig(
|
||||
familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
|
||||
interruptible, speculative, quickRelease, propagation, traceLevel)
|
||||
}
|
||||
|
||||
/**
|
||||
* For more easily creating TransactionFactory from Java.
|
||||
*/
|
||||
class TransactionFactoryBuilder {
|
||||
var familyName: String = TransactionConfig.Default.FamilyName
|
||||
var readonly: JBoolean = TransactionConfig.Default.Readonly
|
||||
var maxRetries: Int = TransactionConfig.Default.MaxRetries
|
||||
var timeout: Duration = TransactionConfig.Default.Timeout
|
||||
var trackReads: JBoolean = TransactionConfig.Default.TrackReads
|
||||
var writeSkew: Boolean = TransactionConfig.Default.WriteSkew
|
||||
var blockingAllowed: Boolean = TransactionConfig.Default.BlockingAllowed
|
||||
var interruptible: Boolean = TransactionConfig.Default.Interruptible
|
||||
var speculative: Boolean = TransactionConfig.Default.Speculative
|
||||
var quickRelease: Boolean = TransactionConfig.Default.QuickRelease
|
||||
var propagation: PropagationLevel = TransactionConfig.Default.Propagation
|
||||
var traceLevel: MTraceLevel = TransactionConfig.Default.TraceLevel
|
||||
|
||||
def setFamilyName(familyName: String) = { this.familyName = familyName; this }
|
||||
def setReadonly(readonly: JBoolean) = { this.readonly = readonly; this }
|
||||
def setMaxRetries(maxRetries: Int) = { this.maxRetries = maxRetries; this }
|
||||
def setTimeout(timeout: Duration) = { this.timeout = timeout; this }
|
||||
def setTrackReads(trackReads: JBoolean) = { this.trackReads = trackReads; this }
|
||||
def setWriteSkew(writeSkew: Boolean) = { this.writeSkew = writeSkew; this }
|
||||
def setBlockingAllowed(blockingAllowed: Boolean) = { this.blockingAllowed = blockingAllowed; this }
|
||||
def setInterruptible(interruptible: Boolean) = { this.interruptible = interruptible; this }
|
||||
def setSpeculative(speculative: Boolean) = { this.speculative = speculative; this }
|
||||
def setQuickRelease(quickRelease: Boolean) = { this.quickRelease = quickRelease; this }
|
||||
def setPropagation(propagation: PropagationLevel) = { this.propagation = propagation; this }
|
||||
def setTraceLevel(traceLevel: MTraceLevel) = { this.traceLevel = traceLevel; this }
|
||||
|
||||
def build() = {
|
||||
val config = new TransactionConfig(
|
||||
familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
|
||||
interruptible, speculative, quickRelease, propagation, traceLevel)
|
||||
new TransactionFactory(config)
|
||||
}
|
||||
}
|
||||
|
|
@ -1,89 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import scala.collection.immutable.HashMap
|
||||
|
||||
import akka.actor.{ newUuid }
|
||||
|
||||
/**
|
||||
* Transactional map that implements the mutable Map interface with an underlying Ref and HashMap.
|
||||
*/
|
||||
object TransactionalMap {
|
||||
def apply[K, V]() = new TransactionalMap[K, V]()
|
||||
|
||||
def apply[K, V](pairs: (K, V)*) = new TransactionalMap(HashMap(pairs: _*))
|
||||
}
|
||||
|
||||
/**
|
||||
* Transactional map that implements the mutable Map interface with an underlying Ref and HashMap.
|
||||
*
|
||||
* TransactionalMap and TransactionalVector look like regular mutable datastructures, they even
|
||||
* implement the standard Scala 'Map' and 'IndexedSeq' interfaces, but they are implemented using
|
||||
* persistent datastructures and managed references under the hood. Therefore they are safe to use
|
||||
* in a concurrent environment through the STM. Underlying TransactionalMap is HashMap, an immutable
|
||||
* Map but with near constant time access and modification operations.
|
||||
*
|
||||
* From Scala you can use TMap as a shorter alias for TransactionalMap.
|
||||
*/
|
||||
class TransactionalMap[K, V](initialValue: HashMap[K, V]) extends Transactional with scala.collection.mutable.Map[K, V] {
|
||||
def this() = this(HashMap[K, V]())
|
||||
|
||||
val uuid = newUuid.toString
|
||||
|
||||
private[this] val ref = Ref(initialValue)
|
||||
|
||||
def -=(key: K) = {
|
||||
remove(key)
|
||||
this
|
||||
}
|
||||
|
||||
def +=(key: K, value: V) = put(key, value)
|
||||
|
||||
def +=(kv: (K, V)) = {
|
||||
put(kv._1, kv._2)
|
||||
this
|
||||
}
|
||||
|
||||
override def remove(key: K) = {
|
||||
val map = ref.get
|
||||
val oldValue = map.get(key)
|
||||
ref.swap(ref.get - key)
|
||||
oldValue
|
||||
}
|
||||
|
||||
def get(key: K): Option[V] = ref.get.get(key)
|
||||
|
||||
override def put(key: K, value: V): Option[V] = {
|
||||
val map = ref.get
|
||||
val oldValue = map.get(key)
|
||||
ref.swap(map.updated(key, value))
|
||||
oldValue
|
||||
}
|
||||
|
||||
override def update(key: K, value: V) = {
|
||||
val map = ref.get
|
||||
val oldValue = map.get(key)
|
||||
ref.swap(map.updated(key, value))
|
||||
}
|
||||
|
||||
def iterator = ref.get.iterator
|
||||
|
||||
override def elements: Iterator[(K, V)] = ref.get.iterator
|
||||
|
||||
override def contains(key: K): Boolean = ref.get.contains(key)
|
||||
|
||||
override def clear = ref.swap(HashMap[K, V]())
|
||||
|
||||
override def size: Int = ref.get.size
|
||||
|
||||
override def hashCode: Int = System.identityHashCode(this);
|
||||
|
||||
override def equals(other: Any): Boolean =
|
||||
other.isInstanceOf[TransactionalMap[_, _]] &&
|
||||
other.hashCode == hashCode
|
||||
|
||||
override def toString = if (Stm.activeTransaction) super.toString else "<TransactionalMap>"
|
||||
}
|
||||
|
|
@ -1,65 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.stm
|
||||
|
||||
import scala.collection.immutable.Vector
|
||||
|
||||
import akka.actor.newUuid
|
||||
|
||||
/**
|
||||
* Transactional vector that implements the IndexedSeq interface with an underlying Ref and Vector.
|
||||
*/
|
||||
object TransactionalVector {
|
||||
def apply[T]() = new TransactionalVector[T]()
|
||||
|
||||
def apply[T](elems: T*) = new TransactionalVector(Vector(elems: _*))
|
||||
}
|
||||
|
||||
/**
|
||||
* Transactional vector that implements the IndexedSeq interface with an underlying Ref and Vector.
|
||||
*
|
||||
* TransactionalMap and TransactionalVector look like regular mutable datastructures, they even
|
||||
* implement the standard Scala 'Map' and 'IndexedSeq' interfaces, but they are implemented using
|
||||
* persistent datastructures and managed references under the hood. Therefore they are safe to use
|
||||
* in a concurrent environment through the STM. Underlying TransactionalVector is Vector, an immutable
|
||||
* sequence but with near constant time access and modification operations.
|
||||
*
|
||||
* From Scala you can use TVector as a shorter alias for TransactionalVector.
|
||||
*/
|
||||
class TransactionalVector[T](initialValue: Vector[T]) extends Transactional with IndexedSeq[T] {
|
||||
def this() = this(Vector[T]())
|
||||
|
||||
val uuid = newUuid.toString
|
||||
|
||||
private[this] val ref = Ref(initialValue)
|
||||
|
||||
def clear = ref.swap(Vector[T]())
|
||||
|
||||
def +(elem: T) = add(elem)
|
||||
|
||||
def add(elem: T) = ref.swap(ref.get :+ elem)
|
||||
|
||||
def get(index: Int): T = ref.get.apply(index)
|
||||
|
||||
/**
|
||||
* Removes the <i>tail</i> element of this vector.
|
||||
*/
|
||||
def pop = ref.swap(ref.get.dropRight(1))
|
||||
|
||||
def update(index: Int, elem: T) = ref.swap(ref.get.updated(index, elem))
|
||||
|
||||
def length: Int = ref.get.length
|
||||
|
||||
def apply(index: Int): T = ref.get.apply(index)
|
||||
|
||||
override def hashCode: Int = System.identityHashCode(this);
|
||||
|
||||
override def equals(other: Any): Boolean =
|
||||
other.isInstanceOf[TransactionalVector[_]] &&
|
||||
other.hashCode == hashCode
|
||||
|
||||
override def toString = if (Stm.activeTransaction) super.toString else "<TransactionalVector>"
|
||||
}
|
||||
|
||||
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