Fail actor creation if mailbox doesn't conform to required type. See #3237
This commit is contained in:
parent
7094daf313
commit
9fd42c7cab
9 changed files with 43 additions and 37 deletions
|
|
@ -7,6 +7,7 @@ package akka.actor
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
|
import akka.TestUtils.verifyActorTermination
|
||||||
|
|
||||||
object ActorMailboxSpec {
|
object ActorMailboxSpec {
|
||||||
val mailboxConf = ConfigFactory.parseString("""
|
val mailboxConf = ConfigFactory.parseString("""
|
||||||
|
|
@ -133,10 +134,9 @@ class ActorMailboxSpec extends AkkaSpec(ActorMailboxSpec.mailboxConf) with Defau
|
||||||
checkMailboxQueue(Props[QueueReportingActor], "default-unbounded-deque", UnboundedDeqMailboxTypes)
|
checkMailboxQueue(Props[QueueReportingActor], "default-unbounded-deque", UnboundedDeqMailboxTypes)
|
||||||
}
|
}
|
||||||
|
|
||||||
"get an unbounded dequeu message queue when it's configured as mailbox overriding RequestMailbox" in {
|
"fail to create actor when an unbounded dequeu message queue is configured as mailbox overriding RequestMailbox" in {
|
||||||
filterEvents(EventFilter[IllegalArgumentException]()) {
|
filterEvents(EventFilter[ActorInitializationException]()) {
|
||||||
checkMailboxQueue(Props[BoundedQueueReportingActor], "default-unbounded-deque-override-trait",
|
verifyActorTermination(system.actorOf(Props[BoundedQueueReportingActor], "default-unbounded-deque-override-trait"))
|
||||||
UnboundedDeqMailboxTypes)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -144,9 +144,9 @@ class ActorMailboxSpec extends AkkaSpec(ActorMailboxSpec.mailboxConf) with Defau
|
||||||
checkMailboxQueue(Props[QueueReportingActor], "unbounded-default", UnboundedMailboxTypes)
|
checkMailboxQueue(Props[QueueReportingActor], "unbounded-default", UnboundedMailboxTypes)
|
||||||
}
|
}
|
||||||
|
|
||||||
"get an unbounded message queue when defined in dispatcher overriding RequestMailbox" in {
|
"fail to create actor when an unbounded message queue is defined in dispatcher overriding RequestMailbox" in {
|
||||||
filterEvents(EventFilter[IllegalArgumentException]()) {
|
filterEvents(EventFilter[ActorInitializationException]()) {
|
||||||
checkMailboxQueue(Props[BoundedQueueReportingActor], "unbounded-default-override-trait", UnboundedMailboxTypes)
|
verifyActorTermination(system.actorOf(Props[BoundedQueueReportingActor], "unbounded-default-override-trait"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -422,7 +422,7 @@ private[akka] class ActorCell(
|
||||||
case message: SystemMessage if shouldStash(message, currentState) ⇒ stash(message)
|
case message: SystemMessage if shouldStash(message, currentState) ⇒ stash(message)
|
||||||
case f: Failed ⇒ handleFailure(f)
|
case f: Failed ⇒ handleFailure(f)
|
||||||
case DeathWatchNotification(a, ec, at) ⇒ watchedActorTerminated(a, ec, at)
|
case DeathWatchNotification(a, ec, at) ⇒ watchedActorTerminated(a, ec, at)
|
||||||
case Create() ⇒ create()
|
case Create(failure) ⇒ create(failure)
|
||||||
case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher)
|
case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher)
|
||||||
case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher)
|
case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher)
|
||||||
case Recreate(cause) ⇒ faultRecreate(cause)
|
case Recreate(cause) ⇒ faultRecreate(cause)
|
||||||
|
|
@ -547,13 +547,16 @@ private[akka] class ActorCell(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected def create(): Unit = {
|
protected def create(failure: Option[ActorInitializationException]): Unit = {
|
||||||
def clearOutActorIfNonNull(): Unit = {
|
def clearOutActorIfNonNull(): Unit = {
|
||||||
if (actor != null) {
|
if (actor != null) {
|
||||||
clearActorFields(actor)
|
clearActorFields(actor)
|
||||||
actor = null // ensure that we know that we failed during creation
|
actor = null // ensure that we know that we failed during creation
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
failure foreach { throw _ }
|
||||||
|
|
||||||
try {
|
try {
|
||||||
val created = newActor()
|
val created = newActor()
|
||||||
actor = created
|
actor = created
|
||||||
|
|
|
||||||
|
|
@ -255,14 +255,7 @@ case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private lazy val cachedActorClass: Class[_ <: Actor] = {
|
||||||
* Obtain an upper-bound approximation of the actor class which is going to
|
|
||||||
* be created by these Props. In other words, the [[#newActor]] method will
|
|
||||||
* produce an instance of this class or a subclass thereof. This is used by
|
|
||||||
* the actor system to select special dispatchers or mailboxes in case
|
|
||||||
* dependencies are encoded in the actor type.
|
|
||||||
*/
|
|
||||||
def actorClass(): Class[_ <: Actor] = {
|
|
||||||
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
|
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
|
||||||
Reflect.instantiate(clazz, args).asInstanceOf[IndirectActorProducer].actorClass
|
Reflect.instantiate(clazz, args).asInstanceOf[IndirectActorProducer].actorClass
|
||||||
} else if (classOf[Actor].isAssignableFrom(clazz)) {
|
} else if (classOf[Actor].isAssignableFrom(clazz)) {
|
||||||
|
|
@ -271,6 +264,15 @@ case class Props(deploy: Deploy, clazz: Class[_], args: immutable.Seq[Any]) {
|
||||||
throw new IllegalArgumentException("unknown actor creator [$clazz]")
|
throw new IllegalArgumentException("unknown actor creator [$clazz]")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
/**
|
||||||
|
* Obtain an upper-bound approximation of the actor class which is going to
|
||||||
|
* be created by these Props. In other words, the [[#newActor]] method will
|
||||||
|
* produce an instance of this class or a subclass thereof. This is used by
|
||||||
|
* the actor system to select special dispatchers or mailboxes in case
|
||||||
|
* dependencies are encoded in the actor type.
|
||||||
|
*/
|
||||||
|
def actorClass(): Class[_ <: Actor] = cachedActorClass
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -10,11 +10,10 @@ import akka.dispatch.sysmsg._
|
||||||
import akka.event.Logging.Error
|
import akka.event.Logging.Error
|
||||||
import akka.util.Unsafe
|
import akka.util.Unsafe
|
||||||
import akka.dispatch.NullMessage
|
import akka.dispatch.NullMessage
|
||||||
import akka.actor.{ NoSerializationVerificationNeeded, InvalidMessageException, ActorRef, ActorCell }
|
import akka.actor._
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import scala.util.control.Exception.Catcher
|
import scala.util.control.Exception.Catcher
|
||||||
import scala.concurrent.ExecutionContext
|
|
||||||
|
|
||||||
private[akka] trait Dispatch { this: ActorCell ⇒
|
private[akka] trait Dispatch { this: ActorCell ⇒
|
||||||
|
|
||||||
|
|
@ -47,23 +46,24 @@ private[akka] trait Dispatch { this: ActorCell ⇒
|
||||||
* this is processed before anything else.
|
* this is processed before anything else.
|
||||||
*/
|
*/
|
||||||
val mbox = dispatcher.createMailbox(this)
|
val mbox = dispatcher.createMailbox(this)
|
||||||
val actorClass = this.props.actorClass
|
|
||||||
if (this.system.mailboxes.hasRequiredType(actorClass)) {
|
// we need to delay the failure to the point of actor creation so we can handle
|
||||||
this.system.mailboxes.getRequiredType(actorClass).foreach {
|
// it properly in the normal way
|
||||||
|
val actorClass = props.actorClass
|
||||||
|
val createMessage = if (system.mailboxes.hasRequiredType(actorClass)) {
|
||||||
|
Create(system.mailboxes.getRequiredType(actorClass).flatMap {
|
||||||
case c if !c.isAssignableFrom(mbox.messageQueue.getClass) ⇒
|
case c if !c.isAssignableFrom(mbox.messageQueue.getClass) ⇒
|
||||||
// FIXME 3237 throw an exception here instead of just logging it,
|
Some(ActorInitializationException(self, s"Actor [${self}] requires mailbox type [${c}]" +
|
||||||
// and update the comment on the RequiresMessageQueue trait
|
s" got [${mbox.messageQueue.getClass}]"))
|
||||||
val e = new IllegalArgumentException(s"Actor [${this.self.path}] requires mailbox type [${c}]" +
|
case _ ⇒ None
|
||||||
s" got [${mbox.messageQueue.getClass}]")
|
})
|
||||||
this.systemImpl.eventStream.publish(Error(e, getClass.getName, getClass, e.getMessage))
|
} else Create(None)
|
||||||
case _ ⇒
|
|
||||||
}
|
|
||||||
}
|
|
||||||
swapMailbox(mbox)
|
swapMailbox(mbox)
|
||||||
mailbox.setActor(this)
|
mailbox.setActor(this)
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
mailbox.systemEnqueue(self, Create())
|
mailbox.systemEnqueue(self, createMessage)
|
||||||
|
|
||||||
if (sendSupervise) {
|
if (sendSupervise) {
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
|
|
|
||||||
|
|
@ -132,7 +132,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
|
||||||
private def finishCreate(): Unit = {
|
private def finishCreate(): Unit = {
|
||||||
try resumeNonRecursive()
|
try resumeNonRecursive()
|
||||||
finally clearFailed()
|
finally clearFailed()
|
||||||
try create()
|
try create(None)
|
||||||
catch handleNonFatalOrInterruptedException { e ⇒
|
catch handleNonFatalOrInterruptedException { e ⇒
|
||||||
handleInvokeFailure(Nil, e)
|
handleInvokeFailure(Nil, e)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -631,6 +631,7 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTime
|
||||||
* The mailbox type will be looked up by mapping the type T via akka.actor.mailbox.requirements in the config,
|
* The mailbox type will be looked up by mapping the type T via akka.actor.mailbox.requirements in the config,
|
||||||
* to a mailbox configuration. If no mailbox is assigned on Props or in deployment config then this one will be used.
|
* to a mailbox configuration. If no mailbox is assigned on Props or in deployment config then this one will be used.
|
||||||
*
|
*
|
||||||
* The queue type of the created mailbox will be checked against the type T and an error will be logged if it doesn't match.
|
* The queue type of the created mailbox will be checked against the type T and actor creation will fail if it doesn't
|
||||||
|
* fulfill the requirements.
|
||||||
*/
|
*/
|
||||||
trait RequiresMessageQueue[T]
|
trait RequiresMessageQueue[T]
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
package akka.dispatch.sysmsg
|
package akka.dispatch.sysmsg
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import akka.actor.{ InternalActorRef, ActorRef, PossiblyHarmful }
|
import akka.actor.{ ActorInitializationException, InternalActorRef, ActorRef, PossiblyHarmful }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -201,7 +201,7 @@ trait StashWhenFailed
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(-4836972106317757555L)
|
@SerialVersionUID(-4836972106317757555L)
|
||||||
private[akka] case class Create() extends SystemMessage // send to self from Dispatcher.register
|
private[akka] case class Create(failure: Option[ActorInitializationException]) extends SystemMessage // sent to self from Dispatcher.register
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -238,7 +238,7 @@ a dispatcher with a specified mailbox type, then that will override this mapping
|
||||||
.. note::
|
.. note::
|
||||||
|
|
||||||
The type of the queue in the mailbox created for an actor will be checked against the required type in the
|
The type of the queue in the mailbox created for an actor will be checked against the required type in the
|
||||||
interface and if the queue doesn't implement the required type an error will be logged.
|
interface and if the queue doesn't implement the required type then actor creation will fail.
|
||||||
|
|
||||||
|
|
||||||
Mailbox configuration precedence
|
Mailbox configuration precedence
|
||||||
|
|
|
||||||
|
|
@ -240,7 +240,7 @@ a dispatcher with a specified mailbox type, then that will override this mapping
|
||||||
.. note::
|
.. note::
|
||||||
|
|
||||||
The type of the queue in the mailbox created for an actor will be checked against the required type in the
|
The type of the queue in the mailbox created for an actor will be checked against the required type in the
|
||||||
trait and if the queue doesn't implement the required type an error will be logged.
|
trait and if the queue doesn't implement the required type then actor creation will fail.
|
||||||
|
|
||||||
|
|
||||||
Mailbox configuration precedence
|
Mailbox configuration precedence
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue