Moving out ConfigurationException from akka.config to akka

This commit is contained in:
Viktor Klang 2012-05-16 17:04:13 +02:00
parent 0527f81c20
commit 95db4bfd37
17 changed files with 111 additions and 60 deletions

View file

@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicInteger
import org.junit.runner.RunWith import org.junit.runner.RunWith
import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef } import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef }
import akka.config.ConfigurationException import akka.ConfigurationException
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.{ ask, gracefulStop } import akka.pattern.{ ask, gracefulStop }
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec } import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }

View file

@ -10,7 +10,7 @@ import akka.testkit._
import akka.util.duration._ import akka.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Duration import akka.util.Duration
import akka.config.ConfigurationException import akka.ConfigurationException
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.pattern.ask import akka.pattern.ask
import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentHashMap

View file

@ -44,3 +44,10 @@ class AkkaException(message: String = "", cause: Throwable = null) extends Runti
def stackTraceToString = AkkaException.stackTraceToString(this) def stackTraceToString = AkkaException.stackTraceToString(this)
} }
/**
* This exception is thrown when Akka detects a problem with the provided configuration
*/
class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
}

View file

@ -4,7 +4,6 @@
package akka.actor package akka.actor
import akka.config.ConfigurationException
import akka.event._ import akka.event._
import akka.dispatch._ import akka.dispatch._
import akka.pattern.ask import akka.pattern.ask
@ -150,7 +149,7 @@ object ActorSystem {
final val JvmExitOnFatalError = getBoolean("akka.jvm-exit-on-fatal-error") final val JvmExitOnFatalError = getBoolean("akka.jvm-exit-on-fatal-error")
if (ConfigVersion != Version) if (ConfigVersion != Version)
throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]")
/** /**
* Returns the String representation of the Config that this Settings is backed by * Returns the String representation of the Config that this Settings is backed by

View file

@ -1,15 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.config
import akka.AkkaException
class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null);
}
class ModuleNotAvailableException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null);
}

View file

@ -33,7 +33,10 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS
} }
} }
object SystemMessage { /**
* INTERNAL API
*/
private[akka] object SystemMessage {
@tailrec @tailrec
final def size(list: SystemMessage, acc: Int = 0): Int = { final def size(list: SystemMessage, acc: Int = 0): Int = {
if (list eq null) acc else size(list.next, acc + 1) if (list eq null) acc else size(list.next, acc + 1)
@ -59,33 +62,57 @@ object SystemMessage {
* system messages is handled in a single thread only and not ever passed around, * system messages is handled in a single thread only and not ever passed around,
* hence no further synchronization is needed. * hence no further synchronization is needed.
* *
* INTERNAL API
*
* ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ * ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
*/ */
sealed trait SystemMessage extends PossiblyHarmful { private[akka] sealed trait SystemMessage extends PossiblyHarmful {
@transient @transient
var next: SystemMessage = _ var next: SystemMessage = _
} }
case class Create() extends SystemMessage // send to self from Dispatcher.register
case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart /**
case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend * INTERNAL API
case class Resume() extends SystemMessage // sent to self from ActorCell.resume */
case class Terminate() extends SystemMessage // sent to self from ActorCell.stop private[akka] case class Create() extends SystemMessage // send to self from Dispatcher.register
case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start /**
case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate * INTERNAL API
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch */
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch private[akka] case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart
/**
* INTERNAL API
*/
private[akka] case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend
/**
* INTERNAL API
*/
private[akka] case class Resume() extends SystemMessage // sent to self from ActorCell.resume
/**
* INTERNAL API
*/
private[akka] case class Terminate() extends SystemMessage // sent to self from ActorCell.stop
/**
* INTERNAL API
*/
private[akka] case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
/**
* INTERNAL API
*/
private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate
/**
* INTERNAL API
*/
private[akka] case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
/**
* INTERNAL API
*/
private[akka] case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () Unit) extends Runnable { final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () Unit) extends Runnable {
def run() { def run(): Unit =
try { try runnable.run() catch {
runnable.run() case NonFatal(e) eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
} catch { } finally cleanup()
case NonFatal(e)
eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
} finally {
cleanup()
}
}
} }
/** /**
@ -170,10 +197,16 @@ trait ExecutionContext {
def reportFailure(t: Throwable): Unit def reportFailure(t: Throwable): Unit
} }
/**
* INTERNAL API
*/
private[akka] trait LoadMetrics { self: Executor private[akka] trait LoadMetrics { self: Executor
def atFullThrottle(): Boolean def atFullThrottle(): Boolean
} }
/**
* INTERNAL API
*/
private[akka] object MessageDispatcher { private[akka] object MessageDispatcher {
val UNSCHEDULED = 0 //WARNING DO NOT CHANGE THE VALUE OF THIS: It relies on the faster init of 0 in AbstractMessageDispatcher val UNSCHEDULED = 0 //WARNING DO NOT CHANGE THE VALUE OF THIS: It relies on the faster init of 0 in AbstractMessageDispatcher
val SCHEDULED = 1 val SCHEDULED = 1
@ -228,7 +261,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
/** /**
* Creates and returns a mailbox for the given actor. * Creates and returns a mailbox for the given actor.
*/ */
protected[akka] def createMailbox(actor: ActorCell): Mailbox protected[akka] def createMailbox(actor: ActorCell): Mailbox //FIXME should this really be private[akka]?
/** /**
* Identifier of this dispatcher, corresponds to the full key * Identifier of this dispatcher, corresponds to the full key
@ -255,7 +288,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
ifSensibleToDoSoThenScheduleShutdown() ifSensibleToDoSoThenScheduleShutdown()
} }
final def execute(runnable: Runnable) { final def execute(runnable: Runnable): Unit = {
val invocation = TaskInvocation(eventStream, runnable, taskCleanup) val invocation = TaskInvocation(eventStream, runnable, taskCleanup)
addInhabitants(+1) addInhabitants(+1)
try { try {
@ -300,6 +333,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
/** /**
* If you override it, you must call it. But only ever once. See "attach" for only invocation. * If you override it, you must call it. But only ever once. See "attach" for only invocation.
*
* INTERNAL API
*/ */
protected[akka] def register(actor: ActorCell) { protected[akka] def register(actor: ActorCell) {
if (debug) actors.put(this, actor.self) if (debug) actors.put(this, actor.self)
@ -308,6 +343,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
/** /**
* If you override it, you must call it. But only ever once. See "detach" for the only invocation * If you override it, you must call it. But only ever once. See "detach" for the only invocation
*
* INTERNAL API
*/ */
protected[akka] def unregister(actor: ActorCell) { protected[akka] def unregister(actor: ActorCell) {
if (debug) actors.remove(this, actor.self) if (debug) actors.remove(this, actor.self)
@ -340,6 +377,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
* When the dispatcher no longer has any actors registered, how long will it wait until it shuts itself down, * When the dispatcher no longer has any actors registered, how long will it wait until it shuts itself down,
* defaulting to your akka configs "akka.actor.default-dispatcher.shutdown-timeout" or default specified in * defaulting to your akka configs "akka.actor.default-dispatcher.shutdown-timeout" or default specified in
* reference.conf * reference.conf
*
* INTERNAL API
*/ */
protected[akka] def shutdownTimeout: Duration protected[akka] def shutdownTimeout: Duration
@ -362,36 +401,59 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
} }
/** /**
* Will be called when the dispatcher is to queue an invocation for execution * Will be called when the dispatcher is to queue an invocation for execution
*
* INTERNAL API
*/ */
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage)
/** /**
* Will be called when the dispatcher is to queue an invocation for execution * Will be called when the dispatcher is to queue an invocation for execution
*
* INTERNAL API
*/ */
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope)
/** /**
* Suggest to register the provided mailbox for execution * Suggest to register the provided mailbox for execution
*
* INTERNAL API
*/ */
protected[akka] def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean protected[akka] def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean
// TODO check whether this should not actually be a property of the mailbox // TODO check whether this should not actually be a property of the mailbox
/**
* INTERNAL API
*/
protected[akka] def throughput: Int protected[akka] def throughput: Int
/**
* INTERNAL API
*/
protected[akka] def throughputDeadlineTime: Duration protected[akka] def throughputDeadlineTime: Duration
@inline /**
protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0 * INTERNAL API
*/
@inline protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0
/**
* INTERNAL API
*/
protected[akka] def executeTask(invocation: TaskInvocation) protected[akka] def executeTask(invocation: TaskInvocation)
/** /**
* Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached
* Must be idempotent * Must be idempotent
*
* INTERNAL API
*/ */
protected[akka] def shutdown(): Unit protected[akka] def shutdown(): Unit
} }
/**
* An ExecutorServiceConfigurator is a class that given some prerequisites and a configuration can create instances of ExecutorService
*/
abstract class ExecutorServiceConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceFactoryProvider abstract class ExecutorServiceConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceFactoryProvider
/** /**

View file

@ -4,12 +4,10 @@
package akka.event package akka.event
import akka.actor._ import akka.actor._
import akka.AkkaException import akka.{ ConfigurationException, AkkaException }
import akka.actor.ActorSystem.Settings import akka.actor.ActorSystem.Settings
import akka.config.ConfigurationException import akka.util.{ Timeout, ReentrantGuard }
import akka.util.ReentrantGuard
import akka.util.duration._ import akka.util.duration._
import akka.util.Timeout
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException

View file

@ -6,7 +6,7 @@ package akka.routing
import akka.actor._ import akka.actor._
import akka.util.Duration import akka.util.Duration
import akka.util.duration._ import akka.util.duration._
import akka.config.ConfigurationException import akka.ConfigurationException
import akka.pattern.pipe import akka.pattern.pipe
import akka.pattern.AskSupport import akka.pattern.AskSupport
import com.typesafe.config.Config import com.typesafe.config.Config

View file

@ -85,7 +85,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
/** /**
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null. * Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
* *
* @throws akka.config.ConfigurationException if no `serialization-bindings` is configured for the * @throws akka.ConfigurationException if no `serialization-bindings` is configured for the
* class of the object * class of the object
*/ */
def findSerializerFor(o: AnyRef): Serializer = o match { def findSerializerFor(o: AnyRef): Serializer = o match {

View file

@ -13,7 +13,7 @@ import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
import akka.util._ import akka.util._
import akka.util.duration._ import akka.util.duration._
import akka.config.ConfigurationException import akka.ConfigurationException
import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean }
import java.util.concurrent.TimeUnit._ import java.util.concurrent.TimeUnit._

View file

@ -6,7 +6,7 @@ package akka.cluster
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.config.ConfigurationException import akka.ConfigurationException
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import akka.actor.Address import akka.actor.Address
import akka.actor.AddressFromURIString import akka.actor.AddressFromURIString

View file

@ -11,7 +11,7 @@ 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.NonFatal import akka.util.NonFatal
import akka.config.ConfigurationException import akka.ConfigurationException
import akka.actor.ActorSystem import akka.actor.ActorSystem
class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType { class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {

View file

@ -9,7 +9,7 @@ import akka.actor._
import akka.dispatch._ import akka.dispatch._
import akka.event.{ DeathWatch, Logging, LoggingAdapter } import akka.event.{ DeathWatch, Logging, LoggingAdapter }
import akka.event.EventStream import akka.event.EventStream
import akka.config.ConfigurationException import akka.ConfigurationException
import java.util.concurrent.{ TimeoutException } import java.util.concurrent.{ TimeoutException }
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.serialization.Serialization import akka.serialization.Serialization

View file

@ -6,7 +6,7 @@ package akka.remote
import akka.actor._ import akka.actor._
import akka.routing._ import akka.routing._
import com.typesafe.config._ import com.typesafe.config._
import akka.config.ConfigurationException import akka.ConfigurationException
case class RemoteScope(node: Address) extends Scope { case class RemoteScope(node: Address) extends Scope {
def withFallback(other: Scope): Scope = this def withFallback(other: Scope): Scope = this

View file

@ -7,7 +7,7 @@ import com.typesafe.config.Config
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import java.net.InetAddress import java.net.InetAddress
import akka.config.ConfigurationException import akka.ConfigurationException
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import akka.actor.Address import akka.actor.Address
import akka.actor.AddressFromURIString import akka.actor.AddressFromURIString

View file

@ -7,7 +7,7 @@ import com.typesafe.config.Config
import akka.util.Duration import akka.util.Duration
import java.util.concurrent.TimeUnit._ import java.util.concurrent.TimeUnit._
import java.net.InetAddress import java.net.InetAddress
import akka.config.ConfigurationException import akka.ConfigurationException
class NettySettings(config: Config, val systemName: String) { class NettySettings(config: Config, val systemName: String) {

View file

@ -10,7 +10,7 @@ import akka.actor.ActorSystemImpl
import akka.actor.Deploy import akka.actor.Deploy
import akka.actor.InternalActorRef import akka.actor.InternalActorRef
import akka.actor.Props import akka.actor.Props
import akka.config.ConfigurationException import akka.ConfigurationException
import akka.remote.RemoteScope import akka.remote.RemoteScope
import akka.actor.AddressFromURIString import akka.actor.AddressFromURIString
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy