fold reservedNames into normal ChildStats trees inside ChildrenContainers

This commit is contained in:
Roland 2012-06-07 10:45:14 +02:00
parent 4f862c4328
commit dd596a20cb
4 changed files with 64 additions and 55 deletions

View file

@ -39,7 +39,7 @@ object ActorSystemSpec {
context stop self context stop self
} }
} }
override def preRestart(cause: Throwable, msg: Option[Any]) { override def preRestart(cause: Throwable, msg: Option[Any]) {
if (master ne null) { if (master ne null) {
master ! "failed with " + cause + " while processing " + msg master ! "failed with " + cause + " while processing " + msg

View file

@ -15,6 +15,7 @@ import akka.serialization.SerializationExtension
import akka.event.Logging.LogEventException import akka.event.Logging.LogEventException
import collection.immutable.{ TreeSet, Stack, TreeMap } import collection.immutable.{ TreeSet, Stack, TreeMap }
import akka.util.{ Unsafe, Duration, Helpers, NonFatal } import akka.util.{ Unsafe, Duration, Helpers, NonFatal }
import java.util.concurrent.atomic.AtomicLong
//TODO: everything here for current compatibility - could be limited more //TODO: everything here for current compatibility - could be limited more
@ -206,16 +207,16 @@ private[akka] object ActorCell {
} }
trait EmptyChildrenContainer extends ChildrenContainer { trait EmptyChildrenContainer extends ChildrenContainer {
val emptyStats = TreeMap.empty[String, ChildRestartStats] val emptyStats = TreeMap.empty[String, ChildStats]
def add(child: ActorRef): ChildrenContainer = def add(child: ActorRef): ChildrenContainer =
new NormalChildrenContainer(emptyStats.updated(child.path.name, ChildRestartStats(child)), Set()) new NormalChildrenContainer(emptyStats.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = this def remove(child: ActorRef): ChildrenContainer = this
def getByName(name: String): Option[ChildRestartStats] = None def getByName(name: String): Option[ChildRestartStats] = None
def getByRef(actor: ActorRef): Option[ChildRestartStats] = None def getByRef(actor: ActorRef): Option[ChildRestartStats] = None
def children: Iterable[ActorRef] = Nil def children: Iterable[ActorRef] = Nil
def stats: Iterable[ChildRestartStats] = Nil def stats: Iterable[ChildRestartStats] = Nil
def shallDie(actor: ActorRef): ChildrenContainer = this def shallDie(actor: ActorRef): ChildrenContainer = this
def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats, Set(name)) def reserve(name: String): ChildrenContainer = new NormalChildrenContainer(emptyStats.updated(name, ChildNameReserved))
def unreserve(name: String): ChildrenContainer = this def unreserve(name: String): ChildrenContainer = this
override def toString = "no children" override def toString = "no children"
} }
@ -241,32 +242,38 @@ private[akka] object ActorCell {
* calling context.stop(child) and processing the ChildTerminated() system * calling context.stop(child) and processing the ChildTerminated() system
* message). * message).
*/ */
class NormalChildrenContainer(c: TreeMap[String, ChildRestartStats], reservedNames: Set[String]) extends ChildrenContainer { class NormalChildrenContainer(c: TreeMap[String, ChildStats]) extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = def add(child: ActorRef): ChildrenContainer =
new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child)), reservedNames - child.path.name) new NormalChildrenContainer(c.updated(child.path.name, ChildRestartStats(child)))
def remove(child: ActorRef): ChildrenContainer = NormalChildrenContainer(c - child.path.name, reservedNames) def remove(child: ActorRef): ChildrenContainer = NormalChildrenContainer(c - child.path.name)
def getByName(name: String): Option[ChildRestartStats] = c get name def getByName(name: String): Option[ChildRestartStats] = c get name match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match { case _ None
case c @ Some(crs) if (crs.child == actor) c
case _ None
} }
def children: Iterable[ActorRef] = c.values.view.map(_.child) def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match {
case c @ Some(crs: ChildRestartStats) if (crs.child == actor) c.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def stats: Iterable[ChildRestartStats] = c.values def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) child }
def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest, reservedNames) def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats c }
def shallDie(actor: ActorRef): ChildrenContainer = TerminatingChildrenContainer(c, Set(actor), UserRequest)
def reserve(name: String): ChildrenContainer = def reserve(name: String): ChildrenContainer =
if ((c contains name) || (reservedNames contains name)) if (c contains name)
throw new InvalidActorNameException("actor name " + name + " is not unique!") throw new InvalidActorNameException("actor name " + name + " is not unique!")
else new NormalChildrenContainer(c, reservedNames + name) else new NormalChildrenContainer(c.updated(name, ChildNameReserved))
def unreserve(name: String): ChildrenContainer = NormalChildrenContainer(c, reservedNames - name) def unreserve(name: String): ChildrenContainer = c get name match {
case Some(ChildNameReserved) NormalChildrenContainer(c - name)
case _ this
}
override def toString = override def toString =
if (c.size > 20) c.size + " children" if (c.size > 20) c.size + " children"
@ -274,9 +281,9 @@ private[akka] object ActorCell {
} }
object NormalChildrenContainer { object NormalChildrenContainer {
def apply(c: TreeMap[String, ChildRestartStats], reservedName: Set[String]): ChildrenContainer = def apply(c: TreeMap[String, ChildStats]): ChildrenContainer =
if (c.isEmpty && reservedName.isEmpty) EmptyChildrenContainer if (c.isEmpty) EmptyChildrenContainer
else new NormalChildrenContainer(c, reservedName) else new NormalChildrenContainer(c)
} }
/** /**
@ -289,44 +296,45 @@ private[akka] object ActorCell {
* type of container, depending on whether or not children are left and whether or not * type of container, depending on whether or not children are left and whether or not
* the reason was Terminating. * the reason was Terminating.
*/ */
case class TerminatingChildrenContainer( case class TerminatingChildrenContainer(c: TreeMap[String, ChildStats], toDie: Set[ActorRef], reason: SuspendReason)
c: TreeMap[String, ChildRestartStats],
toDie: Set[ActorRef],
reason: SuspendReason,
reservedNames: Set[String])
extends ChildrenContainer { extends ChildrenContainer {
def add(child: ActorRef): ChildrenContainer = def add(child: ActorRef): ChildrenContainer = copy(c.updated(child.path.name, ChildRestartStats(child)))
copy(c.updated(child.path.name, ChildRestartStats(child)), reservedNames = reservedNames - child.path.name)
def remove(child: ActorRef): ChildrenContainer = { def remove(child: ActorRef): ChildrenContainer = {
val t = toDie - child val t = toDie - child
if (t.isEmpty) reason match { if (t.isEmpty) reason match {
case Termination TerminatedChildrenContainer case Termination TerminatedChildrenContainer
case _ NormalChildrenContainer(c - child.path.name, reservedNames) case _ NormalChildrenContainer(c - child.path.name)
} }
else copy(c - child.path.name, t) else copy(c - child.path.name, t)
} }
def getByName(name: String): Option[ChildRestartStats] = c get name def getByName(name: String): Option[ChildRestartStats] = c get name match {
case s @ Some(_: ChildRestartStats) s.asInstanceOf[Option[ChildRestartStats]]
def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match { case _ None
case c @ Some(crs) if (crs.child == actor) c
case _ None
} }
def children: Iterable[ActorRef] = c.values.view.map(_.child) def getByRef(actor: ActorRef): Option[ChildRestartStats] = c get actor.path.name match {
case c @ Some(crs: ChildRestartStats) if (crs.child == actor) c.asInstanceOf[Option[ChildRestartStats]]
case _ None
}
def stats: Iterable[ChildRestartStats] = c.values def children: Iterable[ActorRef] = c.values.view.collect { case ChildRestartStats(child, _, _) child }
def stats: Iterable[ChildRestartStats] = c.values.collect { case c: ChildRestartStats c }
def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor) def shallDie(actor: ActorRef): ChildrenContainer = copy(toDie = toDie + actor)
def reserve(name: String): ChildrenContainer = def reserve(name: String): ChildrenContainer =
if ((c contains name) || (reservedNames contains name)) if (c contains name)
throw new InvalidActorNameException("actor name " + name + " is not unique!") throw new InvalidActorNameException("actor name " + name + " is not unique!")
else copy(reservedNames = reservedNames + name) else copy(c = c.updated(name, ChildNameReserved))
def unreserve(name: String): ChildrenContainer = copy(reservedNames = reservedNames - name) def unreserve(name: String): ChildrenContainer = c get name match {
case Some(ChildNameReserved) copy(c = c - name)
case _ this
}
override def toString = override def toString =
if (c.size > 20) c.size + " children" if (c.size > 20) c.size + " children"
@ -387,12 +395,12 @@ private[akka] class ActorCell(
Unsafe.instance.compareAndSwapObject(this, childrenOffset, oldChildren, newChildren) Unsafe.instance.compareAndSwapObject(this, childrenOffset, oldChildren, newChildren)
private def isTerminating = childrenRefs match { private def isTerminating = childrenRefs match {
case TerminatingChildrenContainer(_, _, Termination, _) true case TerminatingChildrenContainer(_, _, Termination) true
case TerminatedChildrenContainer true case TerminatedChildrenContainer true
case _ false case _ false
} }
private def isNormal = childrenRefs match { private def isNormal = childrenRefs match {
case TerminatingChildrenContainer(_, _, Termination | _: Recreation, _) false case TerminatingChildrenContainer(_, _, Termination | _: Recreation) false
case _ true case _ true
} }
@ -461,16 +469,11 @@ private[akka] class ActorCell(
var actor: Actor = _ var actor: Actor = _
private var behaviorStack: Stack[Actor.Receive] = Stack.empty private var behaviorStack: Stack[Actor.Receive] = Stack.empty
@volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status @volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
var nextNameSequence: Long = 0
var watching: Set[ActorRef] = emptyActorRefSet var watching: Set[ActorRef] = emptyActorRefSet
var watchedBy: Set[ActorRef] = emptyActorRefSet var watchedBy: Set[ActorRef] = emptyActorRefSet
//Not thread safe, so should only be used inside the actor that inhabits this ActorCell val nextNameSequence = new AtomicLong
final protected def randomName(): String = { final protected def randomName(): String = Helpers.base64(nextNameSequence.getAndIncrement())
val n = nextNameSequence
nextNameSequence = n + 1
Helpers.base64(n)
}
@inline @inline
final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher) final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
@ -874,7 +877,7 @@ private[akka] class ActorCell(
else remove(ref) else remove(ref)
} }
childrenRefs match { childrenRefs match {
case tc @ TerminatingChildrenContainer(_, _, reason, _) case tc @ TerminatingChildrenContainer(_, _, reason)
val n = remove(child) val n = remove(child)
actor.supervisorStrategy.handleChildTerminated(this, child, children) actor.supervisorStrategy.handleChildTerminated(this, child, children)
if (!n.isInstanceOf[TerminatingChildrenContainer]) reason match { if (!n.isInstanceOf[TerminatingChildrenContainer]) reason match {

View file

@ -553,8 +553,3 @@ class LocalActorRefProvider(
def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None def getExternalAddressFor(addr: Address): Option[Address] = if (addr == rootPath.address) Some(addr) else None
} }
private[akka] class GuardianCell(_system: ActorSystemImpl, _self: InternalActorRef, _props: Props, _parent: InternalActorRef)
extends ActorCell(_system, _self, _props, _parent) {
}

View file

@ -9,11 +9,22 @@ import scala.collection.JavaConversions._
import java.lang.{ Iterable JIterable } import java.lang.{ Iterable JIterable }
import akka.util.Duration import akka.util.Duration
/**
* INTERNAL API
*/
private[akka] sealed trait ChildStats
/**
* INTERNAL API
*/
private[akka] case object ChildNameReserved extends ChildStats
/** /**
* ChildRestartStats is the statistics kept by every parent Actor for every child Actor * ChildRestartStats is the statistics kept by every parent Actor for every child Actor
* and is used for SupervisorStrategies to know how to deal with problems that occur for the children. * and is used for SupervisorStrategies to know how to deal with problems that occur for the children.
*/ */
case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) { case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L)
extends ChildStats {
//FIXME How about making ChildRestartStats immutable and then move these methods into the actual supervisor strategies? //FIXME How about making ChildRestartStats immutable and then move these methods into the actual supervisor strategies?
def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean = def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean =