Add @SerialVersionUID to Serializable classes. See #1786

* @SerialVersionUID(1L)
* UntypedActorFactory serialization, and test
* Removed Serializable from Serializers
This commit is contained in:
Patrik Nordwall 2012-02-06 14:19:59 +01:00
parent d7435547ff
commit 9c8c0d42c5
14 changed files with 51 additions and 12 deletions

View file

@ -203,6 +203,10 @@ object VerifySerializabilitySpec {
}
}
class FooUntypedActor extends UntypedActor {
def onReceive(message: Any) {}
}
class NonSerializableActor(system: ActorSystem) extends Actor {
def receive = {
case s: String sender ! s
@ -210,6 +214,7 @@ object VerifySerializabilitySpec {
}
}
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class VerifySerializabilitySpec extends AkkaSpec(VerifySerializabilitySpec.conf) {
import VerifySerializabilitySpec._
implicit val timeout = Timeout(5 seconds)
@ -221,17 +226,28 @@ class VerifySerializabilitySpec extends AkkaSpec(VerifySerializabilitySpec.conf)
"verify creators" in {
val a = system.actorOf(Props[FooActor])
intercept[NotSerializableException] {
Await.result(a ? new AnyRef, timeout.duration)
}
system stop a
val b = system.actorOf(Props(new FooActor))
system stop b
val c = system.actorOf(Props().withCreator(new UntypedActorFactory {
def create() = new FooUntypedActor
}))
system stop c
intercept[java.io.NotSerializableException] {
val d = system.actorOf(Props(new NonSerializableActor(system)))
}
}
"verify messages" in {
val a = system.actorOf(Props[FooActor])
Await.result(a ? "pigdog", timeout.duration) must be("pigdog")
intercept[java.io.NotSerializableException] {
val b = system.actorOf(Props(new NonSerializableActor(system)))
intercept[NotSerializableException] {
Await.result(a ? new AnyRef, timeout.duration)
}
system stop a
}

View file

@ -33,6 +33,7 @@ object AkkaException {
* <li>toLongString which also includes the stack trace</li>
* </ul>
*/
@SerialVersionUID(1L)
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
val uuid = "%s_%s".format(AkkaException.hostname, newUuid)

View file

@ -38,6 +38,7 @@ object ActorPath {
* is sorted by path elements FROM RIGHT TO LEFT, where RootActorPath >
* ChildActorPath in case the number of elements is different.
*/
@SerialVersionUID(1L)
sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
/**
* The Address under which this path can be reached; walks up the tree to
@ -108,6 +109,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).
*/
@SerialVersionUID(1L)
final case class RootActorPath(address: Address, name: String = "/") extends ActorPath {
def parent: ActorPath = this
@ -130,6 +132,7 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
}
}
@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

@ -333,6 +333,7 @@ private[akka] class LocalActorRef private[akka] (
/**
* Memento pattern for serializing ActorRefs transparently
*/
@SerialVersionUID(1L)
case class SerializedActorRef private (path: String) {
import akka.serialization.Serialization.currentSystem
@ -397,6 +398,7 @@ private[akka] object MinimalActorRef {
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
private[akka] object DeadLetterActorRef {
@SerialVersionUID(1L)
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
@throws(classOf[java.io.ObjectStreamException])
private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters

View file

@ -25,6 +25,7 @@ import akka.util.ReflectiveAccess
* context.actorOf(someProps, "someName", Deploy(scope = RemoteScope("someOtherNodeName")))
* }}}
*/
@SerialVersionUID(1L)
final case class Deploy(
path: String = "",
config: Config = ConfigFactory.empty,
@ -62,6 +63,7 @@ trait Scope {
def withFallback(other: Scope): Scope
}
@SerialVersionUID(1L)
case object LocalScope extends Scope {
/**
* Java API
@ -74,6 +76,7 @@ case object LocalScope extends Scope {
/**
* This is the default value and as such allows overrides.
*/
@SerialVersionUID(1L)
case object NoScopeGiven extends Scope {
def withFallback(other: Scope): Scope = other
}

View file

@ -103,6 +103,7 @@ object Props {
* Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..));
* }}}
*/
@SerialVersionUID(1L)
case class Props(
creator: () Actor = Props.defaultCreator,
dispatcher: String = Dispatchers.DefaultDispatcherId,

View file

@ -429,6 +429,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.
*/
@SerialVersionUID(1L)
case class TypedProps[T <: AnyRef] protected[TypedProps] (
interfaces: Seq[Class[_]],
creator: () T,

View file

@ -158,4 +158,4 @@ abstract class UntypedActor extends Actor {
/**
* Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'.
*/
trait UntypedActorFactory extends Creator[Actor]
trait UntypedActorFactory extends Creator[Actor] with Serializable

View file

@ -168,7 +168,7 @@ object MessageDispatcher {
implicit def defaultDispatcher(implicit system: ActorSystem): MessageDispatcher = system.dispatcher
}
abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Serializable with Executor with ExecutionContext {
abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) extends AbstractMessageDispatcher with Executor with ExecutionContext {
import MessageDispatcher._
import AbstractMessageDispatcher.{ inhabitantsUpdater, shutdownScheduleUpdater }

View file

@ -299,6 +299,7 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
* from lower-precendence sources. The decision whether or not to create a
* router is taken in the LocalActorRefProvider based on Props.
*/
@SerialVersionUID(1L)
case object NoRouter extends RouterConfig {
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
override def withFallback(other: RouterConfig): RouterConfig = other
@ -315,6 +316,7 @@ case object FromConfig extends RouterConfig {
/**
* Java API: Router configuration which has no default, i.e. external configuration is required.
*/
@SerialVersionUID(1L)
case class FromConfig() extends RouterConfig {
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
@ -345,6 +347,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]]
*/
@SerialVersionUID(1L)
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with RoundRobinLike {
@ -424,6 +427,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]]
*/
@SerialVersionUID(1L)
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with RandomLike {
@ -509,6 +513,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]]
*/
@SerialVersionUID(1L)
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with SmallestMailboxLike {
@ -653,6 +658,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]]
*/
@SerialVersionUID(1L)
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with BroadcastLike {
@ -724,6 +730,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]]
*/
@SerialVersionUID(1L)
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration,
override val resizer: Option[Resizer] = None)
extends RouterConfig with ScatterGatherFirstCompletedLike {

View file

@ -7,7 +7,7 @@ package akka.serialization
import akka.actor.Actor
/**
* trait Serializer extends scala.Serializable {
* trait Serializer {
* @volatile
* var classLoader: Option[ClassLoader] = None
* def deepClone(obj: AnyRef): AnyRef = fromBinary(toBinary(obj), Some(obj.getClass))
@ -74,7 +74,7 @@ trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T]
* }
* </pre>
*/
trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable {
trait StatelessActorFormat[T <: Actor] extends Format[T] {
def fromBinary(bytes: Array[Byte], act: T) = act
def toBinary(ac: T) = Array.empty[Byte]
@ -94,7 +94,7 @@ trait StatelessActorFormat[T <: Actor] extends Format[T] with scala.Serializable
* }
* </pre>
*/
trait SerializerBasedActorFormat[T <: Actor] extends Format[T] with scala.Serializable {
trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
val serializer: Serializer
def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.getClass)).asInstanceOf[T]

View file

@ -10,7 +10,7 @@ import akka.util.ClassLoaderObjectInputStream
/**
* A Serializer represents a bimap between an object and an array of bytes representing that object
*/
trait Serializer extends scala.Serializable {
trait Serializer {
/**
* Completely unique value to identify this implementation of Serializer, used to optimize network traffic
* Values from 0 to 16 is reserved for Akka internal usage

View file

@ -8,6 +8,7 @@ import java.util.concurrent.TimeUnit
import TimeUnit._
import java.lang.{ Double JDouble }
@SerialVersionUID(1L)
case class Deadline private (time: Duration) {
def +(other: Duration): Deadline = copy(time = time + other)
def -(other: Duration): Deadline = copy(time = time - other)
@ -232,6 +233,7 @@ object Duration {
* val d3 = d2 + 1.millisecond
* </pre>
*/
@SerialVersionUID(1L)
abstract class Duration extends Serializable with Ordered[Duration] {
def length: Long
def unit: TimeUnit
@ -276,6 +278,7 @@ object FiniteDuration {
}
}
@SerialVersionUID(1L)
class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
import Duration._
@ -525,6 +528,7 @@ class DurationDouble(d: Double) {
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS))
}
@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

@ -222,6 +222,7 @@ trait RemoteRef extends ActorRefScope {
* Remote ActorRef that is used when referencing the Actor on a different node than its "home" node.
* This reference is network-aware (remembers its origin) and immutable.
*/
@SerialVersionUID(1L)
private[akka] class RemoteActorRef private[akka] (
val provider: RemoteActorRefProvider,
remote: RemoteTransport,