Introduced uniquely identifiable, loggable base exception: AkkaException and made use of it throught the project
This commit is contained in:
parent
04af409889
commit
51b22b22e7
22 changed files with 106 additions and 91 deletions
1
.gitignore
vendored
1
.gitignore
vendored
|
|
@ -18,6 +18,7 @@ deploy/*.jar
|
|||
data
|
||||
out
|
||||
logs
|
||||
.#*
|
||||
.codefellow
|
||||
storage
|
||||
.codefellow
|
||||
|
|
|
|||
|
|
@ -5,6 +5,8 @@
|
|||
package se.scalablesolutions.akka.amqp
|
||||
|
||||
import se.scalablesolutions.akka.actor.ActorRef
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import com.rabbitmq.client.AMQP.BasicProperties
|
||||
import com.rabbitmq.client.ShutdownSignalException
|
||||
|
||||
|
|
|
|||
|
|
@ -4,13 +4,15 @@
|
|||
|
||||
package se.scalablesolutions.akka.amqp
|
||||
|
||||
import com.rabbitmq.client.AMQP.Queue.DeclareOk
|
||||
import collection.JavaConversions
|
||||
|
||||
import se.scalablesolutions.akka.amqp.AMQP.ConsumerParameters
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import com.rabbitmq.client.{Channel, Envelope, DefaultConsumer}
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import com.rabbitmq.client.AMQP.Queue.DeclareOk
|
||||
import com.rabbitmq.client.AMQP.BasicProperties
|
||||
import java.lang.Throwable
|
||||
import com.rabbitmq.client.{Channel, Envelope, DefaultConsumer}
|
||||
|
||||
private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
|
||||
extends FaultTolerantChannelActor(
|
||||
|
|
|
|||
|
|
@ -5,7 +5,9 @@
|
|||
package se.scalablesolutions.akka.amqp
|
||||
|
||||
import com.rabbitmq.client._
|
||||
|
||||
import se.scalablesolutions.akka.amqp.AMQP.ProducerParameters
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
private[amqp] class ProducerActor(producerParameters: ProducerParameters)
|
||||
extends FaultTolerantChannelActor(
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package se.scalablesolutions.akka.camel
|
||||
|
||||
import se.scalablesolutions.akka.actor._
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -15,15 +15,14 @@ import org.apache.camel._
|
|||
import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
|
||||
|
||||
import se.scalablesolutions.akka.camel.{Failure, CamelMessageConversion, Message}
|
||||
import CamelMessageConversion.toExchangeAdapter
|
||||
import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
|
||||
import se.scalablesolutions.akka.stm.TransactionConfig
|
||||
import se.scalablesolutions.akka.actor.{ScalaActorRef, ActorRegistry, Actor, ActorRef}
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import scala.reflect.BeanProperty
|
||||
|
||||
import CamelMessageConversion.toExchangeAdapter
|
||||
import java.lang.Throwable
|
||||
import se.scalablesolutions.akka.actor.{ScalaActorRef, ActorRegistry, Actor, ActorRef}
|
||||
|
||||
/**
|
||||
* Camel component for sending messages to and receiving replies from (untyped) actors.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import se.scalablesolutions.akka.config.ScalaConfig._
|
|||
import se.scalablesolutions.akka.serialization.Serializer
|
||||
import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently}
|
||||
import se.scalablesolutions.akka.util.{Logging, Duration}
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import com.google.protobuf.Message
|
||||
|
||||
|
|
@ -67,11 +68,11 @@ case class MaximumNumberOfRestartsWithinTimeRangeReached(
|
|||
@BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
|
||||
|
||||
// Exceptions for Actors
|
||||
class ActorStartException private[akka](message: String) extends RuntimeException(message)
|
||||
class IllegalActorStateException private[akka](message: String) extends RuntimeException(message)
|
||||
class ActorKilledException private[akka](message: String) extends RuntimeException(message)
|
||||
class ActorInitializationException private[akka](message: String) extends RuntimeException(message)
|
||||
class ActorTimeoutException private[akka](message: String) extends RuntimeException(message)
|
||||
class ActorStartException private[akka](message: String) extends AkkaException(message)
|
||||
class IllegalActorStateException private[akka](message: String) extends AkkaException(message)
|
||||
class ActorKilledException private[akka](message: String) extends AkkaException(message)
|
||||
class ActorInitializationException private[akka](message: String) extends AkkaException(message)
|
||||
class ActorTimeoutException private[akka](message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* Actor factory module with factory methods for creating various kinds of Actors.
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@ import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
|
|||
import se.scalablesolutions.akka.remote.{RemoteNode, RemoteServer, RemoteClient, MessageSerializer, RemoteRequestProtocolIdFactory}
|
||||
import se.scalablesolutions.akka.serialization.{Serializer, BinaryString}
|
||||
import se.scalablesolutions.akka.util.{HashCode, Logging, UUID, ReentrantGuard}
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
import RemoteActorSerialization._
|
||||
|
||||
import org.multiverse.api.ThreadLocalTransaction._
|
||||
|
|
|
|||
|
|
@ -5,11 +5,12 @@
|
|||
package se.scalablesolutions.akka.actor
|
||||
|
||||
import se.scalablesolutions.akka.stm.Ref
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.CountDownLatch
|
||||
|
||||
class AgentException private[akka](message: String) extends RuntimeException(message)
|
||||
class AgentException private[akka](message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* The Agent class was strongly inspired by the agent principle in Clojure.
|
||||
|
|
|
|||
|
|
@ -15,10 +15,11 @@
|
|||
*/
|
||||
package se.scalablesolutions.akka.actor
|
||||
|
||||
import _root_.scala.collection.JavaConversions
|
||||
import scala.collection.JavaConversions
|
||||
import java.util.concurrent._
|
||||
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
object Scheduler extends Logging {
|
||||
import Actor._
|
||||
|
|
|
|||
|
|
@ -8,12 +8,13 @@ import se.scalablesolutions.akka.config.ScalaConfig._
|
|||
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.remote.RemoteServer
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
import Actor._
|
||||
|
||||
import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
class SupervisorException private[akka](message: String) extends RuntimeException(message)
|
||||
class SupervisorException private[akka](message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* Factory object for creating supervisors declarative. It creates instances of the 'Supervisor' class.
|
||||
|
|
|
|||
|
|
@ -5,10 +5,11 @@
|
|||
package se.scalablesolutions.akka.config
|
||||
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
|
||||
|
||||
class ConfigurationException(message: String) extends RuntimeException(message)
|
||||
class ConfigurationException(message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* Loads up the configuration (from the akka.conf file).
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
|
|||
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
|
||||
import se.scalablesolutions.akka.actor.Actor._
|
||||
import se.scalablesolutions.akka.dispatch.CompletableFuture
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
/**
|
||||
* Implements Oz-style dataflow (single assignment) variables.
|
||||
|
|
@ -155,7 +156,7 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
|
|||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class DataFlowVariableException(msg: String) extends RuntimeException(msg)
|
||||
class DataFlowVariableException(msg: String) extends AkkaException(msg)
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -4,11 +4,12 @@
|
|||
|
||||
package se.scalablesolutions.akka.dispatch
|
||||
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
class FutureTimeoutException(message: String) extends RuntimeException(message)
|
||||
class FutureTimeoutException(message: String) extends AkkaException(message)
|
||||
|
||||
object Futures {
|
||||
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
|
|||
import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, RemoteActorRef, IllegalActorStateException}
|
||||
import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
|
||||
import se.scalablesolutions.akka.config.Config._
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import org.jboss.netty.channel._
|
||||
import group.DefaultChannelGroup
|
||||
|
|
@ -55,7 +56,7 @@ case class RemoteClientConnected(
|
|||
@BeanProperty val host: String,
|
||||
@BeanProperty val port: Int) extends RemoteClientLifeCycleEvent
|
||||
|
||||
class RemoteClientException private[akka](message: String) extends RuntimeException(message)
|
||||
class RemoteClientException private[akka](message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* The RemoteClient object manages RemoteClient instances and gives you an API to lookup remote actor handles.
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import javax.naming.{InitialContext, Context, NamingException}
|
|||
|
||||
import se.scalablesolutions.akka.config.Config._
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
/**
|
||||
* Detects if there is a UserTransaction or TransactionManager available in the JNDI.
|
||||
|
|
|
|||
|
|
@ -13,15 +13,16 @@ import scala.collection.mutable.HashMap
|
|||
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.config.Config._
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import org.multiverse.api.{Transaction => MultiverseTransaction}
|
||||
import org.multiverse.api.lifecycle.{TransactionLifecycleListener, TransactionLifecycleEvent}
|
||||
import org.multiverse.api.ThreadLocalTransaction._
|
||||
import org.multiverse.api.{TraceLevel => MultiverseTraceLevel}
|
||||
|
||||
class NoTransactionInScopeException extends RuntimeException
|
||||
class TransactionRetryException(message: String) extends RuntimeException(message)
|
||||
class StmConfigurationException(message: String) extends RuntimeException(message)
|
||||
class NoTransactionInScopeException extends AkkaException("No transaction in scope")
|
||||
class TransactionRetryException(message: String) extends AkkaException(message)
|
||||
class StmConfigurationException(message: String) extends AkkaException(message)
|
||||
|
||||
object Transaction {
|
||||
val idFactory = new AtomicLong(-1L)
|
||||
|
|
|
|||
|
|
@ -4,18 +4,15 @@
|
|||
|
||||
package se.scalablesolutions.akka.stm
|
||||
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import org.multiverse.api.{StmUtils => MultiverseStmUtils}
|
||||
import org.multiverse.api.ThreadLocalTransaction._
|
||||
import org.multiverse.api.{Transaction => MultiverseTransaction}
|
||||
import org.multiverse.commitbarriers.CountDownCommitBarrier
|
||||
import org.multiverse.templates.OrElseTemplate
|
||||
|
||||
class TransactionSetAbortedException(msg: String) extends RuntimeException(msg)
|
||||
|
||||
// TODO Should we remove TransactionAwareWrapperException? Not used anywhere yet.
|
||||
class TransactionAwareWrapperException(val cause: Throwable, val tx: Option[Transaction]) extends RuntimeException(cause) {
|
||||
override def toString = "TransactionAwareWrapperException[" + cause + ", " + tx + "]"
|
||||
}
|
||||
class TransactionSetAbortedException(msg: String) extends AkkaException(msg)
|
||||
|
||||
/**
|
||||
* Internal helper methods and properties for transaction management.
|
||||
|
|
|
|||
50
akka-core/src/main/scala/util/AkkaException.scala
Normal file
50
akka-core/src/main/scala/util/AkkaException.scala
Normal file
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package se.scalablesolutions.akka
|
||||
|
||||
import se.scalablesolutions.akka.util.{UUID, Logging}
|
||||
|
||||
import java.io.{StringWriter, PrintWriter}
|
||||
import java.net.{InetAddress, UnknownHostException}
|
||||
|
||||
/**
|
||||
* Akka base Exception. Each Exception gets:
|
||||
* <ul>
|
||||
* <li>a UUID for tracking purposes</li>
|
||||
* <li>a message including exception name, uuid, original message and the stacktrace</li>
|
||||
* <li>a method 'log' that will log the exception once and only once</li>
|
||||
* </ul>
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
@serializable abstract class AkkaException(message: String) extends RuntimeException(message) {
|
||||
@volatile private var isLogged = false
|
||||
val exceptionName = getClass.getName
|
||||
|
||||
val uuid = String.format("%s_%s", AkkaException.hostname, UUID.newUuid.toString)
|
||||
|
||||
override val toString =
|
||||
String.format("%s\n\t[%s]\n\t%s\n\t%s", exceptionName, uuid, message, stackTrace)
|
||||
|
||||
val stackTrace = {
|
||||
val sw = new StringWriter
|
||||
val pw = new PrintWriter(sw)
|
||||
printStackTrace(pw)
|
||||
sw.toString
|
||||
}
|
||||
|
||||
def log = if (!isLogged) {
|
||||
isLogged = true
|
||||
AkkaException.log.error(toString)
|
||||
}
|
||||
}
|
||||
|
||||
object AkkaException extends Logging {
|
||||
val hostname = try {
|
||||
InetAddress.getLocalHost.getHostName
|
||||
} catch {
|
||||
case e: UnknownHostException => "unknown"
|
||||
}
|
||||
}
|
||||
|
|
@ -6,8 +6,6 @@ package se.scalablesolutions.akka.util
|
|||
|
||||
import java.security.MessageDigest
|
||||
|
||||
class SystemFailure(cause: Throwable) extends RuntimeException(cause)
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -35,13 +35,13 @@ trait Logging {
|
|||
* http://download-llnw.oracle.com/javase/6/docs/api/java/lang/String.html#format(java.lang.String,%20java.lang.Object...)
|
||||
*/
|
||||
class Logger(val logger: SLFLogger) {
|
||||
def name = logger.getName
|
||||
def name = logger.getName
|
||||
|
||||
def trace_? = logger.isTraceEnabled
|
||||
def debug_? = logger.isDebugEnabled
|
||||
def info_? = logger.isInfoEnabled
|
||||
def warning_? = logger.isWarnEnabled
|
||||
def error_? = logger.isErrorEnabled
|
||||
def trace_? = logger.isTraceEnabled
|
||||
def debug_? = logger.isDebugEnabled
|
||||
def info_? = logger.isInfoEnabled
|
||||
def warning_? = logger.isWarnEnabled
|
||||
def error_? = logger.isErrorEnabled
|
||||
|
||||
//Trace
|
||||
def trace(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
|
||||
|
|
@ -59,6 +59,7 @@ class Logger(val logger: SLFLogger) {
|
|||
def trace(msg: => String) {
|
||||
if (trace_?) logger trace msg
|
||||
}
|
||||
|
||||
//Debug
|
||||
def debug(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
|
||||
debug(t,message(fmt,arg,argN:_*))
|
||||
|
|
@ -75,6 +76,7 @@ class Logger(val logger: SLFLogger) {
|
|||
def debug(msg: => String) {
|
||||
if (debug_?) logger debug msg
|
||||
}
|
||||
|
||||
//Info
|
||||
def info(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
|
||||
info(t,message(fmt,arg,argN:_*))
|
||||
|
|
@ -91,6 +93,7 @@ class Logger(val logger: SLFLogger) {
|
|||
def info(msg: => String) {
|
||||
if (info_?) logger info msg
|
||||
}
|
||||
|
||||
//Warning
|
||||
def warning(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
|
||||
warning(t,message(fmt,arg,argN:_*))
|
||||
|
|
@ -107,6 +110,7 @@ class Logger(val logger: SLFLogger) {
|
|||
def warning(msg: => String) {
|
||||
if (warning_?) logger warn msg
|
||||
}
|
||||
|
||||
//Error
|
||||
def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
|
||||
error(t,message(fmt,arg,argN:_*))
|
||||
|
|
@ -156,53 +160,3 @@ object Logger {
|
|||
def apply(clazz: Class[_]) : Logger = apply(clazz.getName)
|
||||
def root : Logger = apply(SLFLogger.ROOT_LOGGER_NAME)
|
||||
}
|
||||
|
||||
/**
|
||||
* LoggableException is a subclass of Exception and can be used as the base exception
|
||||
* for application specific exceptions.
|
||||
* <p/>
|
||||
* It keeps track of the exception is logged or not and also stores the unique id,
|
||||
* so that it can be carried all along to the client tier and displayed to the end user.
|
||||
* The end user can call up the customer support using this number.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
// FIXME make use of LoggableException
|
||||
class LoggableException extends Exception with Logging {
|
||||
private val uniqueId = getExceptionID
|
||||
private var originalException: Option[Exception] = None
|
||||
private var isLogged = false
|
||||
|
||||
def this(baseException: Exception) = {
|
||||
this()
|
||||
originalException = Some(baseException)
|
||||
}
|
||||
|
||||
def logException = synchronized {
|
||||
if (!isLogged) {
|
||||
originalException match {
|
||||
case Some(e) => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(e))
|
||||
case None => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(this))
|
||||
}
|
||||
isLogged = true
|
||||
}
|
||||
}
|
||||
|
||||
private def getExceptionID: String = {
|
||||
val hostname: String = try {
|
||||
InetAddress.getLocalHost.getHostName
|
||||
} catch {
|
||||
case e: UnknownHostException =>
|
||||
log.error("Could not get hostname to generate loggable exception")
|
||||
"N/A"
|
||||
}
|
||||
hostname + "_" + System.currentTimeMillis
|
||||
}
|
||||
|
||||
private def getStackTraceAsString(exception: Throwable): String = {
|
||||
val sw = new StringWriter
|
||||
val pw = new PrintWriter(sw)
|
||||
exception.printStackTrace(pw)
|
||||
sw.toString
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,11 +7,9 @@ package se.scalablesolutions.akka.persistence.common
|
|||
import se.scalablesolutions.akka.stm._
|
||||
import se.scalablesolutions.akka.stm.TransactionManagement.transaction
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
// FIXME move to 'stm' package + add message with more info
|
||||
class NoTransactionInScopeException extends RuntimeException
|
||||
|
||||
class StorageException(message: String) extends RuntimeException(message)
|
||||
class StorageException(message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* Example Scala usage.
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue