2018-10-29 17:19:37 +08:00
|
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
2011-06-07 06:36:21 +05:30
|
|
|
|
*/
|
|
|
|
|
|
|
2011-06-22 09:59:00 +02:00
|
|
|
|
package akka.serialization
|
|
|
|
|
|
|
2011-11-29 11:50:22 +01:00
|
|
|
|
import com.typesafe.config.Config
|
2013-03-25 08:42:48 +01:00
|
|
|
|
import akka.actor._
|
2016-10-28 17:22:36 +02:00
|
|
|
|
import akka.event.{ LogMarker, Logging, LoggingAdapter }
|
2012-02-10 12:45:22 +01:00
|
|
|
|
import java.util.concurrent.ConcurrentHashMap
|
2016-10-28 14:52:17 +02:00
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
|
import scala.collection.mutable.ArrayBuffer
|
|
|
|
|
|
import java.io.NotSerializableException
|
2016-10-28 14:52:17 +02:00
|
|
|
|
|
|
|
|
|
|
import scala.util.{ DynamicVariable, Failure, Try }
|
2012-10-30 15:08:41 +01:00
|
|
|
|
import scala.collection.immutable
|
2013-06-02 14:53:31 +02:00
|
|
|
|
import scala.util.control.NonFatal
|
2015-05-28 18:42:22 +02:00
|
|
|
|
import scala.util.Success
|
2016-05-26 11:58:13 +02:00
|
|
|
|
import java.nio.ByteBuffer
|
|
|
|
|
|
import java.util.concurrent.atomic.AtomicReference
|
2016-10-28 14:52:17 +02:00
|
|
|
|
|
2016-05-26 11:58:13 +02:00
|
|
|
|
import scala.annotation.tailrec
|
2016-06-04 21:44:13 +02:00
|
|
|
|
import java.util.NoSuchElementException
|
2017-04-25 20:52:45 +02:00
|
|
|
|
import akka.annotation.InternalApi
|
2018-11-22 16:18:10 +01:00
|
|
|
|
import akka.util.ccompat._
|
2011-06-07 06:36:21 +05:30
|
|
|
|
|
2019-04-03 13:13:44 +02:00
|
|
|
|
@ccompatUsedUntil213
|
2011-11-24 18:53:18 +01:00
|
|
|
|
object Serialization {
|
2012-02-06 21:12:26 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Tuple that represents mapping from Class to Serializer
|
|
|
|
|
|
*/
|
|
|
|
|
|
type ClassSerializer = (Class[_], Serializer)
|
|
|
|
|
|
|
2012-01-27 12:14:28 +01:00
|
|
|
|
/**
|
2018-05-21 16:59:04 +02:00
|
|
|
|
* INTERNAL API: This holds a reference to the current transport serialization information used for
|
|
|
|
|
|
* serializing local actor refs, or if serializer library e.g. custom serializer/deserializer in
|
|
|
|
|
|
* Jackson need access to the current `ActorSystem`.
|
2012-01-27 12:14:28 +01:00
|
|
|
|
*/
|
2018-05-21 16:59:04 +02:00
|
|
|
|
@InternalApi private[akka] val currentTransportInformation = new DynamicVariable[Information](null)
|
2012-01-27 12:14:28 +01:00
|
|
|
|
|
2011-11-29 11:50:22 +01:00
|
|
|
|
class Settings(val config: Config) {
|
2016-09-09 09:01:15 +02:00
|
|
|
|
val Serializers: Map[String, String] = configToMap(config.getConfig("akka.actor.serializers"))
|
|
|
|
|
|
val SerializationBindings: Map[String, String] = {
|
|
|
|
|
|
val defaultBindings = config.getConfig("akka.actor.serialization-bindings")
|
2018-01-10 19:28:51 +09:00
|
|
|
|
val bindings = {
|
2016-09-09 09:01:15 +02:00
|
|
|
|
if (config.getBoolean("akka.actor.enable-additional-serialization-bindings") ||
|
2019-03-11 10:38:24 +01:00
|
|
|
|
!config.getBoolean("akka.actor.allow-java-serialization") ||
|
|
|
|
|
|
config.hasPath("akka.remote.artery.enabled") && config.getBoolean("akka.remote.artery.enabled")) {
|
2018-01-10 19:28:51 +09:00
|
|
|
|
|
|
|
|
|
|
val bs = defaultBindings.withFallback(config.getConfig("akka.actor.additional-serialization-bindings"))
|
|
|
|
|
|
|
|
|
|
|
|
// in addition to the additional settings, we also enable even more bindings if java serialization is disabled:
|
|
|
|
|
|
val additionalWhenJavaOffKey = "akka.actor.java-serialization-disabled-additional-serialization-bindings"
|
|
|
|
|
|
if (!config.getBoolean("akka.actor.allow-java-serialization")) {
|
|
|
|
|
|
bs.withFallback(config.getConfig(additionalWhenJavaOffKey))
|
|
|
|
|
|
} else bs
|
2017-02-16 14:09:04 +01:00
|
|
|
|
} else {
|
|
|
|
|
|
defaultBindings
|
|
|
|
|
|
}
|
2018-01-10 19:28:51 +09:00
|
|
|
|
}
|
2016-09-09 09:01:15 +02:00
|
|
|
|
configToMap(bindings)
|
|
|
|
|
|
}
|
2011-11-24 18:53:18 +01:00
|
|
|
|
|
2016-09-09 09:01:15 +02:00
|
|
|
|
private final def configToMap(cfg: Config): Map[String, String] = {
|
2019-05-24 08:11:50 +02:00
|
|
|
|
import akka.util.ccompat.JavaConverters._
|
2019-03-11 10:38:24 +01:00
|
|
|
|
cfg.root.unwrapped.asScala.toMap.map { case (k, v) => (k -> v.toString) }
|
2012-11-07 16:35:14 +01:00
|
|
|
|
}
|
2011-11-24 18:53:18 +01:00
|
|
|
|
}
|
2013-03-25 08:42:48 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* The serialized path of an actorRef, based on the current transport serialization information.
|
2018-05-21 16:59:04 +02:00
|
|
|
|
* If there is no external address available in the given `ActorRef` then the systems default
|
|
|
|
|
|
* address will be used and that is retrieved from the ThreadLocal `Serialization.Information`
|
|
|
|
|
|
* that was set with [[Serialization#withTransportInformation]].
|
2013-03-25 08:42:48 +01:00
|
|
|
|
*/
|
|
|
|
|
|
def serializedActorPath(actorRef: ActorRef): String = {
|
|
|
|
|
|
val path = actorRef.path
|
2013-03-26 11:25:09 +01:00
|
|
|
|
val originalSystem: ExtendedActorSystem = actorRef match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case a: ActorRefWithCell => a.underlying.system.asInstanceOf[ExtendedActorSystem]
|
|
|
|
|
|
case _ => null
|
2013-03-25 08:42:48 +01:00
|
|
|
|
}
|
|
|
|
|
|
Serialization.currentTransportInformation.value match {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
case null =>
|
|
|
|
|
|
originalSystem match {
|
|
|
|
|
|
case null => path.toSerializationFormat
|
|
|
|
|
|
case system =>
|
|
|
|
|
|
try path.toSerializationFormatWithAddress(system.provider.getDefaultAddress)
|
|
|
|
|
|
catch { case NonFatal(_) => path.toSerializationFormat }
|
|
|
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case Information(address, system) =>
|
2013-03-25 08:42:48 +01:00
|
|
|
|
if (originalSystem == null || originalSystem == system)
|
|
|
|
|
|
path.toSerializationFormatWithAddress(address)
|
|
|
|
|
|
else {
|
|
|
|
|
|
val provider = originalSystem.provider
|
2019-03-11 10:38:24 +01:00
|
|
|
|
path.toSerializationFormatWithAddress(
|
|
|
|
|
|
provider.getExternalAddressFor(address).getOrElse(provider.getDefaultAddress))
|
2013-03-25 08:42:48 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2018-03-12 03:49:14 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
2018-05-21 16:59:04 +02:00
|
|
|
|
* Serialization information needed for serializing local actor refs,
|
|
|
|
|
|
* or if serializer library e.g. custom serializer/deserializer in Jackson need
|
|
|
|
|
|
* access to the current `ActorSystem`.
|
|
|
|
|
|
*/
|
|
|
|
|
|
final case class Information(address: Address, system: ActorSystem)
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Sets serialization information in a `ThreadLocal` and runs `f`. The information is
|
|
|
|
|
|
* needed for serializing local actor refs, or if serializer library e.g. custom serializer/deserializer
|
|
|
|
|
|
* in Jackson need access to the current `ActorSystem`. The current [[Information]] can be accessed within
|
|
|
|
|
|
* `f` via [[Serialization#getCurrentTransportInformation]].
|
2018-03-12 03:49:14 +01:00
|
|
|
|
*
|
2018-05-21 16:59:04 +02:00
|
|
|
|
* Akka Remoting sets this value when serializing and deserializing messages, and when using
|
|
|
|
|
|
* the ordinary `serialize` and `deserialize` methods in [[Serialization]] the value is also
|
|
|
|
|
|
* set automatically.
|
|
|
|
|
|
*
|
|
|
|
|
|
* @return value returned by `f`
|
2018-03-12 03:49:14 +01:00
|
|
|
|
*/
|
2019-02-09 15:25:39 +01:00
|
|
|
|
def withTransportInformation[T](system: ExtendedActorSystem)(f: () => T): T = {
|
2018-05-21 16:59:04 +02:00
|
|
|
|
val info = system.provider.serializationInformation
|
|
|
|
|
|
if (Serialization.currentTransportInformation.value eq info)
|
|
|
|
|
|
f() // already set
|
|
|
|
|
|
else
|
|
|
|
|
|
Serialization.currentTransportInformation.withValue(info) {
|
2018-03-12 03:49:14 +01:00
|
|
|
|
f()
|
|
|
|
|
|
}
|
2018-05-21 16:59:04 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Gets the serialization information from a `ThreadLocal` that was assigned via
|
|
|
|
|
|
* [[Serialization#withTransportInformation]]. The information is needed for serializing
|
|
|
|
|
|
* local actor refs, or if serializer library e.g. custom serializer/deserializer
|
|
|
|
|
|
* in Jackson need access to the current `ActorSystem`.
|
|
|
|
|
|
*
|
|
|
|
|
|
* @throws IllegalStateException if the information was not set
|
|
|
|
|
|
*/
|
|
|
|
|
|
def getCurrentTransportInformation(): Information = {
|
|
|
|
|
|
Serialization.currentTransportInformation.value match {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
case null =>
|
|
|
|
|
|
throw new IllegalStateException(
|
|
|
|
|
|
"currentTransportInformation is not set, use Serialization.withTransportInformation")
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case t => t
|
2018-03-12 03:49:14 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-11-24 18:53:18 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2011-06-22 09:59:00 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Serialization module. Contains methods for serialization and deserialization as well as
|
2012-02-06 21:12:26 +01:00
|
|
|
|
* locating a Serializer for a particular class as defined in the mapping in the configuration.
|
2011-06-22 09:59:00 +02:00
|
|
|
|
*/
|
2012-01-24 11:33:40 +01:00
|
|
|
|
class Serialization(val system: ExtendedActorSystem) extends Extension {
|
2011-11-24 18:53:18 +01:00
|
|
|
|
import Serialization._
|
|
|
|
|
|
|
2011-11-29 11:50:22 +01:00
|
|
|
|
val settings = new Settings(system.settings.config)
|
2017-02-16 14:09:04 +01:00
|
|
|
|
val AllowJavaSerialization: Boolean = system.settings.AllowJavaSerialization
|
|
|
|
|
|
|
2016-10-28 17:22:36 +02:00
|
|
|
|
private[this] val _log = Logging.withMarker(system, getClass.getName)
|
|
|
|
|
|
val log: LoggingAdapter = _log
|
2016-05-26 11:58:13 +02:00
|
|
|
|
private val manifestCache = new AtomicReference[Map[String, Option[Class[_]]]](Map.empty[String, Option[Class[_]]])
|
2011-07-19 14:20:18 +02:00
|
|
|
|
|
2018-05-21 16:59:04 +02:00
|
|
|
|
/** INTERNAL API */
|
|
|
|
|
|
@InternalApi private[akka] def serializationInformation: Serialization.Information =
|
|
|
|
|
|
system.provider.serializationInformation
|
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
|
private def withTransportInformation[T](f: () => T): T = {
|
2018-05-21 16:59:04 +02:00
|
|
|
|
val oldInfo = Serialization.currentTransportInformation.value
|
|
|
|
|
|
try {
|
|
|
|
|
|
if (oldInfo eq null)
|
|
|
|
|
|
Serialization.currentTransportInformation.value = serializationInformation
|
|
|
|
|
|
f()
|
|
|
|
|
|
} finally Serialization.currentTransportInformation.value = oldInfo
|
|
|
|
|
|
}
|
|
|
|
|
|
|
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.
|
|
|
|
|
|
*/
|
2018-05-21 16:59:04 +02:00
|
|
|
|
def serialize(o: AnyRef): Try[Array[Byte]] = {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
withTransportInformation { () =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
Try(findSerializerFor(o).toBinary(o))
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2011-07-19 14:20:18 +02:00
|
|
|
|
|
2011-12-29 16:11:56 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* Deserializes the given array of bytes using the specified serializer id,
|
2015-05-28 18:42:22 +02:00
|
|
|
|
* using the optional type hint to the Serializer.
|
2011-12-29 16:11:56 +01:00
|
|
|
|
* Returns either the resulting object or an Exception if one was thrown.
|
|
|
|
|
|
*/
|
2019-05-20 17:39:46 -04:00
|
|
|
|
@deprecated("Use deserialize that accepts the `manifest` as a class name.", since = "2.6.0")
|
2012-12-14 18:25:04 +01:00
|
|
|
|
def deserialize[T](bytes: Array[Byte], serializerId: Int, clazz: Option[Class[_ <: T]]): Try[T] =
|
2013-07-03 11:26:50 +02:00
|
|
|
|
Try {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
val serializer = try getSerializerById(serializerId)
|
|
|
|
|
|
catch {
|
|
|
|
|
|
case _: NoSuchElementException =>
|
|
|
|
|
|
throw new NotSerializableException(
|
|
|
|
|
|
s"Cannot find serializer with id [$serializerId]${clazz.map(c => " (class [" + c.getName + "])").getOrElse("")}. " +
|
2019-01-15 10:53:35 +01:00
|
|
|
|
"The most probable reason is that the configuration entry " +
|
|
|
|
|
|
"akka.actor.serializers is not in sync between the two systems.")
|
2013-07-03 11:26:50 +02:00
|
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
|
withTransportInformation { () =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
serializer.fromBinary(bytes, clazz).asInstanceOf[T]
|
|
|
|
|
|
}
|
2013-07-03 11:26:50 +02:00
|
|
|
|
}
|
2011-12-29 16:11:56 +01:00
|
|
|
|
|
2015-05-28 18:42:22 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Deserializes the given array of bytes using the specified serializer id,
|
|
|
|
|
|
* using the optional type hint to the Serializer.
|
|
|
|
|
|
* Returns either the resulting object or an Exception if one was thrown.
|
|
|
|
|
|
*/
|
|
|
|
|
|
def deserialize(bytes: Array[Byte], serializerId: Int, manifest: String): Try[AnyRef] =
|
|
|
|
|
|
Try {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
val serializer = try getSerializerById(serializerId)
|
|
|
|
|
|
catch {
|
|
|
|
|
|
case _: NoSuchElementException =>
|
|
|
|
|
|
throw new NotSerializableException(
|
|
|
|
|
|
s"Cannot find serializer with id [$serializerId] (manifest [$manifest]). The most probable reason is that the configuration entry " +
|
2019-01-15 10:53:35 +01:00
|
|
|
|
"akka.actor.serializers is not in sync between the two systems.")
|
2015-05-28 18:42:22 +02:00
|
|
|
|
}
|
2016-05-26 11:58:13 +02:00
|
|
|
|
deserializeByteArray(bytes, serializer, manifest)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private def deserializeByteArray(bytes: Array[Byte], serializer: Serializer, manifest: String): AnyRef = {
|
|
|
|
|
|
|
|
|
|
|
|
@tailrec def updateCache(cache: Map[String, Option[Class[_]]], key: String, value: Option[Class[_]]): Boolean = {
|
|
|
|
|
|
manifestCache.compareAndSet(cache, cache.updated(key, value)) ||
|
2019-03-11 10:38:24 +01:00
|
|
|
|
updateCache(manifestCache.get, key, value) // recursive, try again
|
2016-05-26 11:58:13 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
|
withTransportInformation { () =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
serializer match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case s2: SerializerWithStringManifest => s2.fromBinary(bytes, manifest)
|
|
|
|
|
|
case s1 =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
if (manifest == "")
|
|
|
|
|
|
s1.fromBinary(bytes, None)
|
|
|
|
|
|
else {
|
|
|
|
|
|
val cache = manifestCache.get
|
|
|
|
|
|
cache.get(manifest) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case Some(cachedClassManifest) => s1.fromBinary(bytes, cachedClassManifest)
|
|
|
|
|
|
case None =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
system.dynamicAccess.getClassFor[AnyRef](manifest) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case Success(classManifest) =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
val classManifestOption: Option[Class[_]] = Some(classManifest)
|
|
|
|
|
|
updateCache(cache, manifest, classManifestOption)
|
|
|
|
|
|
s1.fromBinary(bytes, classManifestOption)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case Failure(_) =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
throw new NotSerializableException(
|
|
|
|
|
|
s"Cannot find manifest class [$manifest] for serializer with id [${serializer.identifier}].")
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2015-05-28 18:42:22 +02:00
|
|
|
|
}
|
2018-05-21 16:59:04 +02:00
|
|
|
|
}
|
2015-05-28 18:42:22 +02:00
|
|
|
|
}
|
2016-05-26 11:58:13 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* Deserializes the given ByteBuffer of bytes using the specified serializer id,
|
|
|
|
|
|
* using the optional type hint to the Serializer.
|
|
|
|
|
|
* Returns either the resulting object or throws an exception if deserialization fails.
|
|
|
|
|
|
*/
|
2017-08-08 06:15:18 -05:00
|
|
|
|
@throws(classOf[NotSerializableException])
|
2016-05-26 11:58:13 +02:00
|
|
|
|
def deserializeByteBuffer(buf: ByteBuffer, serializerId: Int, manifest: String): AnyRef = {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
val serializer = try getSerializerById(serializerId)
|
|
|
|
|
|
catch {
|
|
|
|
|
|
case _: NoSuchElementException =>
|
|
|
|
|
|
throw new NotSerializableException(
|
|
|
|
|
|
s"Cannot find serializer with id [$serializerId] (manifest [$manifest]). The most probable reason is that the configuration entry " +
|
2016-05-26 11:58:13 +02:00
|
|
|
|
"akka.actor.serializers is not in synch between the two systems.")
|
|
|
|
|
|
}
|
2018-05-21 16:59:04 +02:00
|
|
|
|
|
|
|
|
|
|
// not using `withTransportInformation { () =>` because deserializeByteBuffer is supposed to be the
|
|
|
|
|
|
// possibility for allocation free serialization
|
|
|
|
|
|
val oldInfo = Serialization.currentTransportInformation.value
|
|
|
|
|
|
try {
|
|
|
|
|
|
if (oldInfo eq null)
|
|
|
|
|
|
Serialization.currentTransportInformation.value = serializationInformation
|
|
|
|
|
|
|
|
|
|
|
|
serializer match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case ser: ByteBufferSerializer =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
ser.fromBinary(buf, manifest)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case _ =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
val bytes = new Array[Byte](buf.remaining())
|
|
|
|
|
|
buf.get(bytes)
|
|
|
|
|
|
deserializeByteArray(bytes, serializer, manifest)
|
|
|
|
|
|
}
|
|
|
|
|
|
} finally Serialization.currentTransportInformation.value = oldInfo
|
2016-05-26 11:58:13 +02:00
|
|
|
|
}
|
2015-05-28 18:42:22 +02:00
|
|
|
|
|
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.
|
|
|
|
|
|
* Returns either the resulting object or an Exception if one was thrown.
|
|
|
|
|
|
*/
|
2018-05-21 16:59:04 +02:00
|
|
|
|
def deserialize[T](bytes: Array[Byte], clazz: Class[T]): Try[T] = {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
withTransportInformation { () =>
|
2018-05-21 16:59:04 +02:00
|
|
|
|
Try(serializerFor(clazz).fromBinary(bytes, Some(clazz)).asInstanceOf[T])
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
2011-07-19 14:20:18 +02:00
|
|
|
|
|
2011-12-29 16:11:56 +01:00
|
|
|
|
/**
|
2012-02-06 21:12:26 +01:00
|
|
|
|
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
|
|
|
|
|
*
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* Throws akka.ConfigurationException if no `serialization-bindings` is configured for the
|
|
|
|
|
|
* class of the object.
|
2011-12-29 16:11:56 +01:00
|
|
|
|
*/
|
2015-05-28 18:42:22 +02:00
|
|
|
|
def findSerializerFor(o: AnyRef): Serializer =
|
|
|
|
|
|
if (o eq null) NullSerializer else serializerFor(o.getClass)
|
2011-07-19 14:20:18 +02:00
|
|
|
|
|
2011-12-29 16:11:56 +01:00
|
|
|
|
/**
|
2012-02-06 21:12:26 +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.
|
|
|
|
|
|
*
|
2015-05-15 16:53:24 +02:00
|
|
|
|
* Throws java.io.NotSerializableException if no `serialization-bindings` is configured for the class.
|
2011-12-29 16:11:56 +01:00
|
|
|
|
*/
|
2017-08-08 06:15:18 -05:00
|
|
|
|
@throws(classOf[NotSerializableException])
|
2012-02-04 17:35:39 +01:00
|
|
|
|
def serializerFor(clazz: Class[_]): Serializer =
|
2012-02-06 21:12:26 +01:00
|
|
|
|
serializerMap.get(clazz) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case null => // bindings are ordered from most specific to least specific
|
2012-10-30 15:08:41 +01:00
|
|
|
|
def unique(possibilities: immutable.Seq[(Class[_], Serializer)]): Boolean =
|
2012-02-07 16:21:48 +01:00
|
|
|
|
possibilities.size == 1 ||
|
2019-03-11 10:38:24 +01:00
|
|
|
|
(possibilities.forall(_._1.isAssignableFrom(possibilities(0)._1))) ||
|
|
|
|
|
|
(possibilities.forall(_._2 == possibilities(0)._2))
|
2012-02-07 15:11:16 +01:00
|
|
|
|
|
2017-02-16 14:09:04 +01:00
|
|
|
|
val ser = {
|
|
|
|
|
|
bindings.filter {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
case (c, _) => c.isAssignableFrom(clazz)
|
2017-02-16 14:09:04 +01:00
|
|
|
|
} match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case immutable.Seq() =>
|
2017-11-21 08:34:13 +01:00
|
|
|
|
throw new NotSerializableException(s"No configured serialization-bindings for class [${clazz.getName}]")
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case possibilities =>
|
2017-11-21 08:34:13 +01:00
|
|
|
|
if (unique(possibilities))
|
|
|
|
|
|
possibilities.head._2
|
|
|
|
|
|
else {
|
|
|
|
|
|
// give JavaSerializer lower priority if multiple serializers found
|
|
|
|
|
|
val possibilitiesWithoutJavaSerializer = possibilities.filter {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case (_, _: JavaSerializer) => false
|
|
|
|
|
|
case (_, _: DisabledJavaSerializer) => false
|
|
|
|
|
|
case _ => true
|
2017-11-21 08:34:13 +01:00
|
|
|
|
}
|
|
|
|
|
|
if (possibilitiesWithoutJavaSerializer.isEmpty) {
|
|
|
|
|
|
// shouldn't happen
|
2019-03-11 10:38:24 +01:00
|
|
|
|
throw new NotSerializableException(
|
|
|
|
|
|
s"More than one JavaSerializer configured for class [${clazz.getName}]")
|
2017-11-21 08:34:13 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
if (!unique(possibilitiesWithoutJavaSerializer)) {
|
2019-03-13 10:56:20 +01:00
|
|
|
|
_log.warning(
|
|
|
|
|
|
LogMarker.Security,
|
|
|
|
|
|
"Multiple serializers found for [{}], choosing first of: [{}]",
|
|
|
|
|
|
clazz.getName,
|
|
|
|
|
|
possibilitiesWithoutJavaSerializer.map { case (_, s) => s.getClass.getName }.mkString(", "))
|
2017-11-21 08:34:13 +01:00
|
|
|
|
}
|
|
|
|
|
|
possibilitiesWithoutJavaSerializer.head._2
|
|
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2017-02-16 14:09:04 +01:00
|
|
|
|
}
|
2012-02-07 15:51:41 +01:00
|
|
|
|
}
|
2017-02-16 14:09:04 +01:00
|
|
|
|
|
2012-02-07 15:51:41 +01:00
|
|
|
|
serializerMap.putIfAbsent(clazz, ser) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case null =>
|
2015-09-24 16:45:51 +02:00
|
|
|
|
if (shouldWarnAboutJavaSerializer(clazz, ser)) {
|
2019-03-13 10:56:20 +01:00
|
|
|
|
_log.warning(
|
|
|
|
|
|
LogMarker.Security,
|
|
|
|
|
|
"Using the default Java serializer for class [{}] which is not recommended because of " +
|
|
|
|
|
|
"performance implications. Use another serializer or disable this warning using the setting " +
|
|
|
|
|
|
"'akka.actor.warn-about-java-serializer-usage'",
|
|
|
|
|
|
clazz.getName)
|
2015-09-24 16:45:51 +02:00
|
|
|
|
}
|
2019-02-22 08:59:17 +01:00
|
|
|
|
|
|
|
|
|
|
if (!warnUnexpectedNonAkkaSerializer(clazz, ser))
|
|
|
|
|
|
log.debug("Using serializer [{}] for message [{}]", ser.getClass.getName, clazz.getName)
|
|
|
|
|
|
|
2013-06-02 14:53:31 +02:00
|
|
|
|
ser
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case some => some
|
2012-02-04 17:35:39 +01:00
|
|
|
|
}
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case ser => ser
|
2012-02-03 17:32:32 +01:00
|
|
|
|
}
|
2011-06-07 06:36:21 +05:30
|
|
|
|
|
2011-07-15 16:21:45 +02:00
|
|
|
|
/**
|
2012-02-09 19:26:02 +01:00
|
|
|
|
* Tries to load the specified Serializer by the fully-qualified name; the actual
|
2012-02-10 11:36:23 +01:00
|
|
|
|
* loading is performed by the system’s [[akka.actor.DynamicAccess]].
|
2011-07-15 16:21:45 +02:00
|
|
|
|
*/
|
2017-02-16 14:09:04 +01:00
|
|
|
|
def serializerOf(serializerFQN: String): Try[Serializer] = {
|
|
|
|
|
|
// We override each instantiation of the JavaSerializer with the "disabled" serializer which will log warnings if used.
|
|
|
|
|
|
val fqn =
|
|
|
|
|
|
if (!system.settings.AllowJavaSerialization && serializerFQN == classOf[JavaSerializer].getName) {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
log.debug(
|
|
|
|
|
|
"Replacing JavaSerializer with DisabledJavaSerializer, " +
|
2017-02-16 14:09:04 +01:00
|
|
|
|
"due to `akka.actor.allow-java-serialization = off`.")
|
|
|
|
|
|
classOf[DisabledJavaSerializer].getName
|
|
|
|
|
|
} else serializerFQN
|
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
|
system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] -> system)).recoverWith {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case _: NoSuchMethodException =>
|
2017-02-16 14:09:04 +01:00
|
|
|
|
system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil)
|
2018-11-22 16:18:10 +01:00
|
|
|
|
// FIXME only needed on 2.13.0-M5 due to https://github.com/scala/bug/issues/11242
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case t => Failure(t)
|
2012-09-06 03:17:51 +02:00
|
|
|
|
}
|
2017-02-16 14:09:04 +01:00
|
|
|
|
}
|
2011-07-15 16:21:45 +02:00
|
|
|
|
|
2016-12-15 15:05:13 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* Programmatically defined serializers
|
|
|
|
|
|
*/
|
2017-02-16 14:09:04 +01:00
|
|
|
|
private val serializerDetails: immutable.Seq[SerializerDetails] =
|
|
|
|
|
|
(system.settings.setup.get[SerializationSetup] match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case None => Vector.empty
|
|
|
|
|
|
case Some(setting) => setting.createSerializers(system)
|
2019-03-11 10:38:24 +01:00
|
|
|
|
}).collect {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case det: SerializerDetails if isDisallowedJavaSerializer(det.serializer) =>
|
2019-03-11 10:38:24 +01:00
|
|
|
|
log.debug(
|
|
|
|
|
|
"Replacing JavaSerializer with DisabledJavaSerializer, " +
|
2017-02-16 14:09:04 +01:00
|
|
|
|
"due to `akka.actor.allow-java-serialization = off`.")
|
|
|
|
|
|
SerializerDetails(det.alias, new DisabledJavaSerializer(system), det.useFor)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case det => det
|
2016-12-15 15:05:13 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2011-07-14 11:25:40 +02:00
|
|
|
|
/**
|
2011-07-19 14:20:18 +02:00
|
|
|
|
* A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer)
|
2012-02-06 21:12:26 +01:00
|
|
|
|
* By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer
|
2011-07-14 11:25:40 +02:00
|
|
|
|
*/
|
2016-12-15 15:05:13 +01:00
|
|
|
|
private val serializers: Map[String, Serializer] = {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
val fromConfig = for ((k: String, v: String) <- settings.Serializers) yield k -> serializerOf(v).get
|
|
|
|
|
|
val result = fromConfig ++ serializerDetails.map(d => d.alias -> d.serializer)
|
|
|
|
|
|
ensureOnlyAllowedSerializers(result.iterator.map { case (_, ser) => ser })
|
2017-02-16 14:09:04 +01:00
|
|
|
|
result
|
2016-12-15 15:05:13 +01:00
|
|
|
|
}
|
2011-06-29 21:25:17 +02:00
|
|
|
|
|
2011-07-14 11:25:40 +02:00
|
|
|
|
/**
|
2012-02-06 21:12:26 +01: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.
|
2011-07-14 11:25:40 +02:00
|
|
|
|
*/
|
2016-12-15 15:05:13 +01:00
|
|
|
|
private[akka] val bindings: immutable.Seq[ClassSerializer] = {
|
|
|
|
|
|
val fromConfig = for {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
(className: String, alias: String) <- settings.SerializationBindings
|
2016-12-15 15:05:13 +01:00
|
|
|
|
if alias != "none" && checkGoogleProtobuf(className)
|
|
|
|
|
|
} yield (system.dynamicAccess.getClassFor[Any](className).get, serializers(alias))
|
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
|
val fromSettings = serializerDetails.flatMap { detail =>
|
|
|
|
|
|
detail.useFor.map(clazz => clazz -> detail.serializer)
|
2016-12-15 15:05:13 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2017-02-16 14:09:04 +01:00
|
|
|
|
val result = sort(fromConfig ++ fromSettings)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
ensureOnlyAllowedSerializers(result.iterator.map { case (_, ser) => ser })
|
|
|
|
|
|
result.foreach { case (clazz, ser) => warnUnexpectedNonAkkaSerializer(clazz, ser) }
|
2017-02-16 14:09:04 +01:00
|
|
|
|
result
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private def ensureOnlyAllowedSerializers(iter: Iterator[Serializer]): Unit = {
|
|
|
|
|
|
if (!system.settings.AllowJavaSerialization) {
|
|
|
|
|
|
require(iter.forall(!isDisallowedJavaSerializer(_)), "Disallowed JavaSerializer binding.")
|
|
|
|
|
|
}
|
2016-12-15 15:05:13 +01:00
|
|
|
|
}
|
2015-08-31 12:38:07 +02:00
|
|
|
|
|
2019-02-22 08:59:17 +01:00
|
|
|
|
private def warnUnexpectedNonAkkaSerializer(clazz: Class[_], ser: Serializer): Boolean = {
|
|
|
|
|
|
if (clazz.getName.startsWith("akka.") && !ser.getClass.getName.startsWith("akka.")) {
|
2019-03-13 10:56:20 +01:00
|
|
|
|
log.warning(
|
|
|
|
|
|
"Using serializer [{}] for message [{}]. Note that this serializer " +
|
|
|
|
|
|
"is not implemented by Akka. It's not recommended to replace serializers for messages " +
|
|
|
|
|
|
"provided by Akka.",
|
|
|
|
|
|
ser.getClass.getName,
|
|
|
|
|
|
clazz.getName)
|
2019-02-22 08:59:17 +01:00
|
|
|
|
true
|
|
|
|
|
|
} else false
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2015-08-31 12:38:07 +02:00
|
|
|
|
// com.google.protobuf serialization binding is only used if the class can be loaded,
|
|
|
|
|
|
// i.e. com.google.protobuf dependency has been added in the application project.
|
|
|
|
|
|
// The reason for this special case is for backwards compatibility so that we still can
|
|
|
|
|
|
// include "com.google.protobuf.GeneratedMessage" = proto in configured serialization-bindings.
|
|
|
|
|
|
private def checkGoogleProtobuf(className: String): Boolean =
|
|
|
|
|
|
(!className.startsWith("com.google.protobuf") || system.dynamicAccess.getClassFor[Any](className).isSuccess)
|
2011-06-29 21:25:17 +02:00
|
|
|
|
|
2011-07-14 11:25:40 +02:00
|
|
|
|
/**
|
2012-02-06 21:12:26 +01:00
|
|
|
|
* Sort so that subtypes always precede their supertypes, but without
|
|
|
|
|
|
* obeying any order between unrelated subtypes (insert sort).
|
2011-07-14 11:25:40 +02:00
|
|
|
|
*/
|
2012-10-30 15:08:41 +01:00
|
|
|
|
private def sort(in: Iterable[ClassSerializer]): immutable.Seq[ClassSerializer] =
|
2019-03-11 10:38:24 +01:00
|
|
|
|
(in
|
|
|
|
|
|
.foldLeft(new ArrayBuffer[ClassSerializer](in.size)) { (buf, ca) =>
|
|
|
|
|
|
buf.indexWhere(_._1.isAssignableFrom(ca._1)) match {
|
|
|
|
|
|
case -1 => buf.append(ca)
|
|
|
|
|
|
case x => buf.insert(x, ca)
|
|
|
|
|
|
}
|
|
|
|
|
|
buf
|
|
|
|
|
|
})
|
|
|
|
|
|
.to(immutable.Seq)
|
2012-02-06 21:12:26 +01:00
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* serializerMap is a Map whose keys is the class that is serializable and values is the serializer
|
|
|
|
|
|
* to be used for that class.
|
|
|
|
|
|
*/
|
2012-06-25 17:09:00 +02:00
|
|
|
|
private val serializerMap: ConcurrentHashMap[Class[_], Serializer] =
|
2019-02-09 15:25:39 +01:00
|
|
|
|
bindings.foldLeft(new ConcurrentHashMap[Class[_], Serializer]) { case (map, (c, s)) => map.put(c, s); map }
|
2011-07-19 14:20:18 +02:00
|
|
|
|
|
|
|
|
|
|
/**
|
2011-12-30 22:00:49 +01:00
|
|
|
|
* Maps from a Serializer Identity (Int) to a Serializer instance (optimization)
|
2011-07-19 14:20:18 +02:00
|
|
|
|
*/
|
2012-02-06 21:12:26 +01:00
|
|
|
|
val serializerByIdentity: Map[Int, Serializer] =
|
2019-02-09 15:25:39 +01:00
|
|
|
|
Map(NullSerializer.identifier -> NullSerializer) ++ serializers.map { case (_, v) => (v.identifier, v) }
|
2015-05-28 18:42:22 +02:00
|
|
|
|
|
2016-06-04 21:44:13 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* Serializers with id 0 - 1023 are stored in an array for quick allocation free access
|
|
|
|
|
|
*/
|
|
|
|
|
|
private val quickSerializerByIdentity: Array[Serializer] = {
|
|
|
|
|
|
val size = 1024
|
2017-03-13 17:49:45 +01:00
|
|
|
|
val table = new Array[Serializer](size)
|
2016-06-04 21:44:13 +02:00
|
|
|
|
serializerByIdentity.foreach {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case (id, ser) => if (0 <= id && id < size) table(id) = ser
|
2016-06-04 21:44:13 +02:00
|
|
|
|
}
|
|
|
|
|
|
table
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
|
* @throws `NoSuchElementException` if no serializer with given `id`
|
|
|
|
|
|
*/
|
|
|
|
|
|
private def getSerializerById(id: Int): Serializer = {
|
|
|
|
|
|
if (0 <= id && id < quickSerializerByIdentity.length) {
|
|
|
|
|
|
quickSerializerByIdentity(id) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case null => throw new NoSuchElementException(s"key not found: $id")
|
|
|
|
|
|
case ser => ser
|
2016-06-04 21:44:13 +02:00
|
|
|
|
}
|
|
|
|
|
|
} else
|
|
|
|
|
|
serializerByIdentity(id)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
|
private val isJavaSerializationWarningEnabled =
|
|
|
|
|
|
settings.config.getBoolean("akka.actor.warn-about-java-serializer-usage")
|
|
|
|
|
|
private val isWarningOnNoVerificationEnabled =
|
|
|
|
|
|
settings.config.getBoolean("akka.actor.warn-on-no-serialization-verification")
|
2016-03-07 20:51:26 +01:00
|
|
|
|
|
2017-02-16 14:09:04 +01:00
|
|
|
|
private def isDisallowedJavaSerializer(serializer: Serializer): Boolean = {
|
|
|
|
|
|
serializer.isInstanceOf[JavaSerializer] && !system.settings.AllowJavaSerialization
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2017-04-25 20:52:45 +02:00
|
|
|
|
/**
|
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
|
*/
|
|
|
|
|
|
@InternalApi private[akka] def shouldWarnAboutJavaSerializer(serializedClass: Class[_], serializer: Serializer) = {
|
2016-07-06 18:48:15 +09:00
|
|
|
|
|
|
|
|
|
|
def suppressWarningOnNonSerializationVerification(serializedClass: Class[_]) = {
|
|
|
|
|
|
//suppressed, only when warn-on-no-serialization-verification = off, and extending NoSerializationVerificationNeeded
|
|
|
|
|
|
!isWarningOnNoVerificationEnabled && classOf[NoSerializationVerificationNeeded].isAssignableFrom(serializedClass)
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2016-03-07 20:51:26 +01:00
|
|
|
|
isJavaSerializationWarningEnabled &&
|
2019-03-11 10:38:24 +01:00
|
|
|
|
(serializer.isInstanceOf[JavaSerializer] || serializer.isInstanceOf[DisabledJavaSerializer]) &&
|
|
|
|
|
|
!serializedClass.getName.startsWith("akka.") &&
|
|
|
|
|
|
!serializedClass.getName.startsWith("java.lang.") &&
|
|
|
|
|
|
!suppressWarningOnNonSerializationVerification(serializedClass)
|
2016-03-07 20:51:26 +01:00
|
|
|
|
}
|
2011-06-07 06:36:21 +05:30
|
|
|
|
}
|