pekko/akka-actor/src/main/scala/akka/serialization/Serialization.scala

181 lines
7.2 KiB
Scala
Raw Normal View History

/**
2012-01-19 18:21:06 +01:00
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.serialization
import akka.AkkaException
import scala.util.DynamicVariable
import com.typesafe.config.Config
import akka.actor.{ Extension, ExtendedActorSystem, Address, DynamicAccess }
import akka.event.Logging
import java.util.concurrent.ConcurrentHashMap
import scala.util.control.NonFatal
import scala.collection.mutable.ArrayBuffer
import java.io.NotSerializableException
2011-11-24 18:53:18 +01:00
object Serialization {
/**
* Tuple that represents mapping from Class to Serializer
*/
type ClassSerializer = (Class[_], Serializer)
/**
* This holds a reference to the current transport address to be inserted
* into local actor refs during serialization.
*/
val currentTransportAddress = new DynamicVariable[Address](null)
class Settings(val config: Config) {
2011-11-24 18:53:18 +01:00
import scala.collection.JavaConverters._
import config._
val Serializers: Map[String, String] = configToMap(getConfig("akka.actor.serializers"))
2011-11-24 18:53:18 +01:00
val SerializationBindings: Map[String, String] = configToMap(getConfig("akka.actor.serialization-bindings"))
private def configToMap(cfg: Config): Map[String, String] =
cfg.root.unwrapped.asScala.toMap.map { case (k, v) (k, v.toString) }
2011-11-24 18:53:18 +01:00
}
}
/**
* Serialization module. Contains methods for serialization and deserialization as well as
* locating a Serializer for a particular class as defined in the mapping in the configuration.
*/
class Serialization(val system: ExtendedActorSystem) extends Extension {
2011-11-24 18:53:18 +01:00
import Serialization._
val settings = new Settings(system.settings.config)
val log = Logging(system, getClass.getName)
2011-12-29 16:11:56 +01:00
/**
* Serializes the given AnyRef/java.lang.Object according to the Serialization configuration
* to either an Array of Bytes or an Exception if one was thrown.
*/
2012-02-09 19:26:02 +01:00
def serialize(o: AnyRef): Either[Throwable, Array[Byte]] =
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750 - PropertyMaster is the only place in Akka which calls ClassLoader.getClass (apart from kernel, which might be special) - all PropertyMaster methods (there are only three) take a ClassManifest of what is to be constructed, and they verify that the obtained object is actually compatible with the required type Other stuff: - noticed that I had forgotten to change to ExtendedActorSystem when constructing Extensions by ExtensionKey (damn you, reflection!) - moved Serializer.currentSystem into JavaSerializer, because that’s the only one needing it (it’s only used in readResolve() methods) - Serializers are constructed now with one-arg constructor taking ExtendedActorSystem (if that exists, otherwise no-arg as before), to allow JavaSerializer to do its magic; possibly necessary for others as well - Removed all Option[ClassLoader] signatures - made it so that the ActorSystem will try context class loader, then the class loader which loaded the class actually calling into ActorSystem.apply, then the loader which loaded ActorSystemImpl - for the second of the above I added a (reflectively accessed hopefully safe) facility for getting caller Class[_] objects by using sun.reflect.Reflection; this is optional an defaults to None, e.g. on Android, which means that getting the caller’s classloader is done on a best effort basis (there’s nothing we can do because a StackTrace does not contain actual Class[_] objects). - refactored DurableMailbox to contain the owner val and use that instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
try Right(findSerializerFor(o).toBinary(o))
2012-02-09 19:26:02 +01:00
catch { case NonFatal(e) Left(e) }
2011-12-29 16:11:56 +01:00
/**
* Deserializes the given array of bytes using the specified serializer id,
* using the optional type hint to the Serializer and the optional ClassLoader ot load it into.
* Returns either the resulting object or an Exception if one was thrown.
*/
def deserialize(bytes: Array[Byte],
serializerId: Int,
2012-02-09 19:26:02 +01:00
clazz: Option[Class[_]]): Either[Throwable, AnyRef] =
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750 - PropertyMaster is the only place in Akka which calls ClassLoader.getClass (apart from kernel, which might be special) - all PropertyMaster methods (there are only three) take a ClassManifest of what is to be constructed, and they verify that the obtained object is actually compatible with the required type Other stuff: - noticed that I had forgotten to change to ExtendedActorSystem when constructing Extensions by ExtensionKey (damn you, reflection!) - moved Serializer.currentSystem into JavaSerializer, because that’s the only one needing it (it’s only used in readResolve() methods) - Serializers are constructed now with one-arg constructor taking ExtendedActorSystem (if that exists, otherwise no-arg as before), to allow JavaSerializer to do its magic; possibly necessary for others as well - Removed all Option[ClassLoader] signatures - made it so that the ActorSystem will try context class loader, then the class loader which loaded the class actually calling into ActorSystem.apply, then the loader which loaded ActorSystemImpl - for the second of the above I added a (reflectively accessed hopefully safe) facility for getting caller Class[_] objects by using sun.reflect.Reflection; this is optional an defaults to None, e.g. on Android, which means that getting the caller’s classloader is done on a best effort basis (there’s nothing we can do because a StackTrace does not contain actual Class[_] objects). - refactored DurableMailbox to contain the owner val and use that instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
try Right(serializerByIdentity(serializerId).fromBinary(bytes, clazz))
2012-02-09 19:26:02 +01:00
catch { case NonFatal(e) Left(e) }
2011-12-29 16:11:56 +01:00
/**
* Deserializes the given array of bytes using the specified type to look up what Serializer should be used.
* You can specify an optional ClassLoader to load the object into.
* Returns either the resulting object or an Exception if one was thrown.
*/
2012-02-09 19:26:02 +01:00
def deserialize(bytes: Array[Byte], clazz: Class[_]): Either[Throwable, AnyRef] =
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750 - PropertyMaster is the only place in Akka which calls ClassLoader.getClass (apart from kernel, which might be special) - all PropertyMaster methods (there are only three) take a ClassManifest of what is to be constructed, and they verify that the obtained object is actually compatible with the required type Other stuff: - noticed that I had forgotten to change to ExtendedActorSystem when constructing Extensions by ExtensionKey (damn you, reflection!) - moved Serializer.currentSystem into JavaSerializer, because that’s the only one needing it (it’s only used in readResolve() methods) - Serializers are constructed now with one-arg constructor taking ExtendedActorSystem (if that exists, otherwise no-arg as before), to allow JavaSerializer to do its magic; possibly necessary for others as well - Removed all Option[ClassLoader] signatures - made it so that the ActorSystem will try context class loader, then the class loader which loaded the class actually calling into ActorSystem.apply, then the loader which loaded ActorSystemImpl - for the second of the above I added a (reflectively accessed hopefully safe) facility for getting caller Class[_] objects by using sun.reflect.Reflection; this is optional an defaults to None, e.g. on Android, which means that getting the caller’s classloader is done on a best effort basis (there’s nothing we can do because a StackTrace does not contain actual Class[_] objects). - refactored DurableMailbox to contain the owner val and use that instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
try Right(serializerFor(clazz).fromBinary(bytes, Some(clazz)))
2012-02-09 19:26:02 +01:00
catch { case NonFatal(e) Left(e) }
2011-12-29 16:11:56 +01:00
/**
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
*
* @throws akka.ConfigurationException if no `serialization-bindings` is configured for the
* class of the object
2011-12-29 16:11:56 +01:00
*/
def findSerializerFor(o: AnyRef): Serializer = o match {
2011-07-26 18:33:59 +12:00
case null NullSerializer
case other serializerFor(other.getClass)
}
2011-12-29 16:11:56 +01:00
/**
* Returns the configured Serializer for the given Class. The configured Serializer
* is used if the configured class `isAssignableFrom` from the `clazz`, i.e.
* the configured class is a super class or implemented interface. In case of
* ambiguity it is primarily using the most specific configured class,
* and secondly the entry configured first.
*
* @throws java.io.NotSerializableException if no `serialization-bindings` is configured for the class
2011-12-29 16:11:56 +01:00
*/
2012-02-04 17:35:39 +01:00
def serializerFor(clazz: Class[_]): Serializer =
serializerMap.get(clazz) match {
case null
// bindings are ordered from most specific to least specific
def unique(possibilities: Seq[(Class[_], Serializer)]): Boolean =
possibilities.size == 1 ||
(possibilities map (_._1) forall (_ isAssignableFrom possibilities(0)._1)) ||
(possibilities map (_._2) forall (_ == possibilities(0)._2))
val ser = bindings filter { _._1 isAssignableFrom clazz } match {
case Seq()
throw new NotSerializableException("No configured serialization-bindings for class [%s]" format clazz.getName)
case possibilities
if (!unique(possibilities))
log.warning("Multiple serializers found for " + clazz + ", choosing first: " + possibilities)
possibilities(0)._2
}
serializerMap.putIfAbsent(clazz, ser) match {
case null log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName); ser
case some some
2012-02-04 17:35:39 +01:00
}
case ser ser
}
/**
2012-02-09 19:26:02 +01:00
* Tries to load the specified Serializer by the fully-qualified name; the actual
* loading is performed by the systems [[akka.actor.DynamicAccess]].
*/
def serializerOf(serializerFQN: String): Either[Throwable, Serializer] =
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq(classOf[ExtendedActorSystem] -> system)).fold(_
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Seq()), Right(_))
/**
* A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer)
* By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer
*/
private val serializers: Map[String, Serializer] =
for ((k: String, v: String) settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity)
2011-06-29 21:25:17 +02:00
/**
* bindings is a Seq of tuple representing the mapping from Class to Serializer.
* It is primarily ordered by the most specific classes first, and secondly in the configured order.
*/
private[akka] val bindings: Seq[ClassSerializer] = {
val configuredBindings = for ((k: String, v: String) settings.SerializationBindings if v != "none") yield {
val c = system.dynamicAccess.getClassFor[Any](k).fold(throw _, identity[Class[_]])
(c, serializers(v))
}
sort(configuredBindings)
}
2011-06-29 21:25:17 +02:00
/**
* Sort so that subtypes always precede their supertypes, but without
* obeying any order between unrelated subtypes (insert sort).
*/
private def sort(in: Iterable[ClassSerializer]): Seq[ClassSerializer] =
(new ArrayBuffer[ClassSerializer](in.size) /: in) { (buf, ca)
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
case -1 buf append ca
case x buf insert (x, ca)
}
buf
}
/**
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer
* to be used for that class.
*/
private val serializerMap: ConcurrentHashMap[Class[_], Serializer] =
(new ConcurrentHashMap[Class[_], Serializer] /: bindings) { case (map, (c, s)) map.put(c, s); map }
/**
* Maps from a Serializer Identity (Int) to a Serializer instance (optimization)
*/
val serializerByIdentity: Map[Int, Serializer] =
2011-07-26 18:33:59 +12:00
Map(NullSerializer.identifier -> NullSerializer) ++ serializers map { case (_, v) (v.identifier, v) }
}