Some polish on the error handling. See #1310

This commit is contained in:
Patrik Nordwall 2012-02-01 14:40:12 +01:00
parent f13b5356f7
commit c447f46224
15 changed files with 101 additions and 113 deletions

View file

@ -7,15 +7,15 @@ import org.scalatest.matchers.MustMatchers
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class NonFatalSpec extends AkkaSpec with MustMatchers { class HarmlessSpec extends AkkaSpec with MustMatchers {
"A NonFatal extractor" must { "A Harmless extractor" must {
"match ordinary RuntimeException" in { "match ordinary RuntimeException" in {
try { try {
throw new RuntimeException("Boom") throw new RuntimeException("Boom")
} catch { } catch {
case NonFatal(e) // as expected case Harmless(e) // as expected
} }
} }
@ -29,7 +29,7 @@ class NonFatalSpec extends AkkaSpec with MustMatchers {
try { try {
blowUp(0) blowUp(0)
} catch { } catch {
case NonFatal(e) assert(false) case Harmless(e) assert(false)
} }
} }
} }
@ -39,22 +39,26 @@ class NonFatalSpec extends AkkaSpec with MustMatchers {
try { try {
throw new InterruptedException("Simulated InterruptedException") throw new InterruptedException("Simulated InterruptedException")
} catch { } catch {
case NonFatal(e) assert(false) case Harmless(e) assert(false)
} }
} }
} }
} "be used together with InterruptedException" in {
"A NonFatalOrInterrupted extractor" must {
"match InterruptedException" in {
try { try {
throw new InterruptedException("Simulated InterruptedException") throw new InterruptedException("Simulated InterruptedException")
} catch { } catch {
case NonFatalOrInterrupted(e) // as expected case _: InterruptedException // as expected
case Harmless(e) assert(false)
}
try {
throw new RuntimeException("Simulated RuntimeException")
} catch {
case Harmless(_) | _: InterruptedException // as expected
} }
} }
} }
} }

View file

@ -14,8 +14,7 @@ import akka.util.{ Duration, Helpers }
import akka.japi.Procedure import akka.japi.Procedure
import java.io.{ NotSerializableException, ObjectOutputStream } import java.io.{ NotSerializableException, ObjectOutputStream }
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.util.NonFatal import akka.util.Harmless
import akka.util.NonFatalOrInterrupted
//TODO: everything here for current compatibility - could be limited more //TODO: everything here for current compatibility - could be limited more
@ -364,7 +363,7 @@ private[akka] class ActorCell(
checkReceiveTimeout checkReceiveTimeout
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(created), "started (" + created + ")")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(created), "started (" + created + ")"))
} catch { } catch {
case NonFatal(e) case Harmless(e)
try { try {
system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor")) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
@ -396,7 +395,7 @@ private[akka] class ActorCell(
actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children) actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children)
} catch { } catch {
case NonFatal(e) try { case Harmless(e) try {
system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor")) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
@ -460,7 +459,7 @@ private[akka] class ActorCell(
case ChildTerminated(child) handleChildTerminated(child) case ChildTerminated(child) handleChildTerminated(child)
} }
} catch { } catch {
case NonFatal(e) case Harmless(e)
system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while processing " + message)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while processing " + message))
//TODO FIXME How should problems here be handled??? //TODO FIXME How should problems here be handled???
throw e throw e
@ -482,26 +481,23 @@ private[akka] class ActorCell(
} }
currentMessage = null // reset current message after successful invocation currentMessage = null // reset current message after successful invocation
} catch { } catch {
case NonFatalOrInterrupted(e) case e: InterruptedException
// make sure that InterruptedException does not leave this thread
val ex = ActorInterruptedException(e)
actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(ex), self)
throw e //Re-throw InterruptedExceptions as expected
case Harmless(e)
system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
e match { actor.supervisorStrategy.handleSupervisorFailing(self, children)
case ie: InterruptedException parent.tell(Failed(e), self)
// make sure that InterruptedException does not leave this thread
val ex = ActorInterruptedException(ie)
actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(ex), self)
throw ie //Re-throw InterruptedExceptions as expected
case _
actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(e), self)
}
} finally { } finally {
checkReceiveTimeout // Reschedule receive timeout checkReceiveTimeout // Reschedule receive timeout
} }
} catch { } catch {
case NonFatal(e) case Harmless(e)
system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage))
throw e throw e
} }

View file

@ -346,8 +346,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Exten
def uncaughtException(thread: Thread, cause: Throwable): Unit = { def uncaughtException(thread: Thread, cause: Throwable): Unit = {
log.error(cause, "Uncaught error from thread [{}]", thread.getName) log.error(cause, "Uncaught error from thread [{}]", thread.getName)
cause match { cause match {
case NonFatalOrInterrupted(e) case Harmless(_) | _: InterruptedException
case _ shutdown() case _ shutdown()
} }
} }
} }

View file

@ -6,7 +6,7 @@ package akka.actor
import akka.japi.{ Creator, Option JOption } import akka.japi.{ Creator, Option JOption }
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
import akka.util.{ Timeout, NonFatal } import akka.util.{ Timeout, Harmless }
import java.util.concurrent.atomic.{ AtomicReference AtomVar } import java.util.concurrent.atomic.{ AtomicReference AtomVar }
import akka.serialization.{ Serialization, SerializationExtension } import akka.serialization.{ Serialization, SerializationExtension }
import akka.dispatch._ import akka.dispatch._
@ -270,7 +270,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
sender ! m(me) sender ! m(me)
} }
} catch { } catch {
case NonFatal(e) case Harmless(e)
sender ! Status.Failure(e) sender ! Status.Failure(e)
throw e throw e
} }

View file

@ -15,7 +15,7 @@ import com.typesafe.config.Config
import akka.util.ReflectiveAccess import akka.util.ReflectiveAccess
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.jsr166y.ForkJoinPool import akka.jsr166y.ForkJoinPool
import akka.util.NonFatal import akka.util.Harmless
final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) { final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) {
if (message.isInstanceOf[AnyRef]) { if (message.isInstanceOf[AnyRef]) {
@ -81,7 +81,7 @@ final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cl
try { try {
runnable.run() runnable.run()
} catch { } catch {
case NonFatal(e) case Harmless(e)
eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage)) eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
} finally { } finally {
cleanup() cleanup()

View file

@ -17,7 +17,7 @@ import akka.util.{ Duration, BoxedType }
import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger } import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger }
import akka.dispatch.Await.CanAwait import akka.dispatch.Await.CanAwait
import java.util.concurrent._ import java.util.concurrent._
import akka.util.NonFatal import akka.util.Harmless
object Await { object Await {
sealed trait CanAwait sealed trait CanAwait
@ -142,7 +142,7 @@ object Future {
try { try {
Right(body) Right(body)
} catch { } catch {
case NonFatal(e) Left(e) case Harmless(e) Left(e)
} }
} }
}) })
@ -322,19 +322,19 @@ object Future {
try { try {
next.apply() next.apply()
} catch { } catch {
case NonFatal(e) case Harmless(e) logError("Future.dispatchTask", e)
executor match {
case m: MessageDispatcher
m.prerequisites.eventStream.publish(Error(e, "Future.dispatchTask", this.getClass, e.getMessage))
case other
// TODO printStackTrace
e.printStackTrace()
}
} }
} }
} finally { _taskStack.remove() } } finally { _taskStack.remove() }
}) })
} }
private def logError(logSource: String, problem: Throwable)(implicit executor: ExecutionContext): Unit = {
executor match {
case m: MessageDispatcher m.prerequisites.eventStream.publish(Error(problem, logSource, this.getClass, problem.getMessage))
case other problem.printStackTrace()
}
}
} }
sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] { sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
@ -489,7 +489,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
future complete (try { future complete (try {
Right(f(res)) Right(f(res))
} catch { } catch {
case NonFatal(e) Left(e) case Harmless(e) Left(e)
}) })
} }
future future
@ -536,7 +536,8 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
try { try {
p completeWith f(r) p completeWith f(r)
} catch { } catch {
case NonFatal(e) p complete Left(e) case Harmless(e) p complete Left(e)
case t p complete Left(new ExecutionException(t)); throw t
} }
} }
p p
@ -574,15 +575,15 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
case r @ Right(res) p complete (try { case r @ Right(res) p complete (try {
if (pred(res)) r else Left(new MatchError(res)) if (pred(res)) r else Left(new MatchError(res))
} catch { } catch {
case NonFatal(e) Left(e) case Harmless(e) Left(e)
}) })
} }
p p
} }
protected def logError(msg: String, problem: Throwable): Unit = { protected def logError(logSource: String, problem: Throwable): Unit = {
executor match { executor match {
case m: MessageDispatcher m.prerequisites.eventStream.publish(Error(problem, msg, this.getClass, problem.getMessage)) case m: MessageDispatcher m.prerequisites.eventStream.publish(Error(problem, logSource, this.getClass, problem.getMessage))
case other problem.printStackTrace() case other problem.printStackTrace()
} }
} }
@ -660,7 +661,7 @@ trait Promise[T] extends Future[T] {
try { try {
fr completeWith cont(thisPromise) fr completeWith cont(thisPromise)
} catch { } catch {
case NonFatal(e) fr failure e case Harmless(e) fr failure e
} }
} }
fr fr
@ -673,7 +674,7 @@ trait Promise[T] extends Future[T] {
try { try {
fr completeWith cont(f) fr completeWith cont(f)
} catch { } catch {
case NonFatal(e) fr failure e case Harmless(e) fr failure e
} }
} }
fr fr
@ -781,7 +782,7 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac
} }
private final def notifyCompleted(func: Either[Throwable, T] Unit, result: Either[Throwable, T]) { private final def notifyCompleted(func: Either[Throwable, T] Unit, result: Either[Throwable, T]) {
try { func(result) } catch { case NonFatal(e) logError("Future onComplete-callback raised an exception", e) } try { func(result) } catch { case Harmless(e) logError("Future.onComplete", e) }
} }
} }

View file

@ -197,7 +197,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
if (nextMessage eq null) nextMessage = systemDrain() if (nextMessage eq null) nextMessage = systemDrain()
} }
} catch { } catch {
case NonFatal(e) case Harmless(e)
actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) actor.system.eventStream.publish(Error(e, actor.self.path.toString, this.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
throw e throw e
} }

View file

@ -0,0 +1,29 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
/**
* Extractor of harmless Throwables. Will not match fatal errors
* like VirtualMachineError (OutOfMemoryError, StackOverflowError)
* ThreadDeath, and InterruptedException.
*
* Usage to catch all harmless throwables:
* {{{
* try {
* // dangerous stuff
* } catch {
* case Harmless(e) => log.error(e, "Something not that bad")
* }
* }}}
*/
object Harmless {
def unapply(t: Throwable): Option[Throwable] = t match {
// VirtualMachineError includes OutOfMemoryError, StackOverflowError and other fatal errors
case _: VirtualMachineError | _: ThreadDeath | _: InterruptedException None
case e Some(e)
}
}

View file

@ -1,41 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
/**
* Extractor of non-fatal Throwables. Will not match
* VirtualMachineError (OutOfMemoryError, StackOverflowError)
* and InterruptedException.
*
* Usage to catch all non-fatal throwables:
* {{{
* try {
* // dangerous stuff
* } catch {
* case NonFatal(e) => log.error(e, "Something not that bad")
* }
* }}}
*/
object NonFatal {
def unapply(t: Throwable): Option[Throwable] = t match {
// VirtualMachineError includes OutOfMemoryError, StackOverflowError and other fatal errors
case e: VirtualMachineError None
case e: ThreadDeath None
case e: InterruptedException None
case e Some(e)
}
}
/**
* Match InterruptedException and the same as
* [[akka.util.NonFatal]].
*/
object NonFatalOrInterrupted {
def unapply(t: Throwable): Option[Throwable] = t match {
case e: InterruptedException Some(e)
case e NonFatal.unapply(t)
}
}

View file

@ -11,6 +11,7 @@ import akka.event.Logging
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.dispatch.MailboxType import akka.dispatch.MailboxType
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Harmless
class FileBasedMailboxType(config: Config) extends MailboxType { class FileBasedMailboxType(config: Config) extends MailboxType {
override def create(owner: ActorContext) = new FileBasedMailbox(owner) override def create(owner: ActorContext) = new FileBasedMailbox(owner)
@ -24,7 +25,7 @@ class FileBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) wi
val queuePath = settings.QueuePath val queuePath = settings.QueuePath
private val queue = try { private val queue = try {
try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case e {} } try { FileUtils.forceMkdir(new java.io.File(queuePath)) } catch { case Harmless(_) {} }
val queue = new filequeue.PersistentQueue(queuePath, name, settings, log) val queue = new filequeue.PersistentQueue(queuePath, name, settings, log)
queue.setup // replays journal queue.setup // replays journal
queue.discardExpired queue.discardExpired
@ -68,8 +69,7 @@ class FileBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) wi
queue.remove queue.remove
true true
} catch { } catch {
// TODO catching all and continue isn't good for OOME, ticket #1418 case Harmless(_) false
case e false
} }
} }

View file

@ -21,6 +21,7 @@ import java.io._
import java.nio.{ ByteBuffer, ByteOrder } import java.nio.{ ByteBuffer, ByteOrder }
import java.nio.channels.FileChannel import java.nio.channels.FileChannel
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.util.Harmless
// returned from journal replay // returned from journal replay
sealed trait JournalItem sealed trait JournalItem
@ -99,7 +100,7 @@ class Journal(queuePath: String, syncJournal: ⇒ Boolean, log: LoggingAdapter)
close() close()
queueFile.delete queueFile.delete
} catch { } catch {
case _ case Harmless(_)
} }
} }

View file

@ -11,6 +11,7 @@ import akka.event.Logging
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.dispatch.MailboxType import akka.dispatch.MailboxType
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Harmless
class RedisBasedMailboxException(message: String) extends AkkaException(message) class RedisBasedMailboxException(message: String) extends AkkaException(message)
@ -47,7 +48,7 @@ class RedisBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) w
envelope envelope
} catch { } catch {
case e: java.util.NoSuchElementException null case e: java.util.NoSuchElementException null
case e case Harmless(e)
log.error(e, "Couldn't dequeue from Redis-based mailbox") log.error(e, "Couldn't dequeue from Redis-based mailbox")
throw e throw e
} }
@ -73,7 +74,7 @@ class RedisBasedMailbox(val owner: ActorContext) extends DurableMailbox(owner) w
clients = connect() clients = connect()
body body
} }
case e case Harmless(e)
val error = new RedisBasedMailboxException("Could not connect to Redis server, due to: " + e.getMessage) val error = new RedisBasedMailboxException("Could not connect to Redis server, due to: " + e.getMessage)
log.error(error, error.getMessage) log.error(error, error.getMessage)
throw error throw error

View file

@ -12,6 +12,7 @@ import akka.cluster.zookeeper.ZooKeeperQueue
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.dispatch.MailboxType import akka.dispatch.MailboxType
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Harmless
class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message) class ZooKeeperBasedMailboxException(message: String) extends AkkaException(message)
@ -45,7 +46,7 @@ class ZooKeeperBasedMailbox(val owner: ActorContext) extends DurableMailbox(owne
} catch { } catch {
case e: java.util.NoSuchElementException null case e: java.util.NoSuchElementException null
case e: InterruptedException null case e: InterruptedException null
case e case Harmless(e)
log.error(e, "Couldn't dequeue from ZooKeeper-based mailbox, due to: " + e.getMessage) log.error(e, "Couldn't dequeue from ZooKeeper-based mailbox, due to: " + e.getMessage)
throw e throw e
} }

View file

@ -9,21 +9,19 @@ import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.concurrent.locks.ReentrantReadWriteLock
import java.util.concurrent.Executors import java.util.concurrent.Executors
import scala.collection.mutable.HashMap import scala.collection.mutable.HashMap
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture } import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroupFuture }
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
import org.jboss.netty.channel.{ ChannelHandlerContext, Channel } import org.jboss.netty.channel.{ ChannelHandlerContext, Channel }
import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder } import org.jboss.netty.handler.codec.protobuf.{ ProtobufEncoder, ProtobufDecoder }
import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor
import org.jboss.netty.util.HashedWheelTimer import org.jboss.netty.util.HashedWheelTimer
import akka.actor.{ Address, ActorSystemImpl, ActorRef } import akka.actor.{ Address, ActorSystemImpl, ActorRef }
import akka.dispatch.MonitorableThreadFactory import akka.dispatch.MonitorableThreadFactory
import akka.event.Logging import akka.event.Logging
import akka.remote.RemoteProtocol.AkkaRemoteProtocol import akka.remote.RemoteProtocol.AkkaRemoteProtocol
import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef } import akka.remote.{ RemoteTransportException, RemoteTransport, RemoteSettings, RemoteMarshallingOps, RemoteActorRefProvider, RemoteActorRef }
import akka.util.Harmless
/** /**
* Provides the implementation of the Netty remote support * Provides the implementation of the Netty remote support
@ -80,7 +78,7 @@ class NettyRemoteTransport(val remoteSettings: RemoteSettings, val system: Actor
try { try {
remoteClients foreach { remoteClients foreach {
case (_, client) try client.shutdown() catch { case (_, client) try client.shutdown() catch {
case e log.error(e, "failure while shutting down [{}]", client) case Harmless(e) log.error(e, "failure while shutting down [{}]", client)
} }
} }
remoteClients.clear() remoteClients.clear()

View file

@ -6,15 +6,13 @@ package akka.testkit
import java.lang.ref.WeakReference import java.lang.ref.WeakReference
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import java.util.LinkedList import java.util.LinkedList
import scala.annotation.tailrec import scala.annotation.tailrec
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell }
import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue }
import akka.util.duration.intToDurationInt import akka.util.duration.intToDurationInt
import akka.util.{ Switch, Duration } import akka.util.{ Switch, Duration }
import akka.util.Harmless
/* /*
* Locking rules: * Locking rules:
@ -225,7 +223,7 @@ class CallingThreadDispatcher(
Thread.currentThread().interrupt() Thread.currentThread().interrupt()
intex = ie intex = ie
true true
case e case Harmless(e)
log.error(e, "Error during message processing") log.error(e, "Error during message processing")
queue.leave queue.leave
false false
@ -235,7 +233,7 @@ class CallingThreadDispatcher(
false false
} else false } else false
} catch { } catch {
case e queue.leave; throw e case Harmless(e) queue.leave; throw e
} finally { } finally {
mbox.ctdLock.unlock mbox.ctdLock.unlock
} }
@ -294,8 +292,8 @@ class CallingThreadMailbox(_receiver: ActorCell) extends Mailbox(_receiver) with
override def cleanUp(): Unit = { override def cleanUp(): Unit = {
/* /*
* This is called from dispatcher.unregister, i.e. under this.lock. If * This is called from dispatcher.unregister, i.e. under this.lock. If
* another thread obtained a reference to this mailbox and enqueues after * another thread obtained a reference to this mailbox and enqueues after
* the gather operation, tough luck: no guaranteed delivery to deadLetters. * the gather operation, tough luck: no guaranteed delivery to deadLetters.
*/ */
suspendSwitch.locked { suspendSwitch.locked {