Introduced uniquely identifiable, loggable base exception: AkkaException and made use of it throught the project

This commit is contained in:
Jonas Bonér 2010-08-19 07:01:09 +02:00
parent 04af409889
commit 51b22b22e7
22 changed files with 106 additions and 91 deletions

1
.gitignore vendored
View file

@ -18,6 +18,7 @@ deploy/*.jar
data
out
logs
.#*
.codefellow
storage
.codefellow

View file

@ -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

View file

@ -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(

View file

@ -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(

View file

@ -5,6 +5,7 @@
package se.scalablesolutions.akka.camel
import se.scalablesolutions.akka.actor._
import java.net.InetSocketAddress
/**

View file

@ -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.
*

View file

@ -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.

View file

@ -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._

View file

@ -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.

View file

@ -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._

View file

@ -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.

View file

@ -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).

View 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&#233;r</a>
*/
class DataFlowVariableException(msg: String) extends RuntimeException(msg)
class DataFlowVariableException(msg: String) extends AkkaException(msg)
}

View file

@ -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 {

View file

@ -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.

View file

@ -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.

View file

@ -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)

View file

@ -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.

View 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&#233;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"
}
}

View file

@ -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&#233;r</a>
*/

View file

@ -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&#233;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
}
}

View file

@ -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.