pekko/akka-serialization-jackson/src/test/scala/akka/serialization/jackson/JacksonSerializerSpec.scala

968 lines
40 KiB
Scala

/*
* Copyright (C) 2016-2020 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.serialization.jackson
import java.nio.charset.StandardCharsets
import java.time.Duration
import java.time.Instant
import java.time.LocalDateTime
import java.time.temporal.ChronoUnit
import java.util.Arrays
import java.util.Locale
import java.util.Optional
import java.util.UUID
import java.util.logging.FileHandler
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.duration.FiniteDuration
import com.fasterxml.jackson.annotation.JsonSubTypes
import com.fasterxml.jackson.annotation.JsonTypeInfo
import com.fasterxml.jackson.core.JsonFactory
import com.fasterxml.jackson.core.JsonGenerator
import com.fasterxml.jackson.core.JsonParser
import com.fasterxml.jackson.core.StreamReadFeature
import com.fasterxml.jackson.core.StreamWriteFeature
import com.fasterxml.jackson.core.`type`.TypeReference
import com.fasterxml.jackson.databind.DeserializationFeature
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.MapperFeature
import com.fasterxml.jackson.databind.Module
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.databind.SerializationFeature
import com.fasterxml.jackson.databind.exc.InvalidTypeIdException
import com.fasterxml.jackson.databind.json.JsonMapper
import com.fasterxml.jackson.databind.node.IntNode
import com.fasterxml.jackson.databind.node.ObjectNode
import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule
import com.fasterxml.jackson.module.scala.JsonScalaEnumeration
import com.github.ghik.silencer.silent
import com.typesafe.config.ConfigFactory
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.BootstrapSetup
import akka.actor.ExtendedActorSystem
import akka.actor.Status
import akka.actor.setup.ActorSystemSetup
import akka.actor.typed.scaladsl.Behaviors
import akka.serialization.Serialization
import akka.serialization.SerializationExtension
import akka.testkit.TestActors
import akka.testkit.TestKit
object ScalaTestMessages {
trait TestMessage
final case class SimpleCommand(name: String) extends TestMessage
// interesting that this doesn't have the same problem with single constructor param
// as JavaTestMessages.SimpleCommand
final class SimpleCommandNotCaseClass(val name: String) extends TestMessage {
override def equals(obj: Any): Boolean = obj match {
case other: SimpleCommandNotCaseClass => other.name == name
}
override def hashCode(): Int = name.hashCode
}
final case class SimpleCommand2(name: String, name2: String) extends TestMessage
final case class OptionCommand(maybe: Option[String]) extends TestMessage
final case class BooleanCommand(published: Boolean) extends TestMessage
final case class TimeCommand(timestamp: LocalDateTime, duration: FiniteDuration) extends TestMessage
final case class InstantCommand(instant: Instant) extends TestMessage
final case class UUIDCommand(uuid: UUID) extends TestMessage
final case class CollectionsCommand(strings: List[String], objects: Vector[SimpleCommand]) extends TestMessage
final case class CommandWithActorRef(name: String, replyTo: ActorRef) extends TestMessage
final case class CommandWithTypedActorRef(name: String, replyTo: akka.actor.typed.ActorRef[String])
extends TestMessage
final case class CommandWithAddress(name: String, address: Address) extends TestMessage
case object SingletonCaseObject extends TestMessage
final case class Event1(field1: String) extends TestMessage
final case class Event2(field1V2: String, field2: Int) extends TestMessage
final case class Zoo(first: Animal) extends TestMessage
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(
Array(
new JsonSubTypes.Type(value = classOf[Lion], name = "lion"),
new JsonSubTypes.Type(value = classOf[Elephant], name = "elephant")))
sealed trait Animal
final case class Lion(name: String) extends Animal
final case class Elephant(name: String, age: Int) extends Animal
// not defined in JsonSubTypes
final case class Cockroach(name: String) extends Animal
final case class OldCommandNotInBindings(name: String)
// #jackson-scala-enumeration
object Planet extends Enumeration {
type Planet = Value
val Mercury, Venus, Earth, Mars, Krypton = Value
}
// Uses default Jackson serialization format for Scala Enumerations
final case class Alien(name: String, planet: Planet.Planet) extends TestMessage
// Serializes planet values as a JsonString
class PlanetType extends TypeReference[Planet.type] {}
final case class Superhero(name: String, @JsonScalaEnumeration(classOf[PlanetType]) planet: Planet.Planet)
extends TestMessage
// #jackson-scala-enumeration
}
class ScalaTestEventMigration extends JacksonMigration {
override def currentVersion = 3
override def transformClassName(fromVersion: Int, className: String): String =
classOf[ScalaTestMessages.Event2].getName
override def transform(fromVersion: Int, json: JsonNode): JsonNode = {
val root = json.asInstanceOf[ObjectNode]
root.set("field1V2", root.get("field1"))
root.remove("field1")
root.set("field2", IntNode.valueOf(17))
root
}
}
class JacksonCborSerializerSpec extends JacksonSerializerSpec("jackson-cbor") {
"have compression disabled by default" in {
val conf = JacksonObjectMapperProvider.configForBinding("jackson-cbor", system.settings.config)
val compressionAlgo = conf.getString("compression.algorithm")
compressionAlgo should ===("off")
}
}
@silent // this test uses Jackson deprecated APIs
class JacksonJsonSerializerSpec extends JacksonSerializerSpec("jackson-json") {
def serializeToJsonString(obj: AnyRef, sys: ActorSystem = system): String = {
val blob = serializeToBinary(obj, sys)
new String(blob, "utf-8")
}
def deserializeFromJsonString(
json: String,
serializerId: Int,
manifest: String,
sys: ActorSystem = system): AnyRef = {
val blob = json.getBytes("utf-8")
deserializeFromBinary(blob, serializerId, manifest, sys)
}
"JacksonJsonSerializer" must {
"support lookup of same ObjectMapper via JacksonObjectMapperProvider" in {
val mapper = serialization()
.serializerFor(classOf[JavaTestMessages.TestMessage])
.asInstanceOf[JacksonSerializer]
.objectMapper
JacksonObjectMapperProvider(system).getOrCreate("jackson-json", None) shouldBe theSameInstanceAs(mapper)
val anotherBindingName = "jackson-json2"
val mapper2 = JacksonObjectMapperProvider(system).getOrCreate(anotherBindingName, None)
mapper2 should not be theSameInstanceAs(mapper)
JacksonObjectMapperProvider(system).getOrCreate(anotherBindingName, None) shouldBe theSameInstanceAs(mapper2)
}
"JacksonSerializer configuration" must {
withSystem("""
akka.actor.serializers.jackson-json2 = "akka.serialization.jackson.JacksonJsonSerializer"
akka.actor.serialization-identifiers.jackson-json2 = 999
akka.serialization.jackson.jackson-json2 {
# on is Jackson's default
serialization-features.WRITE_DURATIONS_AS_TIMESTAMPS = off
# on is Jackson's default
deserialization-features.EAGER_DESERIALIZER_FETCH = off
# off is Jackson's default
mapper-features.SORT_PROPERTIES_ALPHABETICALLY = on
# off is Jackson's default
json-parser-features.ALLOW_COMMENTS = on
# on is Jackson's default
json-generator-features.AUTO_CLOSE_TARGET = off
# off is Jackson's default
stream-read-features.STRICT_DUPLICATE_DETECTION = on
# off is Jackson's default
stream-write-features.WRITE_BIGDECIMAL_AS_PLAIN = on
# off is Jackson's default
json-read-features.ALLOW_YAML_COMMENTS = on
# off is Jackson's default
json-write-features.ESCAPE_NON_ASCII = on
}
""") { sys =>
val identifiedObjectMapper =
serialization(sys).serializerByIdentity(999).asInstanceOf[JacksonJsonSerializer].objectMapper
val namedObjectMapper = JacksonObjectMapperProvider(sys).getOrCreate("jackson-json2", None)
val defaultObjectMapper =
serializerFor(ScalaTestMessages.SimpleCommand("abc")).asInstanceOf[JacksonJsonSerializer].objectMapper
"support serialization features" in {
identifiedObjectMapper.isEnabled(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) should ===(false)
namedObjectMapper.isEnabled(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) should ===(false)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) should ===(false)
}
"support deserialization features" in {
identifiedObjectMapper.isEnabled(DeserializationFeature.EAGER_DESERIALIZER_FETCH) should ===(false)
namedObjectMapper.isEnabled(DeserializationFeature.EAGER_DESERIALIZER_FETCH) should ===(false)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(DeserializationFeature.EAGER_DESERIALIZER_FETCH) should ===(true)
}
"support mapper features" in {
identifiedObjectMapper.isEnabled(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY) should ===(true)
namedObjectMapper.isEnabled(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY) should ===(false)
}
"support json parser features" in {
identifiedObjectMapper.isEnabled(JsonParser.Feature.ALLOW_COMMENTS) should ===(true)
namedObjectMapper.isEnabled(JsonParser.Feature.ALLOW_COMMENTS) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(JsonParser.Feature.ALLOW_COMMENTS) should ===(false)
}
"support json generator features" in {
identifiedObjectMapper.isEnabled(JsonGenerator.Feature.AUTO_CLOSE_TARGET) should ===(false)
namedObjectMapper.isEnabled(JsonGenerator.Feature.AUTO_CLOSE_TARGET) should ===(false)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(JsonGenerator.Feature.AUTO_CLOSE_TARGET) should ===(true)
}
"support stream read features" in {
identifiedObjectMapper.isEnabled(StreamReadFeature.STRICT_DUPLICATE_DETECTION) should ===(true)
namedObjectMapper.isEnabled(StreamReadFeature.STRICT_DUPLICATE_DETECTION) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(StreamReadFeature.STRICT_DUPLICATE_DETECTION) should ===(false)
}
"support stream write features" in {
identifiedObjectMapper.isEnabled(StreamWriteFeature.WRITE_BIGDECIMAL_AS_PLAIN) should ===(true)
namedObjectMapper.isEnabled(StreamWriteFeature.WRITE_BIGDECIMAL_AS_PLAIN) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(StreamWriteFeature.WRITE_BIGDECIMAL_AS_PLAIN) should ===(false)
}
"support json read features" in {
// ATTENTION: this is trick. Although we are configuring `json-read-features`, Jackson
// does not provides a way to check for `StreamReadFeature`s, so we need to check for
// `JsonParser.Feature`.ALLOW_YAML_COMMENTS.
// Same applies for json-write-features and JsonGenerator.Feature.
identifiedObjectMapper.isEnabled(JsonParser.Feature.ALLOW_YAML_COMMENTS) should ===(true)
namedObjectMapper.isEnabled(JsonParser.Feature.ALLOW_YAML_COMMENTS) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(JsonParser.Feature.ALLOW_YAML_COMMENTS) should ===(false)
}
"support json write features" in {
// ATTENTION: this is trickier than `json-read-features` vs JsonParser.Feature
// since the JsonWriteFeature replaces deprecated APIs in JsonGenerator.Feature.
// But just like the test for `json-read-features` there is no API to check for
// `JsonWriteFeature`s, so we need to use the deprecated APIs.
identifiedObjectMapper.isEnabled(JsonGenerator.Feature.ESCAPE_NON_ASCII) should ===(true)
namedObjectMapper.isEnabled(JsonGenerator.Feature.ESCAPE_NON_ASCII) should ===(true)
// Default mapper follows Jackson and reference.conf default configuration
defaultObjectMapper.isEnabled(JsonGenerator.Feature.ESCAPE_NON_ASCII) should ===(false)
}
"fallback to defaults when object mapper is not configured" in {
val notConfigured = JacksonObjectMapperProvider(sys).getOrCreate("jackson-not-configured", None)
// Use Jacksons and Akka defaults
notConfigured.isEnabled(SerializationFeature.WRITE_DURATIONS_AS_TIMESTAMPS) should ===(false)
notConfigured.isEnabled(DeserializationFeature.EAGER_DESERIALIZER_FETCH) should ===(true)
notConfigured.isEnabled(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY) should ===(false)
notConfigured.isEnabled(JsonParser.Feature.ALLOW_COMMENTS) should ===(false)
notConfigured.isEnabled(JsonGenerator.Feature.AUTO_CLOSE_TARGET) should ===(true)
notConfigured.isEnabled(StreamReadFeature.STRICT_DUPLICATE_DETECTION) should ===(false)
notConfigured.isEnabled(StreamWriteFeature.WRITE_BIGDECIMAL_AS_PLAIN) should ===(false)
notConfigured.isEnabled(JsonParser.Feature.ALLOW_YAML_COMMENTS) should ===(false)
notConfigured.isEnabled(JsonGenerator.Feature.ESCAPE_NON_ASCII) should ===(false)
}
}
}
}
"JacksonJsonSerializer with Java message classes" must {
import JavaTestMessages._
// see SerializationFeature.WRITE_DATES_AS_TIMESTAMPS = off
"by default serialize dates and durations as text with ISO-8601 date format" in {
// Default format is defined in com.fasterxml.jackson.databind.util.StdDateFormat
// ISO-8601 yyyy-MM-dd'T'HH:mm:ss.SSSZ (rfc3339)
val msg = new TimeCommand(LocalDateTime.of(2019, 4, 29, 23, 15, 3, 12345), Duration.of(5, ChronoUnit.SECONDS))
val json = serializeToJsonString(msg)
val expected = """{"timestamp":"2019-04-29T23:15:03.000012345","duration":"PT5S"}"""
json should ===(expected)
// and full round trip
checkSerialization(msg)
// and it can still deserialize from numeric timestamps format
val serializer = serializerFor(msg)
val manifest = serializer.manifest(msg)
val serializerId = serializer.identifier
val deserializedFromTimestampsFormat = deserializeFromJsonString(
"""{"timestamp":[2019,4,29,23,15,3,12345],"duration":5.000000000}""",
serializerId,
manifest)
deserializedFromTimestampsFormat should ===(msg)
}
// see SerializationFeature.WRITE_DATES_AS_TIMESTAMPS = on
"be possible to serialize dates and durations as numeric timestamps" in {
withSystem("""
akka.serialization.jackson.serialization-features {
WRITE_DATES_AS_TIMESTAMPS = on
WRITE_DURATIONS_AS_TIMESTAMPS = on
}
""") { sys =>
val msg = new TimeCommand(LocalDateTime.of(2019, 4, 29, 23, 15, 3, 12345), Duration.of(5, ChronoUnit.SECONDS))
val json = serializeToJsonString(msg, sys)
val expected = """{"timestamp":[2019,4,29,23,15,3,12345],"duration":5.000000000}"""
json should ===(expected)
// and full round trip
checkSerialization(msg, sys)
// and it can still deserialize from ISO format
val serializer = serializerFor(msg, sys)
val manifest = serializer.manifest(msg)
val serializerId = serializer.identifier
val deserializedFromIsoFormat = deserializeFromJsonString(
"""{"timestamp":"2019-04-29T23:15:03.000012345","duration":"PT5S"}""",
serializerId,
manifest,
sys)
deserializedFromIsoFormat should ===(msg)
}
}
"serialize Instant as text with ISO-8601 date format (default)" in {
val msg = new InstantCommand(Instant.ofEpochMilli(1559907792075L))
val json = serializeToJsonString(msg)
val expected = """{"instant":"2019-06-07T11:43:12.075Z"}"""
json should ===(expected)
// and full round trip
checkSerialization(msg)
}
// FAIL_ON_UNKNOWN_PROPERTIES = off is default in reference.conf
"not fail on unknown properties" in {
val json = """{"name":"abc","name2":"def","name3":"ghi"}"""
val expected = new SimpleCommand2("abc", "def")
val serializer = serializerFor(expected)
deserializeFromJsonString(json, serializer.identifier, serializer.manifest(expected)) should ===(expected)
}
"be possible to create custom ObjectMapper" in {
pending
}
}
"JacksonJsonSerializer with Scala message classes" must {
import ScalaTestMessages._
"be possible to create custom ObjectMapper" in {
val customJacksonObjectMapperFactory = new JacksonObjectMapperFactory {
override def newObjectMapper(bindingName: String, jsonFactory: JsonFactory): ObjectMapper = {
if (bindingName == "jackson-json") {
val mapper: ObjectMapper = JsonMapper.builder(jsonFactory).build()
// some customer configuration of the mapper
mapper.setLocale(Locale.US)
mapper
} else
super.newObjectMapper(bindingName, jsonFactory)
}
override def overrideConfiguredSerializationFeatures(
bindingName: String,
configuredFeatures: immutable.Seq[(SerializationFeature, Boolean)])
: immutable.Seq[(SerializationFeature, Boolean)] = {
if (bindingName == "jackson-json")
configuredFeatures :+ (SerializationFeature.INDENT_OUTPUT -> true)
else
super.overrideConfiguredSerializationFeatures(bindingName, configuredFeatures)
}
override def overrideConfiguredModules(
bindingName: String,
configuredModules: immutable.Seq[Module]): immutable.Seq[Module] =
if (bindingName == "jackson-json")
configuredModules.filterNot(_.isInstanceOf[JavaTimeModule])
else
super.overrideConfiguredModules(bindingName, configuredModules)
override def overrideConfiguredMapperFeatures(
bindingName: String,
configuredFeatures: immutable.Seq[(MapperFeature, Boolean)]): immutable.Seq[(MapperFeature, Boolean)] =
if (bindingName == "jackson-json")
configuredFeatures :+ (MapperFeature.SORT_PROPERTIES_ALPHABETICALLY -> true)
else
super.overrideConfiguredMapperFeatures(bindingName, configuredFeatures)
override def overrideConfiguredJsonParserFeatures(
bindingName: String,
configuredFeatures: immutable.Seq[(JsonParser.Feature, Boolean)])
: immutable.Seq[(JsonParser.Feature, Boolean)] =
if (bindingName == "jackson-json")
configuredFeatures :+ (JsonParser.Feature.ALLOW_SINGLE_QUOTES -> true)
else
super.overrideConfiguredJsonParserFeatures(bindingName, configuredFeatures)
override def overrideConfiguredJsonGeneratorFeatures(
bindingName: String,
configuredFeatures: immutable.Seq[(JsonGenerator.Feature, Boolean)])
: immutable.Seq[(JsonGenerator.Feature, Boolean)] =
if (bindingName == "jackson-json")
configuredFeatures :+ (JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN -> true)
else
super.overrideConfiguredJsonGeneratorFeatures(bindingName, configuredFeatures)
}
val config = system.settings.config
val setup = ActorSystemSetup()
.withSetup(JacksonObjectMapperProviderSetup(customJacksonObjectMapperFactory))
.withSetup(BootstrapSetup(config))
withSystem(setup) { sys =>
val mapper = JacksonObjectMapperProvider(sys).getOrCreate("jackson-json", None)
mapper.isEnabled(SerializationFeature.INDENT_OUTPUT) should ===(true)
mapper.isEnabled(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY) should ===(true)
mapper.isEnabled(JsonParser.Feature.ALLOW_SINGLE_QUOTES) should ===(true)
mapper.isEnabled(SerializationFeature.INDENT_OUTPUT) should ===(true)
mapper.isEnabled(JsonGenerator.Feature.WRITE_BIGDECIMAL_AS_PLAIN) should ===(true)
val msg = InstantCommand(Instant.ofEpochMilli(1559907792075L))
val json = serializeToJsonString(msg, sys)
// using the custom ObjectMapper with pretty printing enabled, and no JavaTimeModule
json should include(""" "instant" : {""")
json should include(""" "nanos" : 75000000,""")
json should include(""" "seconds" : 1559907792""")
}
}
"allow deserialization of classes in configured allowed-class-prefix" in {
val json = """{"name":"abc"}"""
val old = SimpleCommand("abc")
val serializer = serializerFor(old)
val expected = OldCommandNotInBindings("abc")
deserializeFromJsonString(json, serializer.identifier, serializer.manifest(expected)) should ===(expected)
}
"deserialize Enumerations as String when configured" in {
val json = """{"name":"Superman", "planet":"Krypton"}"""
val expected = Superhero("Superman", Planet.Krypton)
val serializer = serializerFor(expected)
deserializeFromJsonString(json, serializer.identifier, serializer.manifest(expected)) should ===(expected)
}
"compress large payload with gzip" in {
val conf = JacksonObjectMapperProvider.configForBinding("jackson-json", system.settings.config)
val compressionAlgo = conf.getString("compression.algorithm")
compressionAlgo should ===("gzip")
val compressLargerThan = conf.getBytes("compression.compress-larger-than")
compressLargerThan should ===(32 * 1024)
val msg = SimpleCommand("0" * (compressLargerThan + 1).toInt)
val bytes = serializeToBinary(msg)
JacksonSerializer.isGZipped(bytes) should ===(true)
bytes.length should be < compressLargerThan.toInt
}
"not compress small payload with gzip" in {
val msg = SimpleCommand("0" * 1000)
val bytes = serializeToBinary(msg)
JacksonSerializer.isGZipped(bytes) should ===(false)
}
"compress large payload with lz4" in withSystem("""
akka.serialization.jackson.jackson-json.compression {
algorithm = lz4
compress-larger-than = 32 KiB
}
""") { sys =>
val conf = JacksonObjectMapperProvider.configForBinding("jackson-json", sys.settings.config)
val compressLargerThan = conf.getBytes("compression.compress-larger-than")
def check(msg: AnyRef, compressed: Boolean): Unit = {
val bytes = serializeToBinary(msg, sys)
JacksonSerializer.isLZ4(bytes) should ===(compressed)
bytes.length should be < compressLargerThan.toInt
checkSerialization(msg, sys)
}
check(SimpleCommand("0" * (compressLargerThan + 1).toInt), true)
}
"not compress small payload with lz4" in withSystem("""
akka.serialization.jackson.jackson-json.compression {
algorithm = lz4
compress-larger-than = 32 KiB
}
""") { sys =>
val conf = JacksonObjectMapperProvider.configForBinding("jackson-json", sys.settings.config)
val compressLargerThan = conf.getBytes("compression.compress-larger-than")
def check(msg: AnyRef, compressed: Boolean): Unit = {
val bytes = serializeToBinary(msg, sys)
JacksonSerializer.isLZ4(bytes) should ===(compressed)
bytes.length should be < compressLargerThan.toInt
checkSerialization(msg, sys)
}
check(SimpleCommand("Bob"), false)
check(new SimpleCommandNotCaseClass("Bob"), false)
}
}
"JacksonJsonSerializer without type in manifest" should {
import ScalaTestMessages._
"deserialize messages using the serialization bindings" in withSystem(
"""
akka.actor {
serializers.animal = "akka.serialization.jackson.JacksonJsonSerializer"
serialization-identifiers.animal = 9091
serialization-bindings {
"akka.serialization.jackson.ScalaTestMessages$Animal" = animal
}
}
akka.serialization.jackson.animal.type-in-manifest = off
""") { sys =>
val msg = Elephant("Dumbo", 1)
val serializer = serializerFor(msg, sys)
serializer.manifest(msg) should ===("")
val bytes = serializer.toBinary(msg)
val deserialized = serializer.fromBinary(bytes, "")
deserialized should ===(msg)
}
"deserialize messages using the configured deserialization type" in withSystem(
"""
akka.actor {
serializers.animal = "akka.serialization.jackson.JacksonJsonSerializer"
serialization-identifiers.animal = 9091
serialization-bindings {
"akka.serialization.jackson.ScalaTestMessages$Elephant" = animal
"akka.serialization.jackson.ScalaTestMessages$Lion" = animal
}
}
akka.serialization.jackson.animal {
type-in-manifest = off
deserialization-type = "akka.serialization.jackson.ScalaTestMessages$Animal"
}
""") { sys =>
val msg = Elephant("Dumbo", 1)
val serializer = serializerFor(msg, sys)
serializer.manifest(msg) should ===("")
val bytes = serializer.toBinary(msg)
val deserialized = serializer.fromBinary(bytes, "")
deserialized should ===(msg)
}
"fail if multiple serialization bindings are declared with no deserialization type" in {
an[IllegalArgumentException] should be thrownBy {
withSystem("""
akka.actor {
serializers.animal = "akka.serialization.jackson.JacksonJsonSerializer"
serialization-identifiers.animal = 9091
serialization-bindings {
"akka.serialization.jackson.ScalaTestMessages$Elephant" = animal
"akka.serialization.jackson.ScalaTestMessages$Lion" = animal
}
}
akka.serialization.jackson.animal {
type-in-manifest = off
}
""")(sys => checkSerialization(Elephant("Dumbo", 1), sys))
}
}
// issue #28918
"cbor compatibility for reading json" in {
val msg = SimpleCommand("abc")
val jsonSerializer = serializerFor(msg)
jsonSerializer.identifier should ===(31)
val manifest = jsonSerializer.manifest(msg)
val bytes = jsonSerializer.toBinary(msg)
val deserialized = serialization().deserialize(bytes, 32, manifest).get
deserialized should be(msg)
}
}
}
abstract class JacksonSerializerSpec(serializerName: String)
extends TestKit(
ActorSystem(
"JacksonJsonSerializerSpec",
ConfigFactory.parseString(s"""
akka.serialization.jackson.migrations {
"akka.serialization.jackson.JavaTestMessages$$Event1" = "akka.serialization.jackson.JavaTestEventMigration"
"akka.serialization.jackson.JavaTestMessages$$Event2" = "akka.serialization.jackson.JavaTestEventMigration"
"akka.serialization.jackson.ScalaTestMessages$$Event1" = "akka.serialization.jackson.ScalaTestEventMigration"
"akka.serialization.jackson.ScalaTestMessages$$Event2" = "akka.serialization.jackson.ScalaTestEventMigration"
}
akka.actor {
serialization-bindings {
"akka.serialization.jackson.ScalaTestMessages$$TestMessage" = $serializerName
"akka.serialization.jackson.JavaTestMessages$$TestMessage" = $serializerName
}
}
akka.serialization.jackson.allowed-class-prefix = ["akka.serialization.jackson.ScalaTestMessages$$OldCommand"]
""")))
with AnyWordSpecLike
with Matchers
with BeforeAndAfterAll {
def serialization(sys: ActorSystem = system): Serialization = SerializationExtension(sys)
override def afterAll(): Unit = {
shutdown()
}
def withSystem[T](config: String)(block: ActorSystem => T): T = {
val sys = ActorSystem(system.name, ConfigFactory.parseString(config).withFallback(system.settings.config))
try {
block(sys)
} finally shutdown(sys)
}
def withSystem[T](setup: ActorSystemSetup)(block: ActorSystem => T): T = {
val sys = ActorSystem(system.name, setup)
try {
block(sys)
} finally shutdown(sys)
}
def withTransportInformation[T](sys: ActorSystem = system)(block: () => T): T = {
Serialization.withTransportInformation(sys.asInstanceOf[ExtendedActorSystem]) { () =>
block()
}
}
def checkSerialization(obj: AnyRef, sys: ActorSystem = system): Unit = {
val serializer = serializerFor(obj, sys)
val manifest = serializer.manifest(obj)
val serializerId = serializer.identifier
val blob = serializeToBinary(obj, sys)
// Issue #28918, check that CBOR format is used (not JSON).
if (blob.length > 0) {
serializer match {
case _: JacksonJsonSerializer =>
if (!JacksonSerializer.isGZipped(blob) && !JacksonSerializer.isLZ4(blob))
new String(blob.take(1), StandardCharsets.UTF_8) should ===("{")
case _: JacksonCborSerializer =>
new String(blob.take(1), StandardCharsets.UTF_8) should !==("{")
case _ =>
throw new IllegalArgumentException(s"Unexpected serializer $serializer")
}
}
val deserialized = deserializeFromBinary(blob, serializerId, manifest, sys)
deserialized should ===(obj)
}
def serializeToBinary(obj: AnyRef, sys: ActorSystem = system): Array[Byte] =
serialization(sys).serialize(obj).get
def deserializeFromBinary(
blob: Array[Byte],
serializerId: Int,
manifest: String,
sys: ActorSystem = system): AnyRef = {
// TransportInformation added by serialization.deserialize
serialization(sys).deserialize(blob, serializerId, manifest).get
}
def serializerFor(obj: AnyRef, sys: ActorSystem = system): JacksonSerializer =
serialization(sys).findSerializerFor(obj) match {
case serializer: JacksonSerializer => serializer
case s =>
throw new IllegalStateException(s"Wrong serializer ${s.getClass} for ${obj.getClass}")
}
"JacksonSerializer with Java message classes" must {
import JavaTestMessages._
"serialize simple message with one constructor parameter" in {
checkSerialization(new SimpleCommand("Bob"))
}
"serialize simple message with two constructor parameters" in {
checkSerialization(new SimpleCommand2("Bob", "Alice"))
checkSerialization(new SimpleCommand2("Bob", ""))
checkSerialization(new SimpleCommand2("Bob", null))
}
"serialize message with boolean property" in {
checkSerialization(new BooleanCommand(true))
checkSerialization(new BooleanCommand(false))
}
"serialize message with Optional property" in {
checkSerialization(new OptionalCommand(Optional.of("abc")))
checkSerialization(new OptionalCommand(Optional.empty()))
}
"serialize message with collections" in {
val strings = Arrays.asList("a", "b", "c")
val objects = Arrays.asList(new SimpleCommand("a"), new SimpleCommand("2"))
val msg = new CollectionsCommand(strings, objects)
checkSerialization(msg)
}
"serialize message with time" in {
val msg = new TimeCommand(LocalDateTime.now(), Duration.of(5, ChronoUnit.SECONDS))
checkSerialization(msg)
}
"serialize with ActorRef" in {
val echo = system.actorOf(TestActors.echoActorProps)
checkSerialization(new CommandWithActorRef("echo", echo))
}
"serialize with typed.ActorRef" in {
import akka.actor.typed.scaladsl.adapter._
val ref = system.spawnAnonymous(Behaviors.empty[String])
checkSerialization(new CommandWithTypedActorRef("echo", ref))
}
"serialize with Address" in {
val address = Address("akka", "sys", "localhost", 2552)
checkSerialization(new CommandWithAddress("echo", address))
}
"serialize with polymorphism" in {
checkSerialization(new Zoo(new Lion("Simba")))
checkSerialization(new Zoo(new Elephant("Elephant", 49)))
intercept[InvalidTypeIdException] {
// Cockroach not listed in JsonSubTypes
checkSerialization(new Zoo(new Cockroach("huh")))
}
}
"deserialize with migrations" in {
val event1 = new Event1("a")
val serializer = serializerFor(event1)
val blob = serializer.toBinary(event1)
val event2 = serializer.fromBinary(blob, classOf[Event1].getName).asInstanceOf[Event2]
event1.getField1 should ===(event2.getField1V2)
event2.getField2 should ===(17)
}
"deserialize with migrations from V2" in {
val event1 = new Event1("a")
val serializer = serializerFor(event1)
val blob = serializer.toBinary(event1)
val event2 = serializer.fromBinary(blob, classOf[Event1].getName + "#2").asInstanceOf[Event2]
event1.getField1 should ===(event2.getField1V2)
event2.getField2 should ===(17)
}
}
"JacksonSerializer with Scala message classes" must {
import ScalaTestMessages._
"serialize simple message with one constructor parameter" in {
checkSerialization(SimpleCommand("Bob"))
checkSerialization(new SimpleCommandNotCaseClass("Bob"))
}
"serialize simple message with two constructor parameters" in {
checkSerialization(SimpleCommand2("Bob", "Alice"))
checkSerialization(SimpleCommand2("Bob", ""))
checkSerialization(SimpleCommand2("Bob", null))
}
"serialize message with boolean property" in {
checkSerialization(BooleanCommand(true))
checkSerialization(BooleanCommand(false))
}
"serialize message with Enumeration property (using Jackson legacy format)" in {
checkSerialization(Alien("E.T.", Planet.Mars))
}
"serialize message with Enumeration property as a String" in {
checkSerialization(Superhero("Kal El", Planet.Krypton))
}
"serialize message with Optional property" in {
checkSerialization(OptionCommand(Some("abc")))
checkSerialization(OptionCommand(None))
}
"serialize message with collections" in {
val strings = "a" :: "b" :: "c" :: Nil
val objects = Vector(SimpleCommand("a"), SimpleCommand("2"))
val msg = CollectionsCommand(strings, objects)
checkSerialization(msg)
}
"serialize message with time" in {
val msg = TimeCommand(LocalDateTime.now(), 5.seconds)
checkSerialization(msg)
}
"serialize FiniteDuration as java.time.Duration" in {
withTransportInformation() { () =>
val scalaMsg = TimeCommand(LocalDateTime.now(), 5.seconds)
val scalaSerializer = serializerFor(scalaMsg)
val blob = scalaSerializer.toBinary(scalaMsg)
val javaMsg = new JavaTestMessages.TimeCommand(scalaMsg.timestamp, Duration.ofSeconds(5))
val javaSerializer = serializerFor(javaMsg)
val deserialized = javaSerializer.fromBinary(blob, javaSerializer.manifest(javaMsg))
deserialized should ===(javaMsg)
}
}
"serialize message with UUID property" in {
val uuid = UUID.randomUUID()
checkSerialization(UUIDCommand(uuid))
}
"serialize case object" in {
checkSerialization(TopLevelSingletonCaseObject)
checkSerialization(SingletonCaseObject)
}
"serialize with ActorRef" in {
val echo = system.actorOf(TestActors.echoActorProps)
checkSerialization(CommandWithActorRef("echo", echo))
}
"serialize with typed.ActorRef" in {
import akka.actor.typed.scaladsl.adapter._
val ref = system.spawnAnonymous(Behaviors.empty[String])
checkSerialization(CommandWithTypedActorRef("echo", ref))
}
"serialize with Address" in {
val address = Address("akka", "sys", "localhost", 2552)
checkSerialization(CommandWithAddress("echo", address))
}
"serialize with polymorphism" in {
checkSerialization(Zoo(Lion("Simba")))
checkSerialization(Zoo(Elephant("Elephant", 49)))
intercept[InvalidTypeIdException] {
// Cockroach not listed in JsonSubTypes
checkSerialization(Zoo(Cockroach("huh")))
}
}
"deserialize with migrations" in {
val event1 = Event1("a")
val serializer = serializerFor(event1)
val blob = serializer.toBinary(event1)
val event2 = serializer.fromBinary(blob, classOf[Event1].getName).asInstanceOf[Event2]
event1.field1 should ===(event2.field1V2)
event2.field2 should ===(17)
}
"deserialize with migrations from V2" in {
val event1 = Event1("a")
val serializer = serializerFor(event1)
val blob = serializer.toBinary(event1)
val event2 = serializer.fromBinary(blob, classOf[Event1].getName + "#2").asInstanceOf[Event2]
event1.field1 should ===(event2.field1V2)
event2.field2 should ===(17)
}
"not allow serialization of deny listed class" in {
val serializer = serializerFor(SimpleCommand("ok"))
val fileHandler = new FileHandler(s"target/tmp-${this.getClass.getName}")
try {
intercept[IllegalArgumentException] {
serializer.manifest(fileHandler)
}.getMessage.toLowerCase should include("deny list")
} finally fileHandler.close()
}
"not allow deserialization of deny list class" in {
withTransportInformation() { () =>
val msg = SimpleCommand("ok")
val serializer = serializerFor(msg)
val blob = serializer.toBinary(msg)
intercept[IllegalArgumentException] {
// maliciously changing manifest
serializer.fromBinary(blob, classOf[FileHandler].getName)
}.getMessage.toLowerCase should include("deny list")
}
}
"not allow serialization of class that is not in serialization-bindings (allowed-class-prefix)" in {
val serializer = serializerFor(SimpleCommand("ok"))
intercept[IllegalArgumentException] {
serializer.manifest(Status.Success("bad"))
}.getMessage.toLowerCase should include("allowed-class-prefix")
}
"not allow deserialization of class that is not in serialization-bindings (allowed-class-prefix)" in {
withTransportInformation() { () =>
val msg = SimpleCommand("ok")
val serializer = serializerFor(msg)
val blob = serializer.toBinary(msg)
intercept[IllegalArgumentException] {
// maliciously changing manifest
serializer.fromBinary(blob, classOf[Status.Success].getName)
}.getMessage.toLowerCase should include("allowed-class-prefix")
}
}
"not allow serialization-bindings of open-ended types" in {
JacksonSerializer.disallowedSerializationBindings.foreach { clazz =>
val className = clazz.getName
withClue(className) {
intercept[IllegalArgumentException] {
val sys = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.actor.serialization-bindings {
"$className" = $serializerName
"akka.serialization.jackson.ScalaTestMessages$$TestMessage" = $serializerName
}
""").withFallback(system.settings.config))
try {
SerializationExtension(sys).serialize(SimpleCommand("hi")).get
} finally shutdown(sys)
}
}
}
}
}
}
case object TopLevelSingletonCaseObject extends ScalaTestMessages.TestMessage