remove app argument from Dispatchers

(cascading into all dispatchers, mailboxes, other stuff; had to move
deadLetter stuff to ActorSystem again and split its initialization due
to cyclic dependency)
This commit is contained in:
Roland 2011-11-14 16:03:26 +01:00
parent 79daccdce1
commit f2bf27b4de
20 changed files with 180 additions and 130 deletions

View file

@ -150,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)") assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)")
val actor = new TestActorRef[TestLogActor](app, Props[TestLogActor], supervisor, "none") val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none")
val set = receiveWhile(messages = 2) { val set = receiveWhile(messages = 2) {
case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" 1 case Logging.Debug(`supervisor`, msg: String) if msg startsWith "now supervising" 1

View file

@ -421,10 +421,10 @@ class DispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, "foo", app.AkkaConfig.DispatcherThroughput, new Dispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", app.AkkaConfig.DispatcherThroughput,
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Dispatcher" def dispatcherType = "Dispatcher"
@ -458,10 +458,10 @@ class BalancingDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config def newInterceptedDispatcher = ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, "foo", 1, // TODO check why 1 here? (came from old test) new BalancingDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "foo", 1, // TODO check why 1 here? (came from old test)
app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType, app.dispatcherFactory.ThroughputDeadlineTimeMillis, app.dispatcherFactory.MailboxType,
config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor, config, app.dispatcherFactory.DispatcherShutdownMillis) with MessageDispatcherInterceptor,
ThreadPoolConfig(app)).build.asInstanceOf[MessageDispatcherInterceptor] ThreadPoolConfig(app.eventStream)).build.asInstanceOf[MessageDispatcherInterceptor]
def dispatcherType = "Balancing Dispatcher" def dispatcherType = "Balancing Dispatcher"

View file

@ -11,7 +11,7 @@ import org.junit.{ After, Test }
class CallingThreadDispatcherModelSpec extends ActorModelSpec { class CallingThreadDispatcherModelSpec extends ActorModelSpec {
import ActorModelSpec._ import ActorModelSpec._
def newInterceptedDispatcher = new CallingThreadDispatcher(app, "test") with MessageDispatcherInterceptor def newInterceptedDispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, "test") with MessageDispatcherInterceptor
def dispatcherType = "Calling Thread Dispatcher" def dispatcherType = "Calling Thread Dispatcher"
} }

View file

@ -385,8 +385,13 @@ object DeadLetterActorRef {
val serialized = new SerializedDeadLetterActorRef val serialized = new SerializedDeadLetterActorRef
} }
class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath, val dispatcher: MessageDispatcher) extends MinimalActorRef { class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath) extends MinimalActorRef {
val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher) @volatile
var brokenPromise: Future[Any] = _
private[akka] def init(dispatcher: MessageDispatcher) {
brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(dispatcher)
}
override val name: String = "dead-letter" override val name: String = "dead-letter"
@ -401,6 +406,8 @@ class DeadLetterActorRef(val eventStream: EventStream, val path: ActorPath, val
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
eventStream.publish(DeadLetter(message, this, this)) eventStream.publish(DeadLetter(message, this, this))
// leave this in: guard with good visibility against really stupid/weird errors
assert(brokenPromise != null)
brokenPromise brokenPromise
} }

View file

@ -31,10 +31,6 @@ trait ActorRefProvider {
def deathWatch: DeathWatch def deathWatch: DeathWatch
def deadLetters: ActorRef
def deadLetterMailbox: Mailbox
/** /**
* What deployer will be used to resolve deployment configuration? * What deployer will be used to resolve deployment configuration?
*/ */
@ -144,19 +140,6 @@ class LocalActorRefProvider(
// currently still used for tmp actors (e.g. ask actor refs) // currently still used for tmp actors (e.g. ask actor refs)
private val actors = new ConcurrentHashMap[String, AnyRef] private val actors = new ConcurrentHashMap[String, AnyRef]
val deadLetters = new DeadLetterActorRef(eventStream, root / "nul", dispatcher)
val deadLetterMailbox = new Mailbox(null) {
becomeClosed()
override def dispatcher = null //MessageDispatcher.this
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
override def dequeue() = null
override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) }
override def systemDrain(): SystemMessage = null
override def hasMessages = false
override def hasSystemMessages = false
override def numberOfMessages = 0
}
/** /**
* Top-level anchor for the supervision hierarchy of this actor system. Will * Top-level anchor for the supervision hierarchy of this actor system. Will
* receive only Supervise/ChildTerminated system messages or Failure message. * receive only Supervise/ChildTerminated system messages or Failure message.

View file

@ -158,20 +158,34 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
// this provides basic logging (to stdout) until .start() is called below // this provides basic logging (to stdout) until .start() is called below
val eventStream = new EventStream(DebugEventStream) val eventStream = new EventStream(DebugEventStream)
eventStream.startStdoutLogger(AkkaConfig) eventStream.startStdoutLogger(AkkaConfig)
val log = new BusLogging(eventStream, this) val log = new BusLogging(eventStream, this) // this used only for .getClass in tagging messages
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(this)
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
/** /**
* The root actor path for this application. * The root actor path for this application.
*/ */
val root: ActorPath = new RootActorPath(this) val root: ActorPath = new RootActorPath(this)
val deadLetters = new DeadLetterActorRef(eventStream, root / "nul")
val deadLetterMailbox = new Mailbox(null) {
becomeClosed()
override def dispatcher = null //MessageDispatcher.this
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
override def dequeue() = null
override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) }
override def systemDrain(): SystemMessage = null
override def hasMessages = false
override def hasSystemMessages = false
override def numberOfMessages = 0
}
val scheduler = new DefaultScheduler(new HashedWheelTimer(log, Executors.defaultThreadFactory, 100, TimeUnit.MILLISECONDS, 512))
// TODO correctly pull its config from the config
val dispatcherFactory = new Dispatchers(AkkaConfig, eventStream, deadLetterMailbox, scheduler)
implicit val dispatcher = dispatcherFactory.defaultGlobalDispatcher
deadLetters.init(dispatcher)
// TODO think about memory consistency effects when doing funky stuff inside constructor // TODO think about memory consistency effects when doing funky stuff inside constructor
val provider: ActorRefProvider = { val provider: ActorRefProvider = {
val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match { val providerClass = ReflectiveAccess.getClassFor(ProviderClass) match {
@ -191,8 +205,6 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
def guardian: ActorRef = provider.guardian def guardian: ActorRef = provider.guardian
def systemGuardian: ActorRef = provider.systemGuardian def systemGuardian: ActorRef = provider.systemGuardian
def deathWatch: DeathWatch = provider.deathWatch def deathWatch: DeathWatch = provider.deathWatch
def deadLetters: ActorRef = provider.deadLetters
def deadLetterMailbox: Mailbox = provider.deadLetterMailbox
terminationFuture.onComplete(_ scheduler.stop()) terminationFuture.onComplete(_ scheduler.stop())
terminationFuture.onComplete(_ dispatcher.shutdown()) terminationFuture.onComplete(_ dispatcher.shutdown())

View file

@ -13,6 +13,8 @@ import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy,
import akka.actor._ import akka.actor._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.event.EventStream
import akka.actor.ActorSystem.AkkaConfig
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -61,12 +63,12 @@ case class Supervise(child: ActorRef) extends SystemMessage // sent to superviso
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.startsMonitoring
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.stopsMonitoring
final case class TaskInvocation(app: ActorSystem, function: () Unit, cleanup: () Unit) extends Runnable { final case class TaskInvocation(eventStream: EventStream, function: () Unit, cleanup: () Unit) extends Runnable {
def run() { def run() {
try { try {
function() function()
} catch { } catch {
case e app.eventStream.publish(Error(e, this, e.getMessage)) case e eventStream.publish(Error(e, this, e.getMessage))
} finally { } finally {
cleanup() cleanup()
} }
@ -84,7 +86,11 @@ object MessageDispatcher {
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
abstract class MessageDispatcher(val app: ActorSystem) extends Serializable { abstract class MessageDispatcher(
val deadLetterMailbox: Mailbox,
val eventStream: EventStream,
val scheduler: Scheduler) extends Serializable {
import MessageDispatcher._ import MessageDispatcher._
protected val _tasks = new AtomicLong(0L) protected val _tasks = new AtomicLong(0L)
@ -99,11 +105,6 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
*/ */
protected[akka] def createMailbox(actor: ActorCell): Mailbox protected[akka] def createMailbox(actor: ActorCell): Mailbox
/**
* a blackhole mailbox for the purpose of replacing the real one upon actor termination
*/
import app.deadLetterMailbox
/** /**
* Name of this dispatcher. * Name of this dispatcher.
*/ */
@ -133,7 +134,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
shutdownSchedule match { shutdownSchedule match {
case UNSCHEDULED case UNSCHEDULED
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
case SCHEDULED case SCHEDULED
shutdownSchedule = RESCHEDULED shutdownSchedule = RESCHEDULED
case RESCHEDULED //Already marked for reschedule case RESCHEDULED //Already marked for reschedule
@ -154,7 +155,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
_tasks.getAndIncrement() _tasks.getAndIncrement()
try { try {
startIfUnstarted() startIfUnstarted()
executeTask(TaskInvocation(app, block, taskCleanup)) executeTask(TaskInvocation(eventStream, block, taskCleanup))
} catch { } catch {
case e case e
_tasks.decrementAndGet _tasks.decrementAndGet
@ -170,7 +171,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
shutdownSchedule match { shutdownSchedule match {
case UNSCHEDULED case UNSCHEDULED
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
case SCHEDULED case SCHEDULED
shutdownSchedule = RESCHEDULED shutdownSchedule = RESCHEDULED
case RESCHEDULED //Already marked for reschedule case RESCHEDULED //Already marked for reschedule
@ -234,7 +235,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
shutdownSchedule match { shutdownSchedule match {
case RESCHEDULED case RESCHEDULED
shutdownSchedule = SCHEDULED shutdownSchedule = SCHEDULED
app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
case SCHEDULED case SCHEDULED
if (_tasks.get == 0) { if (_tasks.get == 0) {
active switchOff { active switchOff {
@ -329,19 +330,19 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
/** /**
* Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig * Trait to be used for hooking in new dispatchers into Dispatchers.fromConfig
*/ */
abstract class MessageDispatcherConfigurator(val app: ActorSystem) { abstract class MessageDispatcherConfigurator(val AkkaConfig: AkkaConfig, val eventStream: EventStream) {
/** /**
* Returns an instance of MessageDispatcher given a Configuration * Returns an instance of MessageDispatcher given a Configuration
*/ */
def configure(config: Configuration): MessageDispatcher def configure(config: Configuration): MessageDispatcher
def mailboxType(config: Configuration): MailboxType = { def mailboxType(config: Configuration): MailboxType = {
val capacity = config.getInt("mailbox-capacity", app.AkkaConfig.MailboxCapacity) val capacity = config.getInt("mailbox-capacity", AkkaConfig.MailboxCapacity)
if (capacity < 1) UnboundedMailbox() if (capacity < 1) UnboundedMailbox()
else { else {
val duration = Duration( val duration = Duration(
config.getInt("mailbox-push-timeout-time", app.AkkaConfig.MailboxPushTimeout.toMillis.toInt), config.getInt("mailbox-push-timeout-time", AkkaConfig.MailboxPushTimeout.toMillis.toInt),
app.AkkaConfig.DefaultTimeUnit) AkkaConfig.DefaultTimeUnit)
BoundedMailbox(capacity, duration) BoundedMailbox(capacity, duration)
} }
} }
@ -350,8 +351,8 @@ abstract class MessageDispatcherConfigurator(val app: ActorSystem) {
import ThreadPoolConfigDispatcherBuilder.conf_? import ThreadPoolConfigDispatcherBuilder.conf_?
//Apply the following options to the config if they are present in the config //Apply the following options to the config if they are present in the config
ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(app)).configure( ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(eventStream)).configure(
conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, app.AkkaConfig.DefaultTimeUnit))), conf_?(config getInt "keep-alive-time")(time _.setKeepAliveTime(Duration(time, AkkaConfig.DefaultTimeUnit))),
conf_?(config getDouble "core-pool-size-factor")(factor _.setCorePoolSizeFromFactor(factor)), conf_?(config getDouble "core-pool-size-factor")(factor _.setCorePoolSizeFromFactor(factor)),
conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)), conf_?(config getDouble "max-pool-size-factor")(factor _.setMaxPoolSizeFromFactor(factor)),
conf_?(config getInt "executor-bounds")(bounds _.setExecutorBounds(bounds)), conf_?(config getInt "executor-bounds")(bounds _.setExecutorBounds(bounds)),

View file

@ -10,6 +10,8 @@ import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, Concur
import java.util.{ Comparator, Queue } import java.util.{ Comparator, Queue }
import annotation.tailrec import annotation.tailrec
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
/** /**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@ -28,16 +30,16 @@ import akka.actor.ActorSystem
* @author Viktor Klang * @author Viktor Klang
*/ */
class BalancingDispatcher( class BalancingDispatcher(
_app: ActorSystem, _deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
_name: String, _name: String,
throughput: Int, throughput: Int,
throughputDeadlineTime: Int, throughputDeadlineTime: Int,
mailboxType: MailboxType, mailboxType: MailboxType,
config: ThreadPoolConfig, config: ThreadPoolConfig,
_timeoutMs: Long) _timeoutMs: Long)
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) { extends Dispatcher(_deadLetterMailbox, _eventStream, _scheduler, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
import app.deadLetterMailbox
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)

View file

@ -9,6 +9,8 @@ import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue }
import akka.actor.{ ActorCell, ActorKilledException } import akka.actor.{ ActorCell, ActorKilledException }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
/** /**
* Default settings are: * Default settings are:
@ -64,14 +66,16 @@ import akka.actor.ActorSystem
* Larger values (or zero or negative) increase throughput, smaller values increase fairness * Larger values (or zero or negative) increase throughput, smaller values increase fairness
*/ */
class Dispatcher( class Dispatcher(
_app: ActorSystem, _deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
val name: String, val name: String,
val throughput: Int, val throughput: Int,
val throughputDeadlineTime: Int, val throughputDeadlineTime: Int,
val mailboxType: MailboxType, val mailboxType: MailboxType,
executorServiceFactoryProvider: ExecutorServiceFactoryProvider, executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
val timeoutMs: Long) val timeoutMs: Long)
extends MessageDispatcher(_app) { extends MessageDispatcher(_deadLetterMailbox, _eventStream, _scheduler) {
protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name)
protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService))
@ -93,7 +97,7 @@ class Dispatcher(
executorService.get() execute invocation executorService.get() execute invocation
} catch { } catch {
case e: RejectedExecutionException case e: RejectedExecutionException
app.eventStream.publish(Warning(this, e.toString)) eventStream.publish(Warning(this, e.toString))
throw e throw e
} }
} }
@ -120,7 +124,7 @@ class Dispatcher(
} catch { } catch {
case e: RejectedExecutionException case e: RejectedExecutionException
try { try {
app.eventStream.publish(Warning(this, e.toString)) eventStream.publish(Warning(this, e.toString))
} finally { } finally {
mbox.setAsIdle() mbox.setAsIdle()
} }

View file

@ -10,6 +10,9 @@ import akka.util.{ Duration, ReflectiveAccess }
import akka.config.Configuration import akka.config.Configuration
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
import akka.actor.ActorSystem.AkkaConfig
/** /**
* Scala API. Dispatcher factory. * Scala API. Dispatcher factory.
@ -43,15 +46,20 @@ import akka.actor.ActorSystem
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class Dispatchers(val app: ActorSystem) { class Dispatchers(
val ThroughputDeadlineTimeMillis = app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt val AkkaConfig: ActorSystem.AkkaConfig,
val eventStream: EventStream,
val deadLetterMailbox: Mailbox,
val scheduler: Scheduler) {
val ThroughputDeadlineTimeMillis = AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt
val MailboxType: MailboxType = val MailboxType: MailboxType =
if (app.AkkaConfig.MailboxCapacity < 1) UnboundedMailbox() if (AkkaConfig.MailboxCapacity < 1) UnboundedMailbox()
else BoundedMailbox(app.AkkaConfig.MailboxCapacity, app.AkkaConfig.MailboxPushTimeout) else BoundedMailbox(AkkaConfig.MailboxCapacity, AkkaConfig.MailboxPushTimeout)
val DispatcherShutdownMillis = app.AkkaConfig.DispatcherDefaultShutdown.toMillis val DispatcherShutdownMillis = AkkaConfig.DispatcherDefaultShutdown.toMillis
lazy val defaultGlobalDispatcher = lazy val defaultGlobalDispatcher =
app.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build AkkaConfig.config.getSection("akka.actor.default-dispatcher").flatMap(from) getOrElse newDispatcher("AkkaDefaultGlobalDispatcher", 1, MailboxType).build
/** /**
* Creates an thread based dispatcher serving a single actor through the same single thread. * Creates an thread based dispatcher serving a single actor through the same single thread.
@ -60,8 +68,8 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(actor: LocalActorRef) = actor match { def newPinnedDispatcher(actor: LocalActorRef) = actor match {
case null new PinnedDispatcher(app, null, "anon", MailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, "anon", MailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, MailboxType, DispatcherShutdownMillis) case some new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, some.underlying, some.address, MailboxType, DispatcherShutdownMillis)
} }
/** /**
@ -71,8 +79,8 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match { def newPinnedDispatcher(actor: LocalActorRef, mailboxType: MailboxType) = actor match {
case null new PinnedDispatcher(app, null, "anon", mailboxType, DispatcherShutdownMillis) case null new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, "anon", mailboxType, DispatcherShutdownMillis)
case some new PinnedDispatcher(app, some.underlying, some.address, mailboxType, DispatcherShutdownMillis) case some new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, some.underlying, some.address, mailboxType, DispatcherShutdownMillis)
} }
/** /**
@ -81,7 +89,7 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(name: String, mailboxType: MailboxType) = def newPinnedDispatcher(name: String, mailboxType: MailboxType) =
new PinnedDispatcher(app, null, name, mailboxType, DispatcherShutdownMillis) new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, name, mailboxType, DispatcherShutdownMillis)
/** /**
* Creates an thread based dispatcher serving a single actor through the same single thread. * Creates an thread based dispatcher serving a single actor through the same single thread.
@ -89,7 +97,7 @@ class Dispatchers(val app: ActorSystem) {
* E.g. each actor consumes its own thread. * E.g. each actor consumes its own thread.
*/ */
def newPinnedDispatcher(name: String) = def newPinnedDispatcher(name: String) =
new PinnedDispatcher(app, null, name, MailboxType, DispatcherShutdownMillis) new PinnedDispatcher(deadLetterMailbox, eventStream, scheduler, null, name, MailboxType, DispatcherShutdownMillis)
/** /**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -97,8 +105,8 @@ class Dispatchers(val app: ActorSystem) {
* Has a fluent builder interface for configuring its semantics. * Has a fluent builder interface for configuring its semantics.
*/ */
def newDispatcher(name: String) = def newDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new Dispatcher(app, name, app.AkkaConfig.DispatcherThroughput, ThreadPoolConfigDispatcherBuilder(config new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -107,7 +115,7 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@ -116,7 +124,7 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) new Dispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -124,8 +132,8 @@ class Dispatchers(val app: ActorSystem) {
* Has a fluent builder interface for configuring its semantics. * Has a fluent builder interface for configuring its semantics.
*/ */
def newBalancingDispatcher(name: String) = def newBalancingDispatcher(name: String) =
ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(app, name, app.AkkaConfig.DispatcherThroughput, ThreadPoolConfigDispatcherBuilder(config new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, AkkaConfig.DispatcherThroughput,
ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -134,7 +142,7 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def newBalancingDispatcher(name: String, throughput: Int) = def newBalancingDispatcher(name: String, throughput: Int) =
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, MailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -143,7 +151,7 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, ThroughputDeadlineTimeMillis, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool.
@ -152,13 +160,13 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(app, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(app)) new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler, name, throughput, throughputDeadlineMs, mailboxType, config, DispatcherShutdownMillis), ThreadPoolConfig(eventStream))
/** /**
* Utility function that tries to load the specified dispatcher config from the akka.conf * Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher * or else use the supplied default dispatcher
*/ */
def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher = def fromConfig(key: String, default: MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
app.config getSection key flatMap from getOrElse default AkkaConfig.config getSection key flatMap from getOrElse default
/* /*
* Creates of obtains a dispatcher from a ConfigMap according to the format below * Creates of obtains a dispatcher from a ConfigMap according to the format below
@ -185,8 +193,8 @@ class Dispatchers(val app: ActorSystem) {
*/ */
def from(cfg: Configuration): Option[MessageDispatcher] = { def from(cfg: Configuration): Option[MessageDispatcher] = {
cfg.getString("type") flatMap { cfg.getString("type") flatMap {
case "Dispatcher" Some(new DispatcherConfigurator(app)) case "Dispatcher" Some(new DispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler))
case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(app)) case "BalancingDispatcher" Some(new BalancingDispatcherConfigurator(AkkaConfig, deadLetterMailbox, eventStream, scheduler))
case "GlobalDispatcher" None //TODO FIXME remove this case "GlobalDispatcher" None //TODO FIXME remove this
case fqn case fqn
ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match {
@ -206,26 +214,26 @@ class Dispatchers(val app: ActorSystem) {
} }
} }
class DispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { class DispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new Dispatcher(app, configureThreadPool(config, threadPoolConfig new Dispatcher(deadLetterMailbox, eventStream, scheduler,
config.getString("name", newUuid.toString), config.getString("name", newUuid.toString),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), config.getInt("throughput", AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config), mailboxType(config),
threadPoolConfig, threadPoolConfig,
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
} }
} }
class BalancingDispatcherConfigurator(app: ActorSystem) extends MessageDispatcherConfigurator(app) { class BalancingDispatcherConfigurator(AkkaConfig: AkkaConfig, deadLetterMailbox: Mailbox, eventStream: EventStream, scheduler: Scheduler) extends MessageDispatcherConfigurator(AkkaConfig, eventStream) {
def configure(config: Configuration): MessageDispatcher = { def configure(config: Configuration): MessageDispatcher = {
configureThreadPool(config, threadPoolConfig new BalancingDispatcher(app, configureThreadPool(config, threadPoolConfig new BalancingDispatcher(deadLetterMailbox, eventStream, scheduler,
config.getString("name", newUuid.toString), config.getString("name", newUuid.toString),
config.getInt("throughput", app.AkkaConfig.DispatcherThroughput), config.getInt("throughput", AkkaConfig.DispatcherThroughput),
config.getInt("throughput-deadline-time", app.AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt), config.getInt("throughput-deadline-time", AkkaConfig.DispatcherThroughputDeadlineTime.toMillis.toInt),
mailboxType(config), mailboxType(config),
threadPoolConfig, threadPoolConfig,
app.AkkaConfig.DispatcherDefaultShutdown.toMillis)).build AkkaConfig.DispatcherDefaultShutdown.toMillis)).build
} }
} }

View file

@ -262,7 +262,7 @@ object Future {
result completeWithResult currentValue result completeWithResult currentValue
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
result completeWithException e result completeWithException e
} finally { } finally {
results.clear results.clear
@ -631,7 +631,7 @@ sealed trait Future[+T] extends japi.Future[T] {
Right(f(res)) Right(f(res))
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
Left(e) Left(e)
}) })
} }
@ -683,7 +683,7 @@ sealed trait Future[+T] extends japi.Future[T] {
future.completeWith(f(r)) future.completeWith(f(r))
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
future complete Left(e) future complete Left(e)
} }
} }
@ -716,7 +716,7 @@ sealed trait Future[+T] extends japi.Future[T] {
if (p(res)) r else Left(new MatchError(res)) if (p(res)) r else Left(new MatchError(res))
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
Left(e) Left(e)
}) })
} }
@ -811,7 +811,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f) fr completeWith cont(f)
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
fr completeWithException e fr completeWithException e
} }
} }
@ -825,7 +825,7 @@ trait Promise[T] extends Future[T] {
fr completeWith cont(f) fr completeWith cont(f)
} catch { } catch {
case e: Exception case e: Exception
dispatcher.app.eventStream.publish(Error(e, this, e.getMessage)) dispatcher.eventStream.publish(Error(e, this, e.getMessage))
fr completeWithException e fr completeWithException e
} }
} }
@ -979,12 +979,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
val runnable = new Runnable { val runnable = new Runnable {
def run() { def run() {
if (!isCompleted) { if (!isCompleted) {
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) if (!isExpired) dispatcher.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
else func(DefaultPromise.this) else func(DefaultPromise.this)
} }
} }
} }
val timeoutFuture = dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) val timeoutFuture = dispatcher.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
onComplete(_ timeoutFuture.cancel()) onComplete(_ timeoutFuture.cancel())
false false
} else true } else true
@ -1006,18 +1006,18 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi
val runnable = new Runnable { val runnable = new Runnable {
def run() { def run() {
if (!isCompleted) { if (!isCompleted) {
if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) if (!isExpired) dispatcher.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS)
else promise complete (try { Right(fallback) } catch { case e Left(e) }) else promise complete (try { Right(fallback) } catch { case e Left(e) })
} }
} }
} }
dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) dispatcher.scheduler.scheduleOnce(runnable, timeLeft(), NANOS)
promise promise
} }
} else this } else this
private def notifyCompleted(func: Future[T] Unit) { private def notifyCompleted(func: Future[T] Unit) {
try { func(this) } catch { case e dispatcher.app.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really? try { func(this) } catch { case e dispatcher.eventStream.publish(Error(e, this, "Future onComplete-callback raised an exception")) } //TODO catch, everything? Really?
} }
@inline @inline

View file

@ -7,14 +7,23 @@ package akka.dispatch
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.EventStream
import akka.actor.Scheduler
/** /**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mailboxType: MailboxType, _timeoutMs: Long) class PinnedDispatcher(
extends Dispatcher(_app, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_app), _timeoutMs) { _deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
_actor: ActorCell,
_name: String,
_mailboxType: MailboxType,
_timeoutMs: Long)
extends Dispatcher(_deadLetterMailbox, _eventStream, _scheduler, _name, Int.MaxValue, -1, _mailboxType, PinnedDispatcher.oneThread(_eventStream), _timeoutMs) {
@volatile @volatile
protected[akka] var owner: ActorCell = _actor protected[akka] var owner: ActorCell = _actor
@ -34,6 +43,6 @@ class PinnedDispatcher(_app: ActorSystem, _actor: ActorCell, _name: String, _mai
} }
object PinnedDispatcher { object PinnedDispatcher {
def oneThread(app: ActorSystem): ThreadPoolConfig = ThreadPoolConfig(app, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) def oneThread(eventStream: EventStream): ThreadPoolConfig = ThreadPoolConfig(eventStream, allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
} }

View file

@ -11,6 +11,7 @@ import ThreadPoolExecutor.CallerRunsPolicy
import akka.util.Duration import akka.util.Duration
import akka.event.Logging.{ Warning, Error } import akka.event.Logging.{ Warning, Error }
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.event.EventStream
object ThreadPoolConfig { object ThreadPoolConfig {
type Bounds = Int type Bounds = Int
@ -68,7 +69,7 @@ trait ExecutorServiceFactoryProvider {
/** /**
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher * A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/ */
case class ThreadPoolConfig(app: ActorSystem, case class ThreadPoolConfig(eventStream: EventStream,
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout, allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
@ -86,7 +87,7 @@ case class ThreadPoolConfig(app: ActorSystem,
case Right(bounds) case Right(bounds)
val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory) val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory)
service.allowCoreThreadTimeOut(allowCorePoolTimeout) service.allowCoreThreadTimeOut(allowCorePoolTimeout)
new BoundedExecutorDecorator(app, service, bounds) new BoundedExecutorDecorator(eventStream, service, bounds)
} }
} }
} }
@ -210,7 +211,7 @@ class MonitorableThread(runnable: Runnable, name: String)
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
class BoundedExecutorDecorator(val app: ActorSystem, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate { class BoundedExecutorDecorator(val eventStream: EventStream, val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate {
protected val semaphore = new Semaphore(bound) protected val semaphore = new Semaphore(bound)
override def execute(command: Runnable) = { override def execute(command: Runnable) = {
@ -227,10 +228,10 @@ class BoundedExecutorDecorator(val app: ActorSystem, val executor: ExecutorServi
}) })
} catch { } catch {
case e: RejectedExecutionException case e: RejectedExecutionException
app.eventStream.publish(Warning(this, e.toString)) eventStream.publish(Warning(this, e.toString))
semaphore.release semaphore.release
case e: Throwable case e: Throwable
app.eventStream.publish(Error(e, this, e.getMessage)) eventStream.publish(Error(e, this, e.getMessage))
throw e throw e
} }
} }

View file

@ -41,8 +41,6 @@ class RemoteActorRefProvider(
import akka.dispatch.Promise import akka.dispatch.Promise
val local = new LocalActorRefProvider(app, root, eventStream, dispatcher, scheduler) val local = new LocalActorRefProvider(app, root, eventStream, dispatcher, scheduler)
def deadLetterMailbox = local.deadLetterMailbox
def deadLetters = local.deadLetters
def deathWatch = local.deathWatch def deathWatch = local.deathWatch
def guardian = local.guardian def guardian = local.guardian
def systemGuardian = local.systemGuardian def systemGuardian = local.systemGuardian

View file

@ -151,7 +151,7 @@ class Agent[T](initialValue: T, app: ActorSystem) {
def sendOff(f: T T): Unit = { def sendOff(f: T T): Unit = {
send((value: T) { send((value: T) {
suspend() suspend()
val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis)
val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
threadBased ! Update(f) threadBased ! Update(f)
value value
@ -169,7 +169,7 @@ class Agent[T](initialValue: T, app: ActorSystem) {
val result = new DefaultPromise[T](timeout)(app.dispatcher) val result = new DefaultPromise[T](timeout)(app.dispatcher)
send((value: T) { send((value: T) {
suspend() suspend()
val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) val pinnedDispatcher = new PinnedDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis)
val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher))
result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]] result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]]
value value

View file

@ -12,6 +12,8 @@ import java.lang.ref.WeakReference
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.actor.{ ActorCell, ActorRef, ActorSystem } import akka.actor.{ ActorCell, ActorRef, ActorSystem }
import akka.dispatch._ import akka.dispatch._
import akka.actor.Scheduler
import akka.event.EventStream
/* /*
* Locking rules: * Locking rules:
@ -103,7 +105,11 @@ private[testkit] object CallingThreadDispatcher {
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.1 * @since 1.1
*/ */
class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thread") extends MessageDispatcher(_app) { class CallingThreadDispatcher(
_deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
val name: String = "calling-thread") extends MessageDispatcher(_deadLetterMailbox, _eventStream, _scheduler) {
import CallingThreadDispatcher._ import CallingThreadDispatcher._
protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor) protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor)
@ -213,12 +219,12 @@ class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thr
true true
} catch { } catch {
case ie: InterruptedException case ie: InterruptedException
app.eventStream.publish(Error(this, ie)) eventStream.publish(Error(this, ie))
Thread.currentThread().interrupt() Thread.currentThread().interrupt()
intex = ie intex = ie
true true
case e case e
app.eventStream.publish(Error(this, e)) eventStream.publish(Error(this, e))
queue.leave queue.leave
false false
} }

