Enforce mailbox types on System actors. See #3273

This commit is contained in:
Björn Antonsson 2013-04-26 12:18:01 +02:00
parent d6ff7166d7
commit 539df2e98a
27 changed files with 123 additions and 49 deletions

View file

@ -102,7 +102,7 @@ class ActorMailboxSpec extends AkkaSpec(ActorMailboxSpec.mailboxConf) with Defau
actor ! "ping" actor ! "ping"
val q = expectMsgType[MessageQueue] val q = expectMsgType[MessageQueue]
types foreach (t assert(t isInstance q, s"Type [${q.getClass}] is not assignable to [${t}]")) types foreach (t assert(t isInstance q, s"Type [${q.getClass.getName}] is not assignable to [${t.getName}]"))
} }
"An Actor" must { "An Actor" must {

View file

@ -319,7 +319,15 @@ akka {
# an instance of that actor its mailbox type will be decided by looking # an instance of that actor its mailbox type will be decided by looking
# up a mailbox configuration via T in this mapping # up a mailbox configuration via T in this mapping
requirements { requirements {
"akka.dispatch.UnboundedMessageQueueSemantics" = akka.actor.mailbox.unbounded-queue-based
"akka.dispatch.DequeBasedMessageQueue" = akka.actor.mailbox.unbounded-deque-based "akka.dispatch.DequeBasedMessageQueue" = akka.actor.mailbox.unbounded-deque-based
"akka.dispatch.UnboundedDequeBasedMessageQueueSemantics" = akka.actor.mailbox.unbounded-deque-based
}
unbounded-queue-based {
# FQCN of the MailboxType, The Class of the FQCN must have a public constructor
# with (akka.actor.ActorSystem.Settings, com.typesafe.config.Config) parameters.
mailbox-type = "akka.dispatch.UnboundedMailbox"
} }
unbounded-deque-based { unbounded-deque-based {

View file

@ -6,7 +6,7 @@ package akka.actor
import scala.collection.immutable import scala.collection.immutable
import akka.dispatch.sysmsg._ import akka.dispatch.sysmsg._
import akka.dispatch.NullMessage import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue, NullMessage }
import akka.routing._ import akka.routing._
import akka.event._ import akka.event._
import akka.util.{ Switch, Helpers } import akka.util.{ Switch, Helpers }
@ -367,7 +367,8 @@ private[akka] object LocalActorRefProvider {
/* /*
* Root and user guardian * Root and user guardian
*/ */
private class Guardian(override val supervisorStrategy: SupervisorStrategy) extends Actor { private class Guardian(override val supervisorStrategy: SupervisorStrategy) extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
def receive = { def receive = {
case Terminated(_) context.stop(self) case Terminated(_) context.stop(self)
@ -382,7 +383,8 @@ private[akka] object LocalActorRefProvider {
/** /**
* System guardian * System guardian
*/ */
private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy, val guardian: ActorRef) extends Actor { private class SystemGuardian(override val supervisorStrategy: SupervisorStrategy, val guardian: ActorRef)
extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import SystemGuardian._ import SystemGuardian._
var terminationHooks = Set.empty[ActorRef] var terminationHooks = Set.empty[ActorRef]

View file

@ -3,7 +3,7 @@
*/ */
package akka.actor package akka.actor
import akka.dispatch.{ RequiresMessageQueue, Envelope, DequeBasedMessageQueue } import akka.dispatch.{ UnboundedDequeBasedMessageQueueSemantics, RequiresMessageQueue, Envelope, DequeBasedMessageQueue }
import akka.AkkaException import akka.AkkaException
/** /**
@ -47,8 +47,14 @@ import akka.AkkaException
* any trait/class that overrides the `preRestart` callback. This means it's not possible to write * any trait/class that overrides the `preRestart` callback. This means it's not possible to write
* `Actor with MyActor with Stash` if `MyActor` overrides `preRestart`. * `Actor with MyActor with Stash` if `MyActor` overrides `preRestart`.
*/ */
trait Stash extends Actor with RequiresMessageQueue[DequeBasedMessageQueue] { trait Stash extends UnrestrictedStash with RequiresMessageQueue[DequeBasedMessageQueue]
/**
* The `UnboundedStash` trait is a version of `Stash` that enforces an unbounded stash for you actor.
*/
trait UnboundedStash extends UnrestrictedStash with RequiresMessageQueue[UnboundedDequeBasedMessageQueueSemantics]
trait UnrestrictedStash extends Actor {
/* The private stash of the actor. It is only accessible using `stash()` and /* The private stash of the actor. It is only accessible using `stash()` and
* `unstashAll()`. * `unstashAll()`.
*/ */
@ -68,12 +74,12 @@ trait Stash extends Actor with RequiresMessageQueue[DequeBasedMessageQueue] {
private val mailbox: DequeBasedMessageQueue = { private val mailbox: DequeBasedMessageQueue = {
context.asInstanceOf[ActorCell].mailbox.messageQueue match { context.asInstanceOf[ActorCell].mailbox.messageQueue match {
case queue: DequeBasedMessageQueue queue case queue: DequeBasedMessageQueue queue
case other throw ActorInitializationException(self, "DequeBasedMailbox required, got: " + other.getClass() + """ case other throw ActorInitializationException(self, s"DequeBasedMailbox required, got: ${other.getClass.getName}\n" +
An (unbounded) deque-based mailbox can be configured as follows: """An (unbounded) deque-based mailbox can be configured as follows:
my-custom-dispatcher { | my-custom-mailbox {
mailbox-type = "akka.dispatch.UnboundedDequeBasedMailbox" | mailbox-type = "akka.dispatch.UnboundedDequeBasedMailbox"
} | }
""") |""".stripMargin)
} }
} }

View file

@ -31,3 +31,8 @@ package akka.actor
* </pre> * </pre>
*/ */
abstract class UntypedActorWithStash extends UntypedActor with Stash abstract class UntypedActorWithStash extends UntypedActor with Stash
/**
* Actor base class that enforces an unbounded stash for the actor.
*/
abstract class UntypedActorWithUnboundedStash extends UntypedActor with UnboundedStash

View file

@ -19,6 +19,7 @@ import akka.io.IO.HasFailureMessage
import akka.util.Helpers.Requiring import akka.util.Helpers.Requiring
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.util.SerializedSuspendableExecutionContext import akka.util.SerializedSuspendableExecutionContext
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
abstract class SelectionHandlerSettings(config: Config) { abstract class SelectionHandlerSettings(config: Config) {
import config._ import config._
@ -56,7 +57,8 @@ private[io] object SelectionHandler {
case object WriteInterest case object WriteInterest
} }
private[io] class SelectionHandler(settings: SelectionHandlerSettings) extends Actor with ActorLogging { private[io] class SelectionHandler(settings: SelectionHandlerSettings) extends Actor with ActorLogging
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import SelectionHandler._ import SelectionHandler._
import settings._ import settings._

View file

@ -17,14 +17,17 @@ import akka.util.ByteString
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.Tcp._ import akka.io.Tcp._
import akka.io.SelectionHandler._ import akka.io.SelectionHandler._
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* Base class for TcpIncomingConnection and TcpOutgoingConnection. * Base class for TcpIncomingConnection and TcpOutgoingConnection.
* *
* INTERNAL API * INTERNAL API
*/ */
private[io] abstract class TcpConnection(val channel: SocketChannel, private[io] abstract class TcpConnection(
val tcp: TcpExt) extends Actor with ActorLogging { val channel: SocketChannel,
val tcp: TcpExt)
extends Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import tcp.Settings._ import tcp.Settings._
import tcp.bufferPool import tcp.bufferPool
import TcpConnection._ import TcpConnection._

View file

@ -12,6 +12,7 @@ import akka.io.SelectionHandler._
import akka.io.Tcp._ import akka.io.Tcp._
import akka.io.IO.HasFailureMessage import akka.io.IO.HasFailureMessage
import java.net.InetSocketAddress import java.net.InetSocketAddress
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
@ -29,10 +30,12 @@ private[io] object TcpListener {
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class TcpListener(val selectorRouter: ActorRef, private[io] class TcpListener(
val selectorRouter: ActorRef,
val tcp: TcpExt, val tcp: TcpExt,
val bindCommander: ActorRef, val bindCommander: ActorRef,
val bind: Bind) extends Actor with ActorLogging { val bind: Bind)
extends Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import TcpListener._ import TcpListener._
import tcp.Settings._ import tcp.Settings._
import bind._ import bind._

View file

@ -13,6 +13,7 @@ import scala.util.Failure
import akka.actor.Terminated import akka.actor.Terminated
import akka.actor.Props import akka.actor.Props
import akka.util.ByteString import akka.util.ByteString
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
object TcpPipelineHandler { object TcpPipelineHandler {
@ -87,7 +88,7 @@ class TcpPipelineHandler[Ctx <: PipelineContext, Cmd, Evt](
init: TcpPipelineHandler.Init[Ctx, Cmd, Evt], init: TcpPipelineHandler.Init[Ctx, Cmd, Evt],
connection: ActorRef, connection: ActorRef,
handler: ActorRef) handler: ActorRef)
extends Actor { extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import init._ import init._
import TcpPipelineHandler._ import TcpPipelineHandler._

View file

@ -12,13 +12,16 @@ import java.nio.channels.DatagramChannel
import java.nio.channels.SelectionKey._ import java.nio.channels.SelectionKey._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpConnection(val udpConn: UdpConnectedExt, private[io] class UdpConnection(
val udpConn: UdpConnectedExt,
val commander: ActorRef, val commander: ActorRef,
val connect: Connect) extends Actor with ActorLogging { val connect: Connect)
extends Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
def selector: ActorRef = context.parent def selector: ActorRef = context.parent

View file

@ -13,14 +13,16 @@ import java.nio.channels.DatagramChannel
import java.nio.channels.SelectionKey._ import java.nio.channels.SelectionKey._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpListener(val udp: UdpExt, private[io] class UdpListener(
val udp: UdpExt,
val bindCommander: ActorRef, val bindCommander: ActorRef,
val bind: Bind) val bind: Bind)
extends Actor with ActorLogging with WithUdpSend { extends Actor with ActorLogging with WithUdpSend with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import bind._ import bind._
import udp.bufferPool import udp.bufferPool

View file

@ -10,12 +10,16 @@ import akka.io.SelectionHandler.{ ChannelRegistered, RegisterChannel }
import scala.collection.immutable import scala.collection.immutable
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpSender(val udp: UdpExt, options: immutable.Traversable[SocketOption], val commander: ActorRef) private[io] class UdpSender(
extends Actor with ActorLogging with WithUdpSend { val udp: UdpExt,
options: immutable.Traversable[SocketOption],
val commander: ActorRef)
extends Actor with ActorLogging with WithUdpSend with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
def selector: ActorRef = context.parent def selector: ActorRef = context.parent

View file

@ -16,6 +16,7 @@ import akka.actor.SupervisorStrategy.Stop
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.cluster.ClusterEvent._ import akka.cluster.ClusterEvent._
import akka.actor.ActorSelection import akka.actor.ActorSelection
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* Base trait for all cluster messages. All ClusterMessage's are serializable. * Base trait for all cluster messages. All ClusterMessage's are serializable.
@ -175,7 +176,8 @@ private[cluster] object ClusterLeaderAction {
* *
* Supervisor managing the different Cluster daemons. * Supervisor managing the different Cluster daemons.
*/ */
private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Actor with ActorLogging { private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Actor with ActorLogging
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import InternalClusterAction._ import InternalClusterAction._
// Important - don't use Cluster(context.system) here because that would // Important - don't use Cluster(context.system) here because that would
// cause deadlock. The Cluster extension is currently being created and is waiting // cause deadlock. The Cluster extension is currently being created and is waiting
@ -206,7 +208,8 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac
* ClusterCoreDaemon and ClusterDomainEventPublisher can't be restarted because the state * ClusterCoreDaemon and ClusterDomainEventPublisher can't be restarted because the state
* would be obsolete. Shutdown the member if any those actors crashed. * would be obsolete. Shutdown the member if any those actors crashed.
*/ */
private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLogging { private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLogging
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import InternalClusterAction._ import InternalClusterAction._
val publisher = context.actorOf(Props[ClusterDomainEventPublisher]. val publisher = context.actorOf(Props[ClusterDomainEventPublisher].
@ -234,7 +237,8 @@ private[cluster] final class ClusterCoreSupervisor extends Actor with ActorLoggi
/** /**
* INTERNAL API. * INTERNAL API.
*/ */
private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Actor with ActorLogging { private[cluster] final class ClusterCoreDaemon(publisher: ActorRef) extends Actor with ActorLogging
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import ClusterLeaderAction._ import ClusterLeaderAction._
import InternalClusterAction._ import InternalClusterAction._

View file

@ -11,6 +11,7 @@ import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.event.EventStream import akka.event.EventStream
import akka.actor.AddressTerminated import akka.actor.AddressTerminated
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* Domain events published to the event bus. * Domain events published to the event bus.
@ -252,7 +253,8 @@ object ClusterEvent {
* Responsible for domain event subscriptions and publishing of * Responsible for domain event subscriptions and publishing of
* domain events to event bus. * domain events to event bus.
*/ */
private[cluster] final class ClusterDomainEventPublisher extends Actor with ActorLogging { private[cluster] final class ClusterDomainEventPublisher extends Actor with ActorLogging
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import InternalClusterAction._ import InternalClusterAction._
var latestGossip: Gossip = Gossip.empty var latestGossip: Gossip = Gossip.empty

View file

@ -9,6 +9,7 @@ import scala.collection.immutable
import akka.actor.{ Actor, ActorRef, ActorSystemImpl, Address, Props } import akka.actor.{ Actor, ActorRef, ActorSystemImpl, Address, Props }
import akka.cluster.ClusterEvent._ import akka.cluster.ClusterEvent._
import akka.actor.PoisonPill import akka.actor.PoisonPill
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
@ -40,7 +41,7 @@ private[akka] class ClusterReadView(cluster: Cluster) extends Closeable {
// create actor that subscribes to the cluster eventBus to update current read view state // create actor that subscribes to the cluster eventBus to update current read view state
private val eventBusListener: ActorRef = { private val eventBusListener: ActorRef = {
cluster.system.asInstanceOf[ActorSystemImpl].systemActorOf(Props(new Actor { cluster.system.asInstanceOf[ActorSystemImpl].systemActorOf(Props(new Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterDomainEvent]) override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterDomainEvent])
override def postStop(): Unit = cluster.unsubscribe(self) override def postStop(): Unit = cluster.unsubscribe(self)

View file

@ -733,6 +733,12 @@ actor's state which have the same property. The :class:`Stash` traits
implementation of :meth:`preRestart` will call ``unstashAll()``, which is implementation of :meth:`preRestart` will call ``unstashAll()``, which is
usually the desired behavior. usually the desired behavior.
.. note::
If you want to enforce that your actor can only work with an unbounded stash,
then you should use the ``UntypedActorWithUnboundedStash`` class instead.
.. _killing-actors-java: .. _killing-actors-java:
Killing an Actor Killing an Actor

View file

@ -837,6 +837,12 @@ actor's state which have the same property. The :class:`Stash` traits
implementation of :meth:`preRestart` will call ``unstashAll()``, which is implementation of :meth:`preRestart` will call ``unstashAll()``, which is
usually the desired behavior. usually the desired behavior.
.. note::
If you want to enforce that your actor can only work with an unbounded stash,
then you should use the ``UnboundedStash`` trait instead.
.. _killing-actors-scala: .. _killing-actors-scala:
Killing an Actor Killing an Actor

View file

@ -19,6 +19,7 @@ import akka.pattern.{ ask, pipe, AskTimeoutException }
import akka.event.{ LoggingAdapter, Logging } import akka.event.{ LoggingAdapter, Logging }
import java.net.{ InetSocketAddress, ConnectException } import java.net.{ InetSocketAddress, ConnectException }
import akka.remote.transport.ThrottlerTransportAdapter.{ SetThrottle, TokenBucket, Blackhole, Unthrottled } import akka.remote.transport.ThrottlerTransportAdapter.{ SetThrottle, TokenBucket, Blackhole, Unthrottled }
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* The Player is the client component of the * The Player is the client component of the
@ -49,7 +50,7 @@ trait Player { this: TestConductorExt ⇒
if (_client ne null) throw new IllegalStateException("TestConductorClient already started") if (_client ne null) throw new IllegalStateException("TestConductorClient already started")
_client = system.actorOf(Props(classOf[ClientFSM], name, controllerAddr), "TestConductorClient") _client = system.actorOf(Props(classOf[ClientFSM], name, controllerAddr), "TestConductorClient")
val a = system.actorOf(Props(new Actor { val a = system.actorOf(Props(new Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
var waiting: ActorRef = _ var waiting: ActorRef = _
def receive = { def receive = {
case fsm: ActorRef case fsm: ActorRef
@ -140,7 +141,8 @@ private[akka] object ClientFSM {
* *
* INTERNAL API. * INTERNAL API.
*/ */
private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress) extends Actor with LoggingFSM[ClientFSM.State, ClientFSM.Data] { private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress) extends Actor
with LoggingFSM[ClientFSM.State, ClientFSM.Data] with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import ClientFSM._ import ClientFSM._
val settings = TestConductor().Settings val settings = TestConductor().Settings

View file

@ -390,7 +390,9 @@ private[remote] class EndpointWriter(
codec: AkkaPduCodec, codec: AkkaPduCodec,
val refuseUid: Option[Int], val refuseUid: Option[Int],
val receiveBuffers: ConcurrentHashMap[Link, AckedReceiveBuffer[Message]], val receiveBuffers: ConcurrentHashMap[Link, AckedReceiveBuffer[Message]],
val reliableDeliverySupervisor: Option[ActorRef]) extends EndpointActor(localAddress, remoteAddress, transport, settings, codec) with Stash with FSM[EndpointWriter.State, Unit] { val reliableDeliverySupervisor: Option[ActorRef])
extends EndpointActor(localAddress, remoteAddress, transport, settings, codec) with UnboundedStash
with FSM[EndpointWriter.State, Unit] {
import EndpointWriter._ import EndpointWriter._
import context.dispatcher import context.dispatcher

View file

@ -17,6 +17,7 @@ import scala.concurrent.{ ExecutionContext, Future }
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.ConfigurationException import akka.ConfigurationException
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
/** /**
* INTERNAL API * INTERNAL API
@ -31,7 +32,8 @@ private[akka] object RemoteActorRefProvider {
case object WaitTransportShutdown extends TerminatorState case object WaitTransportShutdown extends TerminatorState
case object Finished extends TerminatorState case object Finished extends TerminatorState
private class RemotingTerminator(systemGuardian: ActorRef) extends Actor with FSM[TerminatorState, Option[Internals]] { private class RemotingTerminator(systemGuardian: ActorRef) extends Actor with FSM[TerminatorState, Option[Internals]]
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import context.dispatcher import context.dispatcher
startWith(Uninitialized, None) startWith(Uninitialized, None)

View file

@ -9,6 +9,7 @@ import akka.actor.Terminated
import akka.actor.Actor import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.dispatch.sysmsg.DeathWatchNotification import akka.dispatch.sysmsg.DeathWatchNotification
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
@ -23,7 +24,7 @@ private[akka] object RemoteDeploymentWatcher {
* Responsible for cleaning up child references of remote deployed actors when remote node * Responsible for cleaning up child references of remote deployed actors when remote node
* goes down (jvm crash, network failure), i.e. triggered by [[akka.actor.AddressTerminated]]. * goes down (jvm crash, network failure), i.e. triggered by [[akka.actor.AddressTerminated]].
*/ */
private[akka] class RemoteDeploymentWatcher extends Actor { private[akka] class RemoteDeploymentWatcher extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import RemoteDeploymentWatcher._ import RemoteDeploymentWatcher._
var supervisors = Map.empty[ActorRef, InternalActorRef] var supervisors = Map.empty[ActorRef, InternalActorRef]

View file

@ -14,6 +14,7 @@ import akka.actor.RootActorPath
import akka.actor.Terminated import akka.actor.Terminated
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.ConfigurationException import akka.ConfigurationException
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* INTERNAL API * INTERNAL API
@ -92,7 +93,7 @@ private[akka] class RemoteWatcher(
unreachableReaperInterval: FiniteDuration, unreachableReaperInterval: FiniteDuration,
heartbeatExpectedResponseAfter: FiniteDuration, heartbeatExpectedResponseAfter: FiniteDuration,
numberOfEndHeartbeatRequests: Int) numberOfEndHeartbeatRequests: Int)
extends Actor with ActorLogging { extends Actor with ActorLogging with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import RemoteWatcher._ import RemoteWatcher._
import context.dispatcher import context.dispatcher

View file

@ -23,6 +23,7 @@ import scala.util.control.NonFatal
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.remote.transport.AkkaPduCodec.Message import akka.remote.transport.AkkaPduCodec.Message
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
/** /**
* INTERNAL API * INTERNAL API
@ -82,7 +83,7 @@ private[remote] object Remoting {
case class RegisterTransportActor(props: Props, name: String) case class RegisterTransportActor(props: Props, name: String)
private[Remoting] class TransportSupervisor extends Actor { private[Remoting] class TransportSupervisor extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
override def supervisorStrategy = OneForOneStrategy() { override def supervisorStrategy = OneForOneStrategy() {
case NonFatal(e) Restart case NonFatal(e) Restart
} }
@ -348,7 +349,8 @@ private[remote] object EndpointManager {
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends Actor { private[remote] class EndpointManager(conf: Config, log: LoggingAdapter) extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import EndpointManager._ import EndpointManager._
import context.dispatcher import context.dispatcher

View file

@ -12,7 +12,7 @@ import akka.util.Timeout
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.{ ExecutionContext, Promise, Future } import scala.concurrent.{ ExecutionContext, Promise, Future }
import scala.util.Success import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
trait TransportAdapterProvider { trait TransportAdapterProvider {
/** /**
@ -162,7 +162,8 @@ abstract class ActorTransportAdapter(wrappedTransport: Transport, system: ActorS
override def shutdown(): Unit = manager ! PoisonPill override def shutdown(): Unit = manager ! PoisonPill
} }
abstract class ActorTransportAdapterManager extends Actor { abstract class ActorTransportAdapterManager extends Actor
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import ActorTransportAdapter.{ ListenUnderlying, ListenerRegistered } import ActorTransportAdapter.{ ListenUnderlying, ListenerRegistered }
private var delayedEvents = immutable.Queue.empty[Any] private var delayedEvents = immutable.Queue.empty[Any]

View file

@ -22,6 +22,7 @@ import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.{ Future, Promise } import scala.concurrent.{ Future, Promise }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
@SerialVersionUID(1L) @SerialVersionUID(1L)
class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException(msg, cause) with OnlyCauseStackTrace { class AkkaProtocolException(msg: String, cause: Throwable) extends AkkaException(msg, cause) with OnlyCauseStackTrace {
@ -227,7 +228,8 @@ private[transport] class ProtocolStateActor(initialData: InitialProtocolStateDat
private val settings: AkkaProtocolSettings, private val settings: AkkaProtocolSettings,
private val codec: AkkaPduCodec, private val codec: AkkaPduCodec,
private val failureDetector: FailureDetector) private val failureDetector: FailureDetector)
extends Actor with FSM[AssociationState, ProtocolStateData] { extends Actor with FSM[AssociationState, ProtocolStateData]
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import ProtocolStateActor._ import ProtocolStateActor._
import context.dispatcher import context.dispatcher

View file

@ -22,6 +22,7 @@ import scala.math.min
import scala.util.{ Success, Failure } import scala.util.{ Success, Failure }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.sysmsg.{ Unwatch, Watch } import akka.dispatch.sysmsg.{ Unwatch, Watch }
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
class ThrottlerProvider extends TransportAdapterProvider { class ThrottlerProvider extends TransportAdapterProvider {
@ -336,7 +337,8 @@ private[transport] class ThrottledAssociation(
val associationHandler: AssociationEventListener, val associationHandler: AssociationEventListener,
val originalHandle: AssociationHandle, val originalHandle: AssociationHandle,
val inbound: Boolean) val inbound: Boolean)
extends Actor with LoggingFSM[ThrottledAssociation.ThrottlerState, ThrottledAssociation.ThrottlerData] { extends Actor with LoggingFSM[ThrottledAssociation.ThrottlerState, ThrottledAssociation.ThrottlerData]
with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import ThrottledAssociation._ import ThrottledAssociation._
import context.dispatcher import context.dispatcher

View file

@ -14,6 +14,7 @@ import java.util.concurrent.TimeUnit
import akka.util.Timeout import akka.util.Timeout
import org.zeromq.ZMQException import org.zeromq.ZMQException
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import akka.dispatch.{ UnboundedMessageQueueSemantics, RequiresMessageQueue }
/** /**
* A Model to represent a version of the zeromq library * A Model to represent a version of the zeromq library
@ -241,7 +242,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension {
private val zeromqGuardian: ActorRef = { private val zeromqGuardian: ActorRef = {
verifyZeroMQVersion() verifyZeroMQVersion()
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] {
import SupervisorStrategy._ import SupervisorStrategy._
override def supervisorStrategy = OneForOneStrategy() { override def supervisorStrategy = OneForOneStrategy() {
case ex: ZMQException if nonfatal(ex) Resume case ex: ZMQException if nonfatal(ex) Resume