Merge branch 'master' into wip-fix-io-√

This commit is contained in:
Viktor Klang 2012-08-06 17:29:21 +02:00
commit 7932aaa793
12 changed files with 53 additions and 23 deletions

View file

@ -11,7 +11,7 @@ package akka
* <li>toString that includes exception name, message and uuid</li>
* </ul>
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
class AkkaException(message: String, cause: Throwable) extends RuntimeException(message, cause) with Serializable {
def this(msg: String) = this(msg, null)

View file

@ -29,6 +29,7 @@ trait NoSerializationVerificationNeeded
/**
* Internal use only
*/
@SerialVersionUID(1L)
private[akka] case class Failed(cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful
abstract class PoisonPill extends AutoReceivedMessage with PossiblyHarmful
@ -36,6 +37,7 @@ abstract class PoisonPill extends AutoReceivedMessage with PossiblyHarmful
/**
* A message all Actors will understand, that when processed will terminate the Actor permanently.
*/
@SerialVersionUID(1L)
case object PoisonPill extends PoisonPill {
/**
* Java API: get the singleton instance
@ -48,6 +50,7 @@ abstract class Kill extends AutoReceivedMessage with PossiblyHarmful
* A message all Actors will understand, that when processed will make the Actor throw an ActorKilledException,
* which will trigger supervision.
*/
@SerialVersionUID(1L)
case object Kill extends Kill {
/**
* Java API: get the singleton instance
@ -58,6 +61,7 @@ case object Kill extends Kill {
/**
* When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated.
*/
@SerialVersionUID(1L)
case class Terminated(@BeanProperty actor: ActorRef)(@BeanProperty val existenceConfirmed: Boolean) extends AutoReceivedMessage
abstract class ReceiveTimeout extends PossiblyHarmful
@ -66,6 +70,7 @@ abstract class ReceiveTimeout extends PossiblyHarmful
* When using ActorContext.setReceiveTimeout, the singleton instance of ReceiveTimeout will be sent
* to the Actor when there hasn't been any message for that long.
*/
@SerialVersionUID(1L)
case object ReceiveTimeout extends ReceiveTimeout {
/**
* Java API: get the singleton instance
@ -83,22 +88,26 @@ sealed trait SelectionPath extends AutoReceivedMessage
/**
* Internal use only
*/
@SerialVersionUID(1L)
private[akka] case class SelectChildName(name: String, next: Any) extends SelectionPath
/**
* Internal use only
*/
@SerialVersionUID(1L)
private[akka] case class SelectChildPattern(pattern: Pattern, next: Any) extends SelectionPath
/**
* Internal use only
*/
@SerialVersionUID(1L)
private[akka] case class SelectParent(next: Any) extends SelectionPath
/**
* IllegalActorStateException is thrown when a core invariant in the Actor implementation has been violated.
* For instance, if you try to create an Actor that doesn't extend Actor.
*/
@SerialVersionUID(1L)
class IllegalActorStateException private[akka] (message: String, cause: Throwable = null)
extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
@ -107,6 +116,7 @@ class IllegalActorStateException private[akka] (message: String, cause: Throwabl
/**
* ActorKilledException is thrown when an Actor receives the akka.actor.Kill message
*/
@SerialVersionUID(1L)
class ActorKilledException private[akka] (message: String, cause: Throwable)
extends AkkaException(message, cause)
with NoStackTrace {
@ -117,11 +127,13 @@ class ActorKilledException private[akka] (message: String, cause: Throwable)
* An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name
* which doesn't validate.
*/
@SerialVersionUID(1L)
class InvalidActorNameException(message: String) extends AkkaException(message)
/**
* An ActorInitializationException is thrown when the the initialization logic for an Actor fails.
*/
@SerialVersionUID(1L)
class ActorInitializationException private[akka] (val actor: ActorRef, message: String, cause: Throwable)
extends AkkaException(message, cause) {
def this(msg: String) = this(null, msg, null)
@ -136,6 +148,7 @@ class ActorInitializationException private[akka] (val actor: ActorRef, message:
* @param origCause is the exception which caused the restart in the first place
* @param msg is the message which was optionally passed into preRestart()
*/
@SerialVersionUID(1L)
class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, val origCause: Throwable, val msg: Option[Any])
extends ActorInitializationException(actor, "exception in preRestart(" + origCause.getClass + ", " + msg.map(_.getClass) + ")", cause) {
}
@ -148,6 +161,7 @@ class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, val
* @param cause is the exception thrown by that actor within preRestart()
* @param origCause is the exception which caused the restart in the first place
*/
@SerialVersionUID(1L)
class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, val origCause: Throwable)
extends ActorInitializationException(actor, "exception post restart (" + origCause.getClass + ")", cause) {
}
@ -157,6 +171,7 @@ class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, val
* Technically it's only "null" which is an InvalidMessageException but who knows,
* there might be more of them in the future, or not.
*/
@SerialVersionUID(1L)
class InvalidMessageException private[akka] (message: String, cause: Throwable = null)
extends AkkaException(message, cause) {
def this(msg: String) = this(msg, null)
@ -166,6 +181,7 @@ class InvalidMessageException private[akka] (message: String, cause: Throwable =
* A DeathPactException is thrown by an Actor that receives a Terminated(someActor) message
* that it doesn't handle itself, effectively crashing the Actor and escalating to the supervisor.
*/
@SerialVersionUID(1L)
case class DeathPactException private[akka] (dead: ActorRef)
extends AkkaException("Monitored actor [" + dead + "] terminated")
with NoStackTrace
@ -174,11 +190,13 @@ case class DeathPactException private[akka] (dead: ActorRef)
* When an InterruptedException is thrown inside an Actor, it is wrapped as an ActorInterruptedException as to
* avoid cascading interrupts to other threads than the originally interrupted one.
*/
@SerialVersionUID(1L)
class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
/**
* This message is published to the EventStream whenever an Actor receives a message it doesn't understand
*/
@SerialVersionUID(1L)
case class UnhandledMessage(@BeanProperty message: Any, @BeanProperty sender: ActorRef, @BeanProperty recipient: ActorRef)
/**
@ -191,12 +209,14 @@ object Status {
/**
* This class/message type is preferably used to indicate success of some operation performed.
*/
@SerialVersionUID(1L)
case class Success(status: AnyRef) extends Status
/**
* This class/message type is preferably used to indicate failure of some operation performed.
* As an example, it is used to signal failure with AskSupport is used (ask/?).
*/
@SerialVersionUID(1L)
case class Failure(cause: Throwable) extends Status
}
@ -224,6 +244,7 @@ object Actor {
/**
* emptyBehavior is a Receive-expression that matches no messages at all, ever.
*/
@SerialVersionUID(1L)
object emptyBehavior extends Receive {
def isDefinedAt(x: Any) = false
def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()")

View file

@ -32,7 +32,7 @@ object ActorPath {
* is sorted by path elements FROM RIGHT TO LEFT, where RootActorPath >
* ChildActorPath in case the number of elements is different.
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
/**
* The Address under which this path can be reached; walks up the tree to
@ -103,7 +103,7 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
* Root of the hierarchy of ActorPaths. There is exactly root per ActorSystem
* and node (for remote-enabled or clustered systems).
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
final case class RootActorPath(address: Address, name: String = "/") extends ActorPath {
override def parent: ActorPath = this
@ -126,7 +126,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
}
}
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
final class ChildActorPath(val parent: ActorPath, val name: String) extends ActorPath {
if (name.indexOf('/') != -1) throw new IllegalArgumentException("/ is a path separator and is not legal in ActorPath names: [%s]" format name)

View file

@ -350,7 +350,7 @@ private[akka] class LocalActorRef private[akka] (
* Memento pattern for serializing ActorRefs transparently
* INTERNAL API
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
private[akka] case class SerializedActorRef private (path: String) {
import akka.serialization.JavaSerializer.currentSystem
@ -405,10 +405,11 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
* When a message is sent to an Actor that is terminated before receiving the message, it will be sent as a DeadLetter
* to the ActorSystem's EventStream
*/
@SerialVersionUID(1L)
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
private[akka] object DeadLetterActorRef {
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException])
private def readResolve(): AnyRef = JavaSerializer.currentSystem.value.deadLetters

View file

@ -16,6 +16,7 @@ import annotation.tailrec
* for example a remote transport would want to associate additional
* information with an address, then this must be done externally.
*/
@SerialVersionUID(1L)
final case class Address private (protocol: String, system: String, host: Option[String], port: Option[Int]) {
def this(protocol: String, system: String) = this(protocol, system, None, None)

View file

@ -27,7 +27,7 @@ import annotation.tailrec
* context.actorOf(someProps, "someName", Deploy(scope = RemoteScope("someOtherNodeName")))
* }}}
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
final case class Deploy(
path: String = "",
config: Config = ConfigFactory.empty,
@ -76,7 +76,7 @@ trait Scope {
def withFallback(other: Scope): Scope
}
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
abstract class LocalScope extends Scope
//FIXME docs
@ -92,7 +92,7 @@ case object LocalScope extends LocalScope {
/**
* This is the default value and as such allows overrides.
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
abstract class NoScopeGiven extends Scope
case object NoScopeGiven extends NoScopeGiven {
def withFallback(other: Scope): Scope = other

View file

@ -113,7 +113,7 @@ object Props {
* Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..));
* }}}
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed when SI-4804 is fixed
@SerialVersionUID(1L)
case class Props(
creator: () Actor = Props.defaultCreator,
dispatcher: String = Dispatchers.DefaultDispatcherId,

View file

@ -494,7 +494,7 @@ object TypedProps {
* TypedProps is a TypedActor configuration object, that is thread safe and fully sharable.
* It's used in TypedActorFactory.typedActorOf to configure a TypedActor instance.
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class TypedProps[T <: AnyRef] protected[TypedProps] (
interfaces: Seq[Class[_]],
creator: () T,

View file

@ -310,7 +310,7 @@ trait Router extends Actor {
* INTERNAL API
*/
private object Router {
@SerialVersionUID(1L)
case object Resize
val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() {
@ -325,6 +325,7 @@ private object Router {
*
* Router implementations may choose to handle this message differently.
*/
@SerialVersionUID(1L)
case class Broadcast(message: Any)
/**
@ -333,6 +334,7 @@ case class Broadcast(message: Any)
* about what routees the router is routing over.
*/
abstract class CurrentRoutees
@SerialVersionUID(1L)
case object CurrentRoutees extends CurrentRoutees {
/**
* Java API: get the singleton instance
@ -343,6 +345,7 @@ case object CurrentRoutees extends CurrentRoutees {
/**
* Message used to carry information about what routees the router is currently using.
*/
@SerialVersionUID(1L)
case class RouterRoutees(routees: Iterable[ActorRef])
/**
@ -351,6 +354,7 @@ case class RouterRoutees(routees: Iterable[ActorRef])
* sender should match the sender of the original request, but e.g. the scatter-
* gather router needs to receive the replies with an AskActorRef instead.
*/
@SerialVersionUID(1L)
case class Destination(sender: ActorRef, recipient: ActorRef)
/**
@ -359,7 +363,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
* from lower-precedence sources. The decision whether or not to create a
* router is taken in the LocalActorRefProvider based on Props.
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
abstract class NoRouter extends RouterConfig
case object NoRouter extends NoRouter {
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null // FIXME, null, really??
@ -391,7 +395,7 @@ case object FromConfig extends FromConfig {
* This can be used when the dispatcher to be used for the head Router needs to be configured
* (defaults to default-dispatcher).
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig
with Serializable {
@ -462,7 +466,7 @@ object RoundRobinRouter {
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
@ -581,7 +585,7 @@ object RandomRouter {
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
@ -707,7 +711,7 @@ object SmallestMailboxRouter {
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
@ -907,7 +911,7 @@ object BroadcastRouter {
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy)
@ -1018,7 +1022,7 @@ object ScatterGatherFirstCompletedRouter {
* @param routees string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]]
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration,
override val resizer: Option[Resizer] = None,
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
@ -1132,6 +1136,7 @@ case object DefaultResizer {
}
//FIXME DOCUMENT ME
@SerialVersionUID(1L)
case class DefaultResizer(
/**
* The fewest number of routees the router should ever have.

View file

@ -10,7 +10,7 @@ import java.util.concurrent.TimeUnit
import java.lang.{ Double JDouble }
import scala.concurrent.util.Duration
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
@SerialVersionUID(1L)
case class Timeout(duration: Duration) {
def this(timeout: Long) = this(Duration(timeout, TimeUnit.MILLISECONDS))
def this(length: Long, unit: TimeUnit) = this(Duration(length, unit))

View file

@ -116,7 +116,8 @@ akka {
# (I&O) Increase this if you want to be able to send messages with large payloads
message-frame-size = 1 MiB
# (O) Timeout duration
# (O) Sets the connectTimeoutMillis of all outbound connections,
# i.e. how long a connect may take until it is timed out
connection-timeout = 120s
# (I) Sets the size of the connection backlog

View file

@ -67,7 +67,7 @@ object AkkaBuild extends Build {
sphinxLatex <<= sphinxLatex in LocalProject(docs.id) map identity,
sphinxPdf <<= sphinxPdf in LocalProject(docs.id) map identity
),
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, /*akkaSbtPlugin,*/ samples, tutorials, osgi, osgiAries, docs)
aggregate = Seq(actor, testkit, actorTests, remote, remoteTests, camel, cluster, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, samples, tutorials, osgi, osgiAries, docs)
)
lazy val actor = Project(
@ -263,6 +263,7 @@ object AkkaBuild extends Build {
base = file("akka-sbt-plugin"),
settings = defaultSettings ++ Seq(
sbtPlugin := true,
scalacOptions in Compile := Seq("-encoding", "UTF-8", "-deprecation", "-unchecked"),
scalaVersion := "2.9.1"
)
)
@ -405,7 +406,7 @@ object AkkaBuild extends Build {
// compile options
scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-deprecation", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Ywarn-adapted-args"),
javacOptions in Compile ++= Seq("-Xlint:unchecked", "-Xlint:deprecation"),
javacOptions in Compile ++= Seq("-target", "1.6", "-Xlint:unchecked", "-Xlint:deprecation"),
ivyLoggingLevel in ThisBuild := UpdateLogging.Quiet,