View file

@ -10,6 +10,8 @@ import com.eaio.uuid.UUID
import akka.actor.Props._ import akka.actor.Props._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import akka.dispatch.Mailbox
import akka.event.EventStream
/** /**
* This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it
@ -19,8 +21,15 @@ import java.util.concurrent.atomic.AtomicLong
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.1 * @since 1.1
*/ */
class TestActorRef[T <: Actor](_app: ActorSystem, _props: Props, _supervisor: ActorRef, name: String) class TestActorRef[T <: Actor](
extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_app)), _supervisor, _supervisor.path / name, false) { _app: ActorSystem,
_deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
_props: Props,
_supervisor: ActorRef,
name: String)
extends LocalActorRef(_app, _props.withDispatcher(new CallingThreadDispatcher(_deadLetterMailbox, _eventStream, _scheduler)), _supervisor, _supervisor.path / name, false) {
/** /**
* Directly inject messages into actor receive behavior. Any exceptions * Directly inject messages into actor receive behavior. Any exceptions
* thrown will be available to you, while still being able to use * thrown will be available to you, while still being able to use
@ -57,7 +66,7 @@ object TestActorRef {
def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, app.guardian, name) def apply[T <: Actor](props: Props, name: String)(implicit app: ActorSystem): TestActorRef[T] = apply[T](props, app.guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = { def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit app: ActorSystem): TestActorRef[T] = {
new TestActorRef(app, props, supervisor, name) new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, props, supervisor, name)
} }
def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName) def apply[T <: Actor](implicit m: Manifest[T], app: ActorSystem): TestActorRef[T] = apply[T](randomName)

View file

@ -8,6 +8,8 @@ import akka.actor._
import akka.util._ import akka.util._
import com.eaio.uuid.UUID import com.eaio.uuid.UUID
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.dispatch.Mailbox
import akka.event.EventStream
/** /**
* This is a specialised form of the TestActorRef with support for querying and * This is a specialised form of the TestActorRef with support for querying and
@ -34,8 +36,15 @@ import akka.actor.ActorSystem
* @author Roland Kuhn * @author Roland Kuhn
* @since 1.2 * @since 1.2
*/ */
class TestFSMRef[S, D, T <: Actor](app: ActorSystem, props: Props, supervisor: ActorRef, name: String)(implicit ev: T <:< FSM[S, D]) class TestFSMRef[S, D, T <: Actor](
extends TestActorRef(app, props, supervisor, name) { app: ActorSystem,
_deadLetterMailbox: Mailbox,
_eventStream: EventStream,
_scheduler: Scheduler,
props: Props,
supervisor: ActorRef,
name: String)(implicit ev: T <:< FSM[S, D])
extends TestActorRef(app, _deadLetterMailbox, _eventStream, _scheduler, props, supervisor, name) {
private def fsm: T = underlyingActor private def fsm: T = underlyingActor
@ -81,8 +90,8 @@ class TestFSMRef[S, D, T <: Actor](app: ActorSystem, props: Props, supervisor: A
object TestFSMRef { object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
new TestFSMRef(app, Props(creator = () factory), app.guardian, TestActorRef.randomName) new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, TestActorRef.randomName)
def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] = def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], app: ActorSystem): TestFSMRef[S, D, T] =
new TestFSMRef(app, Props(creator = () factory), app.guardian, name) new TestFSMRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(creator = () factory), app.guardian, name)
} }

View file

@ -91,7 +91,8 @@ class TestKit(_app: ActorSystem) {
* ActorRef of the test actor. Access is provided to enable e.g. * ActorRef of the test actor. Access is provided to enable e.g.
* registration as message target. * registration as message target.
*/ */
val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue)).copy(dispatcher = new CallingThreadDispatcher(app)), val testActor: ActorRef = app.systemActorOf(Props(new TestActor(queue))
.copy(dispatcher = new CallingThreadDispatcher(app.deadLetterMailbox, app.eventStream, app.scheduler)),
"testActor" + TestKit.testActorId.incrementAndGet) "testActor" + TestKit.testActorId.incrementAndGet)
private var end: Duration = Duration.Undefined private var end: Duration = Duration.Undefined

View file

@ -171,7 +171,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach {
val boss = TestActorRef(Props(new TActor { val boss = TestActorRef(Props(new TActor {
val ref = new TestActorRef(app, Props(new TActor { val ref = new TestActorRef(app, app.deadLetterMailbox, app.eventStream, app.scheduler, Props(new TActor {
def receiveT = { case _ } def receiveT = { case _ }
override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 } override def preRestart(reason: Throwable, msg: Option[Any]) { counter -= 1 }
override def postRestart(reason: Throwable) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 }