Disable Java serialization by default, #22333 (#27285)

* akka.actor.allow-java-serialization = off
* Moved primitive (Long, Int, String, ByteString) serializers
  from akka-remote to akka-actor since they had no dependency
  and are useful also in local systems, e.g. persistence.
  * e.g. needed for persistence-tck
  * less allow-java-serialization=on in tests
* CborSerializable in Jackson/test module for ease of use
* JavaSerializable for Java serialization in tests, already in akka-testkit,
  but misconfigured
* Made tests pass
  * allow-java-serialization=on in akka-persistence
  * allow-java-serialization=on in classic remoting tests
  * JavaSerializable and CborSerializable in other remoting tests
* Added serialization for
  * Boolean
  * java.util.concurrent.TimeoutException, AskTimeoutException
* support for testing serialization with the inmem journal
* utility to verifySerialization, in SerializationTestKit
* remove AccountExampleWithCommandHandlersInState becuase not possible to serialize State when it's not static
  * Effect() is factory in EventSourcedBehavior  class
* test the account examples
* SharedLeveldbJournal.configToEnableJavaSerializationForTest
* support for exceptions from remote deployed child actors
  * fallback to akka.remote.serialization.ThrowableNotSerializableException
    if exception is not serializable when wrapped in system messages from
    remote deployed child actors and Status.Failure messages
  * it's implemented in `WrappedPayloadSupport.payloadBuilder`
* update reference documentation
* serialize-messages=off in most places, separate ticket for
  improving or removing that feature
* migration guide, including description of rolling update

* fix 2.13 compiler error

* minor review feedback
This commit is contained in:
Patrik Nordwall 2019-07-11 14:04:24 +02:00 committed by GitHub
parent a4f090b622
commit 3efc1c2877
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
191 changed files with 4041 additions and 2321 deletions

View file

@ -199,4 +199,9 @@ final class ActorTestKit private[akka] (delegate: akka.actor.testkit.typed.scala
*/
def shutdownTestKit(): Unit = delegate.shutdownTestKit()
/**
* Additional testing utilities for serialization.
*/
val serializationTestKit: SerializationTestKit = new SerializationTestKit(delegate.internalSystem)
}

View file

@ -0,0 +1,28 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
import akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
/**
* Utilities to test serialization.
*/
class SerializationTestKit(system: ActorSystem[_]) {
private val delegate = new scaladsl.SerializationTestKit(system)
/**
* Verify serialization roundtrip.
* Throws exception from serializer if `obj` can't be serialized and deserialized.
*
* @param obj the object to verify
* @param assertEquality if `true` the deserialized object is verified to be equal to `obj`,
* and if not an `AssertionError` is thrown
* @return the deserialized object
*/
def verifySerialization[M](obj: M, assertEquality: Boolean): M =
delegate.verifySerialization(obj, assertEquality)
}

View file

@ -139,6 +139,11 @@ final class TestKitJunitResource(_kit: ActorTestKit) extends ExternalResource {
*/
def stop[T](ref: ActorRef[T]): Unit = testKit.stop(ref)
/**
* Additional testing utilities for serialization.
*/
def serializationTestKit: SerializationTestKit = testKit.serializationTestKit
override def after(): Unit = {
testKit.shutdownTestKit()
}

View file

@ -6,23 +6,23 @@ package akka.actor.testkit.typed.scaladsl
import java.util.concurrent.TimeoutException
import akka.actor.typed.scaladsl.AskPattern._
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.ActorTestKitGuardian
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.scaladsl.AskPattern._
import akka.annotation.InternalApi
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.internal.ActorTestKitGuardian
import akka.actor.testkit.typed.internal.TestKitUtils
import akka.util.Timeout
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.util.Timeout
object ActorTestKit {
/**
@ -112,7 +112,10 @@ final class ActorTestKit private[akka] (val name: String, val config: Config, se
implicit def testKitSettings: TestKitSettings =
settings.getOrElse(TestKitSettings(system))
private val internalSystem: ActorSystem[ActorTestKitGuardian.TestKitCommand] =
/**
* INTERNAL API
*/
@InternalApi private[akka] val internalSystem: ActorSystem[ActorTestKitGuardian.TestKitCommand] =
if (config eq ActorTestKit.noConfigSet) ActorSystem(ActorTestKitGuardian.testKitGuardian, name)
else ActorSystem(ActorTestKitGuardian.testKitGuardian, name, config)
@ -185,6 +188,11 @@ final class ActorTestKit private[akka] (val name: String, val config: Config, se
*/
def createTestProbe[M](name: String): TestProbe[M] = TestProbe(name)(system)
/**
* Additional testing utilities for serialization.
*/
val serializationTestKit: SerializationTestKit = new SerializationTestKit(internalSystem)
// FIXME needed for Akka internal tests but, users shouldn't spawn system actors?
@InternalApi
private[akka] def systemActor[T](behavior: Behavior[T], name: String): ActorRef[T] =

View file

@ -100,6 +100,11 @@ abstract class ActorTestKitBase(val testKit: ActorTestKit) {
*/
def createTestProbe[M](name: String): TestProbe[M] = testKit.createTestProbe(name)
/**
* Additional testing utilities for serialization.
*/
def serializationTestKit: SerializationTestKit = testKit.serializationTestKit
/**
* To be implemented by "more" concrete class that can mixin `BeforeAndAfterAll` or similar,
* for example `FlatSpecLike with BeforeAndAfterAll`. Implement by calling

View file

@ -0,0 +1,60 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.ActorSystem
import akka.serialization.SerializationExtension
import akka.serialization.Serializers
/**
* Utilities to test serialization.
*/
class SerializationTestKit(system: ActorSystem[_]) {
private val serialization = SerializationExtension(system.toUntyped)
/**
* Verify serialization roundtrip.
* Throws exception from serializer if `obj` can't be serialized and deserialized.
* Also tests that the deserialized object is equal to `obj`, and if not an
* `AssertionError` is thrown.
*
* @param obj the object to verify
* @return the deserialized object
*/
def verifySerialization[M](obj: M): M =
verifySerialization(obj, assertEquality = true)
/**
* Verify serialization roundtrip.
* Throws exception from serializer if `obj` can't be serialized and deserialized.
*
* @param obj the object to verify
* @param assertEquality if `true` the deserialized object is verified to be equal to `obj`,
* and if not an `AssertionError` is thrown
* @return the deserialized object
*/
def verifySerialization[M](obj: M, assertEquality: Boolean): M = {
val result = roundtrip(obj)
if (assertEquality && result != obj)
throw new AssertionError(s"Serialization verification expected $obj, but was $result")
result
}
/**
* Verify serialization roundtrip.
* Throws exception from serializer if `obj` can't be serialized and deserialized.
*
* @return the deserialized object
*/
private def roundtrip[M](obj: M): M = {
val objAnyRef = obj.asInstanceOf[AnyRef]
val bytes = serialization.serialize(objAnyRef).get
val serializer = serialization.findSerializerFor(objAnyRef)
val manifest = Serializers.manifestFor(serializer, objAnyRef)
serialization.deserialize(bytes, serializer.identifier, manifest).get.asInstanceOf[M]
}
}

View file

@ -3,8 +3,7 @@ akka {
library-extensions += "akka.actor.InstanceCountingExtension"
actor {
serialize-messages = on
warn-about-java-serializer-usage = off
serialize-messages = off
}
}

View file

@ -109,7 +109,10 @@ object ActorRefSpec {
}
}
class ActorRefSpec extends AkkaSpec with DefaultTimeout {
class ActorRefSpec extends AkkaSpec("""
# testing Java serialization of ActorRef
akka.actor.allow-java-serialization = on
""") with DefaultTimeout {
import akka.actor.ActorRefSpec._
def promiseIntercept(f: => Actor)(to: Promise[Actor]): Actor =

View file

@ -36,7 +36,12 @@ object FunctionRefSpec {
}
class FunctionRefSpec extends AkkaSpec with ImplicitSender {
class FunctionRefSpec extends AkkaSpec("""
# test is using Java serialization and relies on serialize-messages=on
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
akka.actor.serialize-messages = on
""") with ImplicitSender {
import FunctionRefSpec._
def commonTests(s: ActorRef) = {

View file

@ -28,7 +28,12 @@ object PropsCreationSpec {
}
class PropsCreationSpec extends AkkaSpec("akka.actor.serialize-creators = on") {
class PropsCreationSpec extends AkkaSpec("""
# test is using Java serialization and relies on serialize-creators=on
akka.actor.serialize-creators = on
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
""") {
import akka.actor.PropsCreationSpec._

View file

@ -36,6 +36,9 @@ object TypedActorSpec {
}
akka.actor.serializers.sample = "akka.actor.TypedActorSpec$SampleSerializerWithStringManifest$"
akka.actor.serialization-bindings."akka.actor.TypedActorSpec$WithStringSerializedClass" = sample
# test is using Java serialization and not priority to convert
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
akka.actor.serialize-messages = off
"""

View file

@ -31,10 +31,8 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
getBoolean("akka.daemonic") should ===(false)
// WARNING: This setting should be off in the default reference.conf, but should be on when running
// the test suite.
getBoolean("akka.actor.serialize-messages") should ===(true)
settings.SerializeAllMessages should ===(true)
getBoolean("akka.actor.serialize-messages") should ===(false)
settings.SerializeAllMessages should ===(false)
getInt("akka.scheduler.ticks-per-wheel") should ===(512)
getDuration("akka.scheduler.tick-duration", TimeUnit.MILLISECONDS) should ===(10L)
@ -76,6 +74,9 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
getBoolean("akka.coordinated-shutdown.run-by-actor-system-terminate") should ===(true)
settings.CoordinatedShutdownRunByActorSystemTerminate should ===(true)
getBoolean("akka.actor.allow-java-serialization") should ===(false)
settings.AllowJavaSerialization should ===(false)
}
{

View file

@ -18,6 +18,11 @@ import akka.testkit._
object ConsistentHashingRouterSpec {
val config = """
akka.actor {
serialize-messages = off
# consistent hashing is serializing the hash key, unless it's bytes or string
allow-java-serialization = on
}
akka.actor.deployment {
/router1 {
router = consistent-hashing-pool

View file

@ -0,0 +1,176 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.serialization
import java.nio.ByteBuffer
import java.nio.ByteOrder
import scala.util.Random
import akka.testkit.AkkaSpec
import akka.util.ByteString
import com.typesafe.config.ConfigFactory
object PrimitivesSerializationSpec {
val serializationTestOverrides = ""
val testConfig = ConfigFactory.parseString(serializationTestOverrides).withFallback(AkkaSpec.testConf)
}
class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.testConfig) {
val buffer = {
val b = ByteBuffer.allocate(4096)
b.order(ByteOrder.LITTLE_ENDIAN)
b
}
val serialization = SerializationExtension(system)
def verifySerialization(msg: AnyRef): Unit = {
val serializer = serialization.serializerFor(msg.getClass)
serializer.fromBinary(serializer.toBinary(msg), None) should ===(msg)
}
def verifySerializationByteBuffer(msg: AnyRef): Unit = {
val serializer = serialization.serializerFor(msg.getClass).asInstanceOf[Serializer with ByteBufferSerializer]
buffer.clear()
serializer.toBinary(msg, buffer)
buffer.flip()
// also make sure that the Array and ByteBuffer formats are equal, given LITTLE_ENDIAN
val array1 = new Array[Byte](buffer.remaining())
buffer.get(array1)
val array2 = serializer.toBinary(msg)
ByteString(array1) should ===(ByteString(array2))
buffer.rewind()
serializer.fromBinary(buffer, "") should ===(msg)
}
"LongSerializer" must {
Seq(0L, 1L, -1L, Long.MinValue, Long.MinValue + 1L, Long.MaxValue, Long.MaxValue - 1L)
.map(_.asInstanceOf[AnyRef])
.foreach { item =>
s"resolve serializer for value $item" in {
serialization.serializerFor(item.getClass).getClass should ===(classOf[LongSerializer])
}
s"serialize and de-serialize value $item" in {
verifySerialization(item)
}
s"serialize and de-serialize value $item using ByteBuffers" in {
verifySerializationByteBuffer(item)
}
}
"have right serializer id" in {
// checking because moved to akka-actor
serialization.serializerFor(1L.asInstanceOf[AnyRef].getClass).identifier === 18
}
}
"IntSerializer" must {
Seq(0, 1, -1, Int.MinValue, Int.MinValue + 1, Int.MaxValue, Int.MaxValue - 1).map(_.asInstanceOf[AnyRef]).foreach {
item =>
s"resolve serializer for value $item" in {
serialization.serializerFor(item.getClass).getClass should ===(classOf[IntSerializer])
}
s"serialize and de-serialize value $item" in {
verifySerialization(item)
}
s"serialize and de-serialize value $item using ByteBuffers" in {
verifySerializationByteBuffer(item)
}
}
"have right serializer id" in {
// checking because moved to akka-actor
serialization.serializerFor(1L.asInstanceOf[AnyRef].getClass).identifier === 19
}
}
"Boolean" must {
Seq(false, true, java.lang.Boolean.FALSE, java.lang.Boolean.TRUE).map(_.asInstanceOf[AnyRef]).zipWithIndex.foreach {
case (item, i) =>
s"resolve serializer for value $item ($i)" in {
serialization.serializerFor(item.getClass).getClass should ===(classOf[BooleanSerializer])
}
s"serialize and de-serialize value $item ($i)" in {
verifySerialization(item)
}
s"serialize and de-serialize value $item ($i) using ByteBuffers" in {
verifySerializationByteBuffer(item)
}
}
"have right serializer id ($i)" in {
// checking because moved to akka-actor
serialization.serializerFor(true.asInstanceOf[AnyRef].getClass).identifier === 35
}
}
"StringSerializer" must {
val random = Random.nextString(256)
Seq("empty string" -> "", "hello" -> "hello", "árvíztűrőütvefúrógép" -> "árvíztűrőütvefúrógép", "random" -> random)
.foreach {
case (scenario, item) =>
s"resolve serializer for [$scenario]" in {
serialization.serializerFor(item.getClass).getClass should ===(classOf[StringSerializer])
}
s"serialize and de-serialize [$scenario]" in {
verifySerialization(item)
}
s"serialize and de-serialize value [$scenario] using ByteBuffers" in {
verifySerializationByteBuffer(item)
}
}
"have right serializer id" in {
// checking because moved to akka-actor
serialization.serializerFor(1L.asInstanceOf[AnyRef].getClass).identifier === 20
}
}
"ByteStringSerializer" must {
Seq(
"empty string" -> ByteString.empty,
"simple content" -> ByteString("hello"),
"concatenated content" -> (ByteString("hello") ++ ByteString("world")),
"sliced content" -> ByteString("helloabc").take(5),
"large concatenated" ->
(ByteString(Array.fill[Byte](1000)(1)) ++ ByteString(Array.fill[Byte](1000)(2)))).foreach {
case (scenario, item) =>
s"resolve serializer for [$scenario]" in {
val serializer = SerializationExtension(system)
serializer.serializerFor(item.getClass).getClass should ===(classOf[ByteStringSerializer])
}
s"serialize and de-serialize [$scenario]" in {
verifySerialization(item)
}
s"serialize and de-serialize value [$scenario] using ByteBuffers" in {
verifySerializationByteBuffer(item)
}
}
"have right serializer id" in {
// checking because moved to akka-actor
serialization.serializerFor(1L.asInstanceOf[AnyRef].getClass).identifier === 21
}
}
}

View file

@ -11,11 +11,9 @@ import java.util.{ Date => SerializableDummy }
import akka.actor.setup.ActorSystemSetup
import akka.actor.{ ActorSystem, BootstrapSetup, ExtendedActorSystem }
import akka.testkit.{ AkkaSpec, TestKit, TestProbe }
import akka.testkit.{ AkkaSpec, TestKit }
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
class ConfigurationDummy
class ProgrammaticDummy
@ -58,6 +56,8 @@ object SerializationSetupSpec {
akka {
actor {
serialize-messages = off
allow-java-serialization = on
# this is by default on, but tests are running with off, use defaults here
warn-about-java-serializer-usage = on
@ -91,6 +91,14 @@ class SerializationSetupSpec
import SerializationSetupSpec._
private def verifySerialization(sys: ActorSystem, obj: AnyRef): Unit = {
val serialization = SerializationExtension(sys)
val bytes = serialization.serialize(obj).get
val serializer = serialization.findSerializerFor(obj)
val manifest = Serializers.manifestFor(serializer, obj)
serialization.deserialize(bytes, serializer.identifier, manifest).get
}
"The serialization settings" should {
"allow for programmatic configuration of serializers" in {
@ -165,15 +173,18 @@ class SerializationSetupSpec
}
"have replaced java serializer" in {
val p = TestProbe()(addedJavaSerializationViaSettingsSystem) // only receiver has the serialization disabled
// allow-java-serialization = on in `system`
val serializer = SerializationExtension(system).findSerializerFor(new ProgrammaticJavaDummy)
serializer.getClass should ===(classOf[JavaSerializer])
p.ref ! new ProgrammaticJavaDummy
SerializationExtension(system).findSerializerFor(new ProgrammaticJavaDummy).toBinary(new ProgrammaticJavaDummy)
// should not receive this one, it would have been java serialization!
p.expectNoMessage(100.millis)
// should not allow deserialization, it would have been java serialization!
val serializer2 =
SerializationExtension(addedJavaSerializationViaSettingsSystem).findSerializerFor(new ProgrammaticJavaDummy)
serializer2.getClass should ===(classOf[DisabledJavaSerializer])
serializer2.identifier should ===(serializer.identifier)
p.ref ! new ProgrammaticDummy
p.expectMsgType[ProgrammaticDummy]
verifySerialization(system, new ProgrammaticDummy)
verifySerialization(addedJavaSerializationViaSettingsSystem, new ProgrammaticDummy)
}
"disable java serialization also for incoming messages if serializer id usually would have found the serializer" in {

View file

@ -5,14 +5,16 @@
package akka.serialization
import language.postfixOps
import akka.testkit.{ AkkaSpec, EventFilter }
import akka.actor._
import java.io._
import scala.concurrent.Await
import akka.util.{ unused, Timeout }
import akka.util.{ unused, Timeout }
import scala.concurrent.duration._
import com.typesafe.config._
import akka.pattern.ask
import java.nio.ByteOrder
@ -22,6 +24,7 @@ import akka.actor.dungeon.SerializationCheckFailedException
import com.github.ghik.silencer.silent
import test.akka.serialization.NoVerification
import SerializationTests._
import akka.util.ByteString
object SerializationTests {
@ -38,12 +41,12 @@ object SerializationTests {
serialization-bindings {
"akka.serialization.SerializationTests$$Person" = java
"akka.serialization.SerializationTests$$Address" = java
"akka.serialization.TestSerializable" = test
"akka.serialization.SerializationTests$$Marker" = test
"akka.serialization.SerializationTests$$PlainMessage" = test
"akka.serialization.SerializationTests$$A" = java
"akka.serialization.SerializationTests$$B" = test
"akka.serialization.SerializationTests$$D" = test
"akka.serialization.TestSerializable2" = test2
"akka.serialization.SerializationTests$$Marker2" = test2
"akka.serialization.SerializationTests$$AbstractOther" = other
}
}
@ -56,13 +59,15 @@ object SerializationTests {
final case class Record(id: Int, person: Person)
protected[akka] trait Marker
protected[akka] trait Marker2
@silent // can't use unused otherwise case class below gets a deprecated
class SimpleMessage(s: String) extends TestSerializable
class SimpleMessage(s: String) extends Marker
@silent
class ExtendedSimpleMessage(s: String, i: Int) extends SimpleMessage(s)
trait AnotherInterface extends TestSerializable
trait AnotherInterface extends Marker
class AnotherMessage extends AnotherInterface
@ -74,7 +79,7 @@ object SerializationTests {
class BothTestSerializableAndJavaSerializable(s: String) extends SimpleMessage(s) with Serializable
class BothTestSerializableAndTestSerializable2(@unused s: String) extends TestSerializable with TestSerializable2
class BothTestSerializableAndTestSerializable2(@unused s: String) extends Marker with Marker2
trait A
trait B
@ -93,6 +98,7 @@ object SerializationTests {
actor {
serialize-messages = on
serialize-creators = on
allow-java-serialization = on
}
}
"""
@ -120,6 +126,18 @@ object SerializationTests {
ActorSystem("SerializationSystem", mostlyReferenceConf)
}
def allowJavaSerializationSystem: ActorSystem = {
val referenceConf = ConfigFactory.defaultReference()
val conf = ConfigFactory
.parseString("""
akka.actor.warn-about-java-serializer-usage = on
akka.actor.allow-java-serialization = on
""")
.withFallback(ConfigFactory.parseString(serializeConf))
.withFallback(AkkaSpec.testConf.withFallback(referenceConf))
ActorSystem("SerializationSystem", conf)
}
val systemMessageMultiSerializerConf = """
akka {
actor {
@ -139,36 +157,18 @@ object SerializationTests {
class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) {
val ser = SerializationExtension(system)
import ser._
val address = SerializationTests.Address("120", "Monroe Street", "Santa Clara", "95050")
val person = SerializationTests.Person(
"debasish ghosh",
25,
SerializationTests.Address("120", "Monroe Street", "Santa Clara", "95050"))
"Serialization" must {
"have correct bindings" in {
ser.bindings.collectFirst { case (c, s) if c == address.getClass => s.getClass } should ===(
Some(classOf[JavaSerializer]))
Some(classOf[DisabledJavaSerializer]))
ser.bindings.collectFirst { case (c, s) if c == classOf[PlainMessage] => s.getClass } should ===(
Some(classOf[NoopSerializer]))
}
"serialize Address" in {
assert(deserialize(serialize(address).get, classOf[SerializationTests.Address]).get === address)
}
"serialize Person" in {
assert(deserialize(serialize(person).get, classOf[Person]).get === person)
}
"serialize record with default serializer" in {
val r = Record(100, person)
assert(deserialize(serialize(r).get, classOf[Record]).get === r)
}
"not serialize ActorCell" in {
val a = system.actorOf(Props(new Actor {
def receive = {
@ -182,25 +182,6 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) {
system.stop(a)
}
"serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf)
val a = ActorSystem("SerializeDeadLeterActorRef", AkkaSpec.testConf)
try {
out.writeObject(a.deadLetters)
out.flush()
out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
JavaSerializer.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) should ===(true)
}
} finally {
shutdown(a)
}
}
"resolve serializer by direct interface" in {
ser.serializerFor(classOf[SimpleMessage]).getClass should ===(classOf[NoopSerializer])
}
@ -237,14 +218,14 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) {
}
"resolve serializer in the order of the bindings" in {
ser.serializerFor(classOf[A]).getClass should ===(classOf[JavaSerializer])
ser.serializerFor(classOf[A]).getClass should ===(classOf[DisabledJavaSerializer])
ser.serializerFor(classOf[B]).getClass should ===(classOf[NoopSerializer])
// JavaSerializer lower prio when multiple found
ser.serializerFor(classOf[C]).getClass should ===(classOf[NoopSerializer])
}
"resolve serializer in the order of most specific binding first" in {
ser.serializerFor(classOf[A]).getClass should ===(classOf[JavaSerializer])
ser.serializerFor(classOf[A]).getClass should ===(classOf[DisabledJavaSerializer])
ser.serializerFor(classOf[D]).getClass should ===(classOf[NoopSerializer])
ser.serializerFor(classOf[E]).getClass should ===(classOf[NoopSerializer])
}
@ -337,15 +318,67 @@ class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyRefer
"Serialization settings from reference.conf" must {
"declare Serializable classes to be use JavaSerializer" in {
serializerMustBe(classOf[Serializable], classOf[JavaSerializer])
serializerMustBe(classOf[String], classOf[JavaSerializer])
"declare Serializable classes to be use DisabledJavaSerializer" in {
serializerMustBe(classOf[Serializable], classOf[DisabledJavaSerializer])
}
"declare Array[Byte] to use ByteArraySerializer" in {
serializerMustBe(classOf[Array[Byte]], classOf[ByteArraySerializer])
}
"declare Long, Int, String, ByteString to use primitive serializers" in {
serializerMustBe(classOf[java.lang.Long], classOf[LongSerializer])
serializerMustBe(classOf[java.lang.Integer], classOf[IntSerializer])
serializerMustBe(classOf[String], classOf[StringSerializer])
serializerMustBe(classOf[ByteString.ByteString1], classOf[ByteStringSerializer])
serializerMustBe(classOf[ByteString.ByteString1C], classOf[ByteStringSerializer])
serializerMustBe(classOf[ByteString.ByteStrings], classOf[ByteStringSerializer])
}
"not support serialization for other classes" in {
intercept[NotSerializableException] { ser.serializerFor(classOf[Object]) }
}
"not allow serialize function" in {
val f = (i: Int) => i + 1
serializerMustBe(f.getClass, classOf[DisabledJavaSerializer])
}
}
}
class AllowJavaSerializationSpec extends AkkaSpec(SerializationTests.allowJavaSerializationSystem) {
val ser = SerializationExtension(system)
def serializerMustBe(toSerialize: Class[_], expectedSerializer: Class[_]) =
ser.serializerFor(toSerialize).getClass should ===(expectedSerializer)
val address = SerializationTests.Address("120", "Monroe Street", "Santa Clara", "95050")
val person = SerializationTests.Person(
"debasish ghosh",
25,
SerializationTests.Address("120", "Monroe Street", "Santa Clara", "95050"))
val messagePrefix = "Using the Java serializer for class"
"Serialization settings with allow-java-serialization = on" must {
"declare Serializable classes to be use JavaSerializer" in {
serializerMustBe(classOf[Serializable], classOf[JavaSerializer])
}
"declare Array[Byte] to use ByteArraySerializer" in {
serializerMustBe(classOf[Array[Byte]], classOf[ByteArraySerializer])
}
"declare Long, Int, String, ByteString to use primitive serializers" in {
serializerMustBe(classOf[java.lang.Long], classOf[LongSerializer])
serializerMustBe(classOf[java.lang.Integer], classOf[IntSerializer])
serializerMustBe(classOf[String], classOf[StringSerializer])
serializerMustBe(classOf[ByteString.ByteString1], classOf[ByteStringSerializer])
}
"not support serialization for other classes" in {
intercept[NotSerializableException] { ser.serializerFor(classOf[Object]) }
}
@ -359,17 +392,6 @@ class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyRefer
f2(3) should ===(4)
}
}
}
class DefaultSerializationWarningSpec
extends AkkaSpec(ConfigFactory.parseString("akka.actor.warn-about-java-serializer-usage = on")) {
val ser = SerializationExtension(system)
val messagePrefix = "Using the default Java serializer for class"
"Using the default Java serializer" must {
"log a warning when serializing classes outside of java.lang package" in {
EventFilter.warning(start = messagePrefix, occurrences = 1).intercept {
ser.serializerFor(classOf[java.math.BigDecimal])
@ -382,20 +404,72 @@ class DefaultSerializationWarningSpec
}
}
}
"have correct bindings" in {
ser.bindings.collectFirst { case (c, s) if c == address.getClass => s.getClass } should ===(
Some(classOf[JavaSerializer]))
ser.bindings.collectFirst { case (c, s) if c == classOf[PlainMessage] => s.getClass } should ===(
Some(classOf[NoopSerializer]))
}
"serialize Address" in {
assert(ser.deserialize(ser.serialize(address).get, classOf[SerializationTests.Address]).get === address)
}
"serialize Person" in {
assert(ser.deserialize(ser.serialize(person).get, classOf[Person]).get === person)
}
"serialize record with Java serializer" in {
val r = Record(100, person)
assert(ser.deserialize(ser.serialize(r).get, classOf[Record]).get === r)
}
"not serialize ActorCell" in {
val a = system.actorOf(Props(new Actor {
def receive = {
case o: ObjectOutputStream =>
try o.writeObject(this)
catch { case _: NotSerializableException => testActor ! "pass" }
}
}))
a ! new ObjectOutputStream(new ByteArrayOutputStream())
expectMsg("pass")
system.stop(a)
}
"serialize DeadLetterActorRef" in {
val outbuf = new ByteArrayOutputStream()
val out = new ObjectOutputStream(outbuf)
val a = ActorSystem("SerializeDeadLeterActorRef", AkkaSpec.testConf)
try {
out.writeObject(a.deadLetters)
out.flush()
out.close()
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
JavaSerializer.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) {
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
(deadLetters eq a.deadLetters) should ===(true)
}
} finally {
shutdown(a)
}
}
}
}
class NoVerificationWarningSpec
extends AkkaSpec(
ConfigFactory.parseString(
"akka.actor.warn-about-java-serializer-usage = on\n" +
"akka.actor.warn-on-no-serialization-verification = on")) {
extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = on
akka.actor.warn-on-no-serialization-verification = on
""")) {
val ser = SerializationExtension(system)
val messagePrefix = "Using the default Java serializer for class"
val messagePrefix = "Using the Java serializer for class"
"When warn-on-no-serialization-verification = on, using the default Java serializer" must {
"When warn-on-no-serialization-verification = on, using the Java serializer" must {
"log a warning on classes without extending NoSerializationVerificationNeeded" in {
EventFilter.warning(start = messagePrefix, occurrences = 1).intercept {
@ -412,15 +486,16 @@ class NoVerificationWarningSpec
}
class NoVerificationWarningOffSpec
extends AkkaSpec(
ConfigFactory.parseString(
"akka.actor.warn-about-java-serializer-usage = on\n" +
"akka.actor.warn-on-no-serialization-verification = off")) {
extends AkkaSpec(ConfigFactory.parseString("""
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = on
akka.actor.warn-on-no-serialization-verification = off
""")) {
val ser = SerializationExtension(system)
val messagePrefix = "Using the default Java serializer for class"
val messagePrefix = "Using the Java serializer for class"
"When warn-on-no-serialization-verification = off, using the default Java serializer" must {
"When warn-on-no-serialization-verification = off, using the Java serializer" must {
"log a warning on classes without extending NoSerializationVerificationNeeded" in {
EventFilter.warning(start = messagePrefix, occurrences = 1).intercept {
@ -436,9 +511,6 @@ class NoVerificationWarningOffSpec
}
}
protected[akka] trait TestSerializable
protected[akka] trait TestSerializable2
protected[akka] class NoopSerializer extends Serializer {
def includeManifest: Boolean = false

View file

@ -16,7 +16,9 @@ object ActorRefSerializationSpec {
def config = ConfigFactory.parseString("""
akka.actor {
serialize-messages = off
allow-java-serialization = true
# test is verifying Java serialization of ActorRef
allow-java-serialization = on
warn-about-java-serializer-usage = off
}
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0

View file

@ -705,13 +705,29 @@ akka {
#
# Please note that this option does not stop you from manually invoking java serialization
#
# The default value for this might be changed to off in future versions of Akka.
allow-java-serialization = on
allow-java-serialization = off
# Log warnings when the Java serialization is used to serialize messages.
# Java serialization is not very performant and should not be used in production
# environments unless you don't care about performance and security. In that case
# you can turn this off.
warn-about-java-serializer-usage = on
# To be used with the above warn-about-java-serializer-usage
# When warn-about-java-serializer-usage = on, and this warn-on-no-serialization-verification = off,
# warnings are suppressed for classes extending NoSerializationVerificationNeeded
# to reduce noise.
warn-on-no-serialization-verification = on
# Entries for pluggable serializers and their bindings.
serializers {
java = "akka.serialization.JavaSerializer"
bytes = "akka.serialization.ByteArraySerializer"
primitive-long = "akka.serialization.LongSerializer"
primitive-int = "akka.serialization.IntSerializer"
primitive-string = "akka.serialization.StringSerializer"
primitive-bytestring = "akka.serialization.ByteStringSerializer"
primitive-boolean = "akka.serialization.BooleanSerializer"
}
# Class to Serializer binding. You only need to specify the name of an
@ -724,20 +740,19 @@ akka {
serialization-bindings {
"[B" = bytes
"java.io.Serializable" = java
"java.lang.String" = primitive-string
"akka.util.ByteString$ByteString1C" = primitive-bytestring
"akka.util.ByteString$ByteString1" = primitive-bytestring
"akka.util.ByteString$ByteStrings" = primitive-bytestring
"java.lang.Long" = primitive-long
"scala.Long" = primitive-long
"java.lang.Integer" = primitive-int
"scala.Int" = primitive-int
"java.lang.Boolean" = primitive-boolean
"scala.Boolean" = primitive-boolean
}
# Log warnings when the default Java serialization is used to serialize messages.
# The default serializer uses Java serialization which is not very performant and should not
# be used in production environments unless you don't care about performance. In that case
# you can turn this off.
warn-about-java-serializer-usage = on
# To be used with the above warn-about-java-serializer-usage
# When warn-about-java-serializer-usage = on, and this warn-on-no-serialization-verification = off,
# warnings are suppressed for classes extending NoSerializationVerificationNeeded
# to reduce noize.
warn-on-no-serialization-verification = on
# Configuration namespace of serialization identifiers.
# Each serializer implementation must have an entry in the following format:
# `akka.actor.serialization-identifiers."FQCN" = ID`
@ -747,6 +762,12 @@ akka {
serialization-identifiers {
"akka.serialization.JavaSerializer" = 1
"akka.serialization.ByteArraySerializer" = 4
primitive-long = 18
primitive-int = 19
primitive-string = 20
primitive-bytestring = 21
primitive-boolean = 35
}
}

View file

@ -0,0 +1,198 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.serialization
import java.nio.{ BufferOverflowException, ByteBuffer }
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.util.ByteString
/**
* INTERNAL API: only public by configuration
*/
@InternalApi private[akka] final class LongSerializer(val system: ExtendedActorSystem)
extends Serializer
with ByteBufferSerializer {
override def includeManifest: Boolean = false
override val identifier: Int = BaseSerializer.identifierFromConfig("primitive-long", system)
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = {
buf.putLong(Long.unbox(o))
}
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
Long.box(buf.getLong)
}
override def toBinary(o: AnyRef): Array[Byte] = {
val result = new Array[Byte](8)
var long = Long.unbox(o)
var i = 0
while (long != 0) {
result(i) = (long & 0xFF).toByte
i += 1
long >>>= 8
}
result
}
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = {
var result = 0L
var i = 7
while (i >= 0) {
result <<= 8
result |= (bytes(i).toLong & 0xFF)
i -= 1
}
Long.box(result)
}
}
/**
* INTERNAL API: only public by configuration
*/
@InternalApi private[akka] final class IntSerializer(val system: ExtendedActorSystem)
extends Serializer
with ByteBufferSerializer {
override def includeManifest: Boolean = false
override val identifier: Int = BaseSerializer.identifierFromConfig("primitive-int", system)
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.putInt(Int.unbox(o))
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = Int.box(buf.getInt)
override def toBinary(o: AnyRef): Array[Byte] = {
val result = new Array[Byte](4)
var int = Int.unbox(o)
var i = 0
while (int != 0) {
result(i) = (int & 0xFF).toByte
i += 1
int >>>= 8
}
result
}
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = {
var result = 0
var i = 3
while (i >= 0) {
result <<= 8
result |= (bytes(i).toInt & 0xFF)
i -= 1
}
Int.box(result)
}
}
/**
* INTERNAL API: only public by configuration
*/
@InternalApi private[akka] final class StringSerializer(val system: ExtendedActorSystem)
extends Serializer
with ByteBufferSerializer {
override def includeManifest: Boolean = false
override val identifier: Int = BaseSerializer.identifierFromConfig("primitive-string", system)
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.put(toBinary(o))
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
val bytes = new Array[Byte](buf.remaining())
buf.get(bytes)
new String(bytes, "UTF-8")
}
override def toBinary(o: AnyRef): Array[Byte] = o.asInstanceOf[String].getBytes("UTF-8")
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = new String(bytes, "UTF-8")
}
/**
* INTERNAL API: only public by configuration
*/
@InternalApi private[akka] final class ByteStringSerializer(val system: ExtendedActorSystem)
extends Serializer
with ByteBufferSerializer {
override def includeManifest: Boolean = false
override val identifier: Int = BaseSerializer.identifierFromConfig("primitive-bytestring", system)
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = {
val bs = o.asInstanceOf[ByteString]
// ByteString.copyToBuffer does not throw BufferOverflowException
if (bs.copyToBuffer(buf) < bs.length)
throw new BufferOverflowException()
}
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef =
ByteString.fromByteBuffer(buf)
override def toBinary(o: AnyRef): Array[Byte] = {
val bs = o.asInstanceOf[ByteString]
val result = new Array[Byte](bs.length)
bs.copyToArray(result, 0, bs.length)
result
}
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = {
ByteString(bytes)
}
}
/**
* INTERNAL API: only public by configuration
*/
@InternalApi private[akka] final class BooleanSerializer(val system: ExtendedActorSystem)
extends Serializer
with ByteBufferSerializer {
import java.lang.Boolean.{ FALSE, TRUE }
private val FalseB = 0.toByte
private val TrueB = 1.toByte
override def includeManifest: Boolean = false
override val identifier: Int = BaseSerializer.identifierFromConfig("primitive-boolean", system)
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = {
val flag = o match {
case TRUE => TrueB
case FALSE => FalseB
}
buf.put(flag)
}
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
buf.get() match {
case TrueB => TRUE
case FalseB => FALSE
}
}
override def toBinary(o: AnyRef): Array[Byte] = {
val flag = o match {
case TRUE => TrueB
case FALSE => FalseB
}
val result = new Array[Byte](1)
result(0) = flag
result
}
override def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = {
bytes(0) match {
case TrueB => TRUE
case FalseB => FALSE
}
}
}

View file

@ -350,7 +350,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
if (shouldWarnAboutJavaSerializer(clazz, ser)) {
_log.warning(
LogMarker.Security,
"Using the default Java serializer for class [{}] which is not recommended because of " +
"Using the 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)
@ -550,4 +550,5 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
!serializedClass.getName.startsWith("java.lang.") &&
!suppressWarningOnNonSerializationVerification(serializedClass)
}
}

View file

@ -4,18 +4,22 @@
package akka.serialization
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, NotSerializableException, ObjectOutputStream }
import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream
import java.io.NotSerializableException
import java.io.ObjectOutputStream
import java.nio.ByteBuffer
import java.util.concurrent.Callable
import akka.util.ClassLoaderObjectInputStream
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.event.{ LogMarker, Logging }
import scala.util.DynamicVariable
import scala.util.control.NoStackTrace
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.event.LogMarker
import akka.event.Logging
import akka.util.ClassLoaderObjectInputStream
/**
* A Serializer represents a bimap between an object and an array of bytes representing that object.
*

View file

@ -41,7 +41,6 @@ class ORSetSerializationBenchmark {
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
""")

View file

@ -9,6 +9,7 @@ import java.lang.management.ManagementFactory
import scala.concurrent.Await
import scala.concurrent.duration._
import com.typesafe.config.Config
import com.typesafe.config.ConfigFactory
import akka.actor._
@ -23,6 +24,7 @@ import akka.routing.ActorRefRoutee
import akka.routing.Routees
import akka.cluster.routing.ClusterRouterPool
import akka.cluster.routing.ClusterRouterPoolSettings
import akka.serialization.jackson.CborSerializable
import akka.util.unused
object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
@ -52,7 +54,7 @@ object AdaptiveLoadBalancingRouterConfig extends MultiNodeConfig {
}
case object AllocateMemory
final case class Reply(address: Address)
final case class Reply(address: Address) extends CborSerializable
val node1 = role("node-1")
val node2 = role("node-2")

View file

@ -4,8 +4,10 @@
package akka.cluster.metrics.sample
import akka.serialization.jackson.CborSerializable
//#messages
final case class StatsJob(text: String)
final case class StatsResult(meanWordLength: Double)
final case class JobFailed(reason: String)
final case class StatsJob(text: String) extends CborSerializable
final case class StatsResult(meanWordLength: Double) extends CborSerializable
final case class JobFailed(reason: String) extends CborSerializable
//#messages

View file

@ -1,7 +1,6 @@
akka {
actor {
serialize-messages = on
warn-about-java-serializer-usage = off
serialize-messages = off
}
}

View file

@ -13,7 +13,6 @@ import akka.cluster.TestMember
class MessageSerializerSpec extends AkkaSpec("""
akka.actor.provider = cluster
akka.actor.serialize-messages = off
akka.actor.allow-java-serialization = off
""") {
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])

View file

@ -31,7 +31,8 @@ public class ClusterShardingPersistenceTest extends JUnitSuite {
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n");
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n"
+ "akka.persistence.journal.inmem.test-serialization = on \n");
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(config);

View file

@ -35,7 +35,8 @@ public class AccountExampleTest extends JUnitSuite {
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n");
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n"
+ "akka.persistence.journal.inmem.test-serialization = on \n");
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(config);
@ -142,4 +143,47 @@ public class AccountExampleTest extends JUnitSuite {
.get(3, TimeUnit.SECONDS);
assertEquals(BigDecimal.valueOf(90), balance);
}
@Test
public void verifySerialization() {
TestProbe<OperationResult> opProbe = testKit.createTestProbe();
testKit.serializationTestKit().verifySerialization(new CreateAccount(opProbe.getRef()), false);
Deposit deposit2 =
testKit
.serializationTestKit()
.verifySerialization(new Deposit(BigDecimal.valueOf(100), opProbe.getRef()), false);
assertEquals(BigDecimal.valueOf(100), deposit2.amount);
assertEquals(opProbe.getRef(), deposit2.replyTo());
testKit
.serializationTestKit()
.verifySerialization(new Withdraw(BigDecimal.valueOf(90), opProbe.getRef()), false);
testKit.serializationTestKit().verifySerialization(new CloseAccount(opProbe.getRef()), false);
testKit.serializationTestKit().verifySerialization(Confirmed.INSTANCE, false);
testKit.serializationTestKit().verifySerialization(new Rejected("overdraft"), false);
TestProbe<CurrentBalance> getProbe = testKit.createTestProbe();
testKit.serializationTestKit().verifySerialization(new GetBalance(getProbe.getRef()), false);
testKit
.serializationTestKit()
.verifySerialization(new CurrentBalance(BigDecimal.valueOf(100)), false);
testKit.serializationTestKit().verifySerialization(new AccountCreated(), false);
testKit
.serializationTestKit()
.verifySerialization(new Deposited(BigDecimal.valueOf(100)), false);
testKit
.serializationTestKit()
.verifySerialization(new Withdrawn(BigDecimal.valueOf(90)), false);
testKit.serializationTestKit().verifySerialization(new AccountClosed(), false);
testKit.serializationTestKit().verifySerialization(new EmptyAccount(), false);
OpenedAccount openedAccount2 =
testKit
.serializationTestKit()
.verifySerialization(new OpenedAccount(BigDecimal.valueOf(100)), false);
assertEquals(BigDecimal.valueOf(100), openedAccount2.balance);
testKit.serializationTestKit().verifySerialization(new ClosedAccount(), false);
}
}

View file

@ -1,297 +0,0 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.cluster.sharding.typed;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors;
import akka.cluster.sharding.typed.javadsl.EntityTypeKey;
import akka.cluster.sharding.typed.javadsl.EventSourcedEntityWithEnforcedReplies;
import akka.persistence.typed.ExpectingReply;
import akka.persistence.typed.PersistenceId;
import akka.persistence.typed.javadsl.CommandHandlerWithReply;
import akka.persistence.typed.javadsl.CommandHandlerWithReplyBuilder;
import akka.persistence.typed.javadsl.EventHandler;
import akka.persistence.typed.javadsl.EventHandlerBuilder;
import akka.persistence.typed.javadsl.ReplyEffect;
import java.math.BigDecimal;
/**
* Bank account example illustrating: - different state classes representing the lifecycle of the
* account - event handlers that delegate to methods in the state classes - command handlers that
* delegate to methods in the state classes - replies of various types, using ExpectingReply and
* EventSourcedBehaviorWithEnforcedReplies
*/
public interface AccountExampleWithCommandHandlersInState {
// #account-entity
public class AccountEntity
extends EventSourcedEntityWithEnforcedReplies<
AccountEntity.AccountCommand, AccountEntity.AccountEvent, AccountEntity.Account> {
public static final EntityTypeKey<AccountCommand> ENTITY_TYPE_KEY =
EntityTypeKey.create(AccountCommand.class, "Account");
// Command
interface AccountCommand<Reply> extends ExpectingReply<Reply> {}
public static class CreateAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
public CreateAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@Override
public ActorRef<OperationResult> replyTo() {
return replyTo;
}
}
public static class Deposit implements AccountCommand<OperationResult> {
public final BigDecimal amount;
private final ActorRef<OperationResult> replyTo;
public Deposit(BigDecimal amount, ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
this.amount = amount;
}
@Override
public ActorRef<OperationResult> replyTo() {
return replyTo;
}
}
public static class Withdraw implements AccountCommand<OperationResult> {
public final BigDecimal amount;
private final ActorRef<OperationResult> replyTo;
public Withdraw(BigDecimal amount, ActorRef<OperationResult> replyTo) {
this.amount = amount;
this.replyTo = replyTo;
}
@Override
public ActorRef<OperationResult> replyTo() {
return replyTo;
}
}
public static class GetBalance implements AccountCommand<CurrentBalance> {
private final ActorRef<CurrentBalance> replyTo;
public GetBalance(ActorRef<CurrentBalance> replyTo) {
this.replyTo = replyTo;
}
@Override
public ActorRef<CurrentBalance> replyTo() {
return replyTo;
}
}
public static class CloseAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
public CloseAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@Override
public ActorRef<OperationResult> replyTo() {
return replyTo;
}
}
// Reply
interface AccountCommandReply {}
interface OperationResult extends AccountCommandReply {}
enum Confirmed implements OperationResult {
INSTANCE
}
public static class Rejected implements OperationResult {
public final String reason;
public Rejected(String reason) {
this.reason = reason;
}
}
public static class CurrentBalance implements AccountCommandReply {
public final BigDecimal balance;
public CurrentBalance(BigDecimal balance) {
this.balance = balance;
}
}
// Event
interface AccountEvent {}
public static class AccountCreated implements AccountEvent {}
public static class Deposited implements AccountEvent {
public final BigDecimal amount;
Deposited(BigDecimal amount) {
this.amount = amount;
}
}
public static class Withdrawn implements AccountEvent {
public final BigDecimal amount;
Withdrawn(BigDecimal amount) {
this.amount = amount;
}
}
public static class AccountClosed implements AccountEvent {}
// State
interface Account {}
public class EmptyAccount implements Account {
ReplyEffect<AccountEvent, Account> createAccount(CreateAccount command) {
return Effect()
.persist(new AccountCreated())
.thenReply(command, account2 -> Confirmed.INSTANCE);
}
OpenedAccount openedAccount() {
return new OpenedAccount(BigDecimal.ZERO);
}
}
public class OpenedAccount implements Account {
public final BigDecimal balance;
public OpenedAccount(BigDecimal balance) {
this.balance = balance;
}
ReplyEffect<AccountEvent, Account> deposit(Deposit command) {
return Effect()
.persist(new Deposited(command.amount))
.thenReply(command, account2 -> Confirmed.INSTANCE);
}
ReplyEffect<AccountEvent, Account> withdraw(Withdraw command) {
if (!canWithdraw(command.amount)) {
return Effect()
.reply(command, new Rejected("not enough funds to withdraw " + command.amount));
} else {
return Effect()
.persist(new Withdrawn(command.amount))
.thenReply(command, account2 -> Confirmed.INSTANCE);
}
}
ReplyEffect<AccountEvent, Account> getBalance(GetBalance command) {
return Effect().reply(command, new CurrentBalance(balance));
}
ReplyEffect<AccountEvent, Account> closeAccount(CloseAccount command) {
if (balance.equals(BigDecimal.ZERO)) {
return Effect()
.persist(new AccountClosed())
.thenReply(command, account2 -> Confirmed.INSTANCE);
} else {
return Effect().reply(command, new Rejected("balance must be zero for closing account"));
}
}
OpenedAccount makeDeposit(BigDecimal amount) {
return new OpenedAccount(balance.add(amount));
}
boolean canWithdraw(BigDecimal amount) {
return (balance.subtract(amount).compareTo(BigDecimal.ZERO) >= 0);
}
OpenedAccount makeWithdraw(BigDecimal amount) {
if (!canWithdraw(amount))
throw new IllegalStateException("Account balance can't be negative");
return new OpenedAccount(balance.subtract(amount));
}
ClosedAccount closedAccount() {
return new ClosedAccount();
}
}
public static class ClosedAccount implements Account {}
public static AccountEntity create(String accountNumber) {
return new AccountEntity(accountNumber);
}
public AccountEntity(String accountNumber) {
super(ENTITY_TYPE_KEY, accountNumber);
}
@Override
public Account emptyState() {
return new EmptyAccount();
}
@Override
public CommandHandlerWithReply<AccountCommand, AccountEvent, Account> commandHandler() {
CommandHandlerWithReplyBuilder<AccountCommand, AccountEvent, Account> builder =
newCommandHandlerWithReplyBuilder();
builder
.forStateType(EmptyAccount.class)
.onCommand(CreateAccount.class, EmptyAccount::createAccount);
builder
.forStateType(OpenedAccount.class)
.onCommand(Deposit.class, OpenedAccount::deposit)
.onCommand(Withdraw.class, OpenedAccount::withdraw)
.onCommand(GetBalance.class, OpenedAccount::getBalance)
.onCommand(CloseAccount.class, OpenedAccount::closeAccount);
builder
.forStateType(ClosedAccount.class)
.onAnyCommand(() -> Effect().unhandled().thenNoReply());
return builder.build();
}
@Override
public EventHandler<Account, AccountEvent> eventHandler() {
EventHandlerBuilder<Account, AccountEvent> builder = newEventHandlerBuilder();
builder
.forStateType(EmptyAccount.class)
.onEvent(AccountCreated.class, (account, event) -> account.openedAccount());
builder
.forStateType(OpenedAccount.class)
.onEvent(
Deposited.class,
(account, deposited) -> {
account.makeDeposit(deposited.amount);
return account;
})
.onEvent(
Withdrawn.class,
(account, withdrawn) -> {
account.makeWithdraw(withdrawn.amount);
return account;
})
.onEvent(AccountClosed.class, (account, closed) -> account.closedAccount());
return builder.build();
}
}
// #account-entity
}

View file

@ -13,6 +13,8 @@ import akka.persistence.typed.javadsl.CommandHandlerWithReplyBuilder;
import akka.persistence.typed.javadsl.EventHandler;
import akka.persistence.typed.javadsl.EventHandlerBuilder;
import akka.persistence.typed.javadsl.ReplyEffect;
import akka.serialization.jackson.CborSerializable;
import com.fasterxml.jackson.annotation.JsonCreator;
import java.math.BigDecimal;
@ -36,12 +38,13 @@ public interface AccountExampleWithEventHandlersInState {
// Command
// #reply-command
interface AccountCommand<Reply> extends ExpectingReply<Reply> {}
interface AccountCommand<Reply> extends ExpectingReply<Reply>, CborSerializable {}
// #reply-command
public static class CreateAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CreateAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -85,6 +88,7 @@ public interface AccountExampleWithEventHandlersInState {
public static class GetBalance implements AccountCommand<CurrentBalance> {
private final ActorRef<CurrentBalance> replyTo;
@JsonCreator
public GetBalance(ActorRef<CurrentBalance> replyTo) {
this.replyTo = replyTo;
}
@ -98,6 +102,7 @@ public interface AccountExampleWithEventHandlersInState {
public static class CloseAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CloseAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -110,7 +115,7 @@ public interface AccountExampleWithEventHandlersInState {
// Reply
// #reply-command
interface AccountCommandReply {}
interface AccountCommandReply extends CborSerializable {}
interface OperationResult extends AccountCommandReply {}
@ -121,6 +126,7 @@ public interface AccountExampleWithEventHandlersInState {
public static class Rejected implements OperationResult {
public final String reason;
@JsonCreator
public Rejected(String reason) {
this.reason = reason;
}
@ -130,19 +136,21 @@ public interface AccountExampleWithEventHandlersInState {
public static class CurrentBalance implements AccountCommandReply {
public final BigDecimal balance;
@JsonCreator
public CurrentBalance(BigDecimal balance) {
this.balance = balance;
}
}
// Event
interface AccountEvent {}
interface AccountEvent extends CborSerializable {}
public static class AccountCreated implements AccountEvent {}
public static class Deposited implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Deposited(BigDecimal amount) {
this.amount = amount;
}
@ -151,6 +159,7 @@ public interface AccountExampleWithEventHandlersInState {
public static class Withdrawn implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Withdrawn(BigDecimal amount) {
this.amount = amount;
}
@ -159,7 +168,7 @@ public interface AccountExampleWithEventHandlersInState {
public static class AccountClosed implements AccountEvent {}
// State
interface Account {}
interface Account extends CborSerializable {}
public static class EmptyAccount implements Account {
OpenedAccount openedAccount() {
@ -168,8 +177,9 @@ public interface AccountExampleWithEventHandlersInState {
}
public static class OpenedAccount implements Account {
private final BigDecimal balance;
final BigDecimal balance;
@JsonCreator
public OpenedAccount(BigDecimal balance) {
this.balance = balance;
}

View file

@ -16,6 +16,8 @@ import akka.persistence.typed.javadsl.CommandHandlerWithReplyBuilder;
import akka.persistence.typed.javadsl.EventHandler;
import akka.persistence.typed.javadsl.EventHandlerBuilder;
import akka.persistence.typed.javadsl.ReplyEffect;
import akka.serialization.jackson.CborSerializable;
import com.fasterxml.jackson.annotation.JsonCreator;
import java.math.BigDecimal;
@ -36,11 +38,12 @@ public interface AccountExampleWithMutableState {
EntityTypeKey.create(AccountCommand.class, "Account");
// Command
interface AccountCommand<Reply> extends ExpectingReply<Reply> {}
interface AccountCommand<Reply> extends ExpectingReply<Reply>, CborSerializable {}
public static class CreateAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CreateAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -84,6 +87,7 @@ public interface AccountExampleWithMutableState {
public static class GetBalance implements AccountCommand<CurrentBalance> {
private final ActorRef<CurrentBalance> replyTo;
@JsonCreator
public GetBalance(ActorRef<CurrentBalance> replyTo) {
this.replyTo = replyTo;
}
@ -97,6 +101,7 @@ public interface AccountExampleWithMutableState {
public static class CloseAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CloseAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -108,7 +113,7 @@ public interface AccountExampleWithMutableState {
}
// Reply
interface AccountCommandReply {}
interface AccountCommandReply extends CborSerializable {}
interface OperationResult extends AccountCommandReply {}
@ -119,6 +124,7 @@ public interface AccountExampleWithMutableState {
public static class Rejected implements OperationResult {
public final String reason;
@JsonCreator
public Rejected(String reason) {
this.reason = reason;
}
@ -127,19 +133,21 @@ public interface AccountExampleWithMutableState {
public static class CurrentBalance implements AccountCommandReply {
public final BigDecimal balance;
@JsonCreator
public CurrentBalance(BigDecimal balance) {
this.balance = balance;
}
}
// Event
interface AccountEvent {}
interface AccountEvent extends CborSerializable {}
public static class AccountCreated implements AccountEvent {}
public static class Deposited implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Deposited(BigDecimal amount) {
this.amount = amount;
}
@ -148,6 +156,7 @@ public interface AccountExampleWithMutableState {
public static class Withdrawn implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Withdrawn(BigDecimal amount) {
this.amount = amount;
}
@ -156,7 +165,7 @@ public interface AccountExampleWithMutableState {
public static class AccountClosed implements AccountEvent {}
// State
interface Account {}
interface Account extends CborSerializable {}
public static class EmptyAccount implements Account {
OpenedAccount openedAccount() {

View file

@ -18,6 +18,8 @@ import akka.persistence.typed.javadsl.EventHandler;
import akka.persistence.typed.javadsl.EventHandlerBuilder;
import akka.persistence.typed.javadsl.EventSourcedBehaviorWithEnforcedReplies;
import akka.persistence.typed.javadsl.ReplyEffect;
import akka.serialization.jackson.CborSerializable;
import com.fasterxml.jackson.annotation.JsonCreator;
import java.math.BigDecimal;
@ -38,11 +40,12 @@ public interface AccountExampleWithNullState {
EntityTypeKey.create(AccountCommand.class, "Account");
// Command
interface AccountCommand<Reply> extends ExpectingReply<Reply> {}
interface AccountCommand<Reply> extends ExpectingReply<Reply>, CborSerializable {}
public static class CreateAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CreateAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -86,6 +89,7 @@ public interface AccountExampleWithNullState {
public static class GetBalance implements AccountCommand<CurrentBalance> {
private final ActorRef<CurrentBalance> replyTo;
@JsonCreator
public GetBalance(ActorRef<CurrentBalance> replyTo) {
this.replyTo = replyTo;
}
@ -99,6 +103,7 @@ public interface AccountExampleWithNullState {
public static class CloseAccount implements AccountCommand<OperationResult> {
private final ActorRef<OperationResult> replyTo;
@JsonCreator
public CloseAccount(ActorRef<OperationResult> replyTo) {
this.replyTo = replyTo;
}
@ -110,7 +115,7 @@ public interface AccountExampleWithNullState {
}
// Reply
interface AccountCommandReply {}
interface AccountCommandReply extends CborSerializable {}
interface OperationResult extends AccountCommandReply {}
@ -121,6 +126,7 @@ public interface AccountExampleWithNullState {
public static class Rejected implements OperationResult {
public final String reason;
@JsonCreator
public Rejected(String reason) {
this.reason = reason;
}
@ -129,19 +135,21 @@ public interface AccountExampleWithNullState {
public static class CurrentBalance implements AccountCommandReply {
public final BigDecimal balance;
@JsonCreator
public CurrentBalance(BigDecimal balance) {
this.balance = balance;
}
}
// Event
interface AccountEvent {}
interface AccountEvent extends CborSerializable {}
public static class AccountCreated implements AccountEvent {}
public static class Deposited implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Deposited(BigDecimal amount) {
this.amount = amount;
}
@ -150,6 +158,7 @@ public interface AccountExampleWithNullState {
public static class Withdrawn implements AccountEvent {
public final BigDecimal amount;
@JsonCreator
Withdrawn(BigDecimal amount) {
this.amount = amount;
}
@ -158,7 +167,7 @@ public interface AccountExampleWithNullState {
public static class AccountClosed implements AccountEvent {}
// State
interface Account {}
interface Account extends CborSerializable {}
public static class OpenedAccount implements Account {
public final BigDecimal balance;
@ -167,6 +176,7 @@ public interface AccountExampleWithNullState {
this.balance = BigDecimal.ZERO;
}
@JsonCreator
public OpenedAccount(BigDecimal balance) {
this.balance = balance;
}
@ -273,18 +283,8 @@ public interface AccountExampleWithNullState {
builder
.forStateType(OpenedAccount.class)
.onEvent(
Deposited.class,
(account, deposited) -> {
account.makeDeposit(deposited.amount);
return account;
})
.onEvent(
Withdrawn.class,
(account, withdrawn) -> {
account.makeWithdraw(withdrawn.amount);
return account;
})
.onEvent(Deposited.class, (account, deposited) -> account.makeDeposit(deposited.amount))
.onEvent(Withdrawn.class, (account, withdrawn) -> account.makeWithdraw(withdrawn.amount))
.onEvent(AccountClosed.class, (account, closed) -> account.closedAccount());
return builder.build();

View file

@ -28,7 +28,8 @@ public class HelloWorldEventSourcedEntityExampleTest extends JUnitSuite {
+ "akka.remote.classic.netty.tcp.port = 0 \n"
+ "akka.remote.artery.canonical.port = 0 \n"
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n");
+ "akka.persistence.journal.plugin = \"akka.persistence.journal.inmem\" \n"
+ "akka.persistence.journal.inmem.test-serialization = on \n");
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(config);
@ -64,4 +65,20 @@ public class HelloWorldEventSourcedEntityExampleTest extends JUnitSuite {
assertEquals("Bob", greeting2.whom);
assertEquals(2, greeting2.numberOfPeople);
}
@Test
public void testSerialization() {
TestProbe<HelloWorld.Greeting> probe = testKit.createTestProbe(HelloWorld.Greeting.class);
testKit
.serializationTestKit()
.verifySerialization(new HelloWorld.Greet("Alice", probe.getRef()), false);
testKit.serializationTestKit().verifySerialization(new HelloWorld.Greeted("Alice"), false);
testKit.serializationTestKit().verifySerialization(new HelloWorld.Greeted("Alice"), false);
HelloWorld.KnownPeople state = new HelloWorld.KnownPeople();
state = state.add("Alice").add("Bob");
HelloWorld.KnownPeople state2 =
testKit.serializationTestKit().verifySerialization(state, false);
assertEquals(state.numberOfPeople(), state2.numberOfPeople());
}
}

View file

@ -25,7 +25,9 @@ import akka.persistence.typed.javadsl.EventHandler;
import akka.cluster.sharding.typed.javadsl.ClusterSharding;
import akka.cluster.sharding.typed.javadsl.EntityRef;
import akka.cluster.sharding.typed.javadsl.Entity;
import akka.serialization.jackson.CborSerializable;
import akka.util.Timeout;
import com.fasterxml.jackson.annotation.JsonCreator;
// #persistent-entity-usage-import
public class HelloWorldPersistentEntityExample {
@ -65,7 +67,7 @@ public class HelloWorldPersistentEntityExample {
extends EventSourcedEntity<HelloWorld.Command, HelloWorld.Greeted, HelloWorld.KnownPeople> {
// Command
interface Command {}
interface Command extends CborSerializable {}
public static final class Greet implements Command {
public final String whom;
@ -78,7 +80,7 @@ public class HelloWorldPersistentEntityExample {
}
// Response
public static final class Greeting {
public static final class Greeting implements CborSerializable {
public final String whom;
public final int numberOfPeople;
@ -89,16 +91,17 @@ public class HelloWorldPersistentEntityExample {
}
// Event
public static final class Greeted {
public static final class Greeted implements CborSerializable {
public final String whom;
@JsonCreator
public Greeted(String whom) {
this.whom = whom;
}
}
// State
static final class KnownPeople {
static final class KnownPeople implements CborSerializable {
private Set<String> names = Collections.emptySet();
KnownPeople() {}

View file

@ -45,6 +45,7 @@ object ClusterShardingPersistenceSpec {
akka.remote.artery.canonical.hostname = 127.0.0.1
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
akka.persistence.journal.inmem.test-serialization = on
""")
sealed trait Command

View file

@ -51,8 +51,6 @@ object ClusterShardingSpec {
akka.actor {
serialize-messages = off
# issue #24465 missing serializer for GetShardRegionStats
#allow-java-serialization = off
serializers {
test = "akka.cluster.sharding.typed.scaladsl.ClusterShardingSpec$$Serializer"

View file

@ -22,9 +22,10 @@ object AccountExampleSpec {
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.remote.artery.canonical.hostname = 127.0.0.1
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
""")
akka.persistence.journal.inmem.test-serialization = on
""")
}
@ -115,5 +116,30 @@ class AccountExampleSpec extends ScalaTestWithActorTestKit(AccountExampleSpec.co
ref.ask(GetBalance(_)).map(_.balance).futureValue should ===(90)
}
"verifySerialization" in {
val opProbe = createTestProbe[OperationResult]()
serializationTestKit.verifySerialization(CreateAccount(opProbe.ref))
serializationTestKit.verifySerialization(Deposit(100, opProbe.ref))
serializationTestKit.verifySerialization(Withdraw(90, opProbe.ref))
serializationTestKit.verifySerialization(CloseAccount(opProbe.ref))
serializationTestKit.verifySerialization(Confirmed)
serializationTestKit.verifySerialization(Rejected("overdraft"))
val getProbe = createTestProbe[CurrentBalance]()
serializationTestKit.verifySerialization(GetBalance(getProbe.ref))
serializationTestKit.verifySerialization(CurrentBalance(100))
serializationTestKit.verifySerialization(AccountCreated)
serializationTestKit.verifySerialization(Deposited(100))
serializationTestKit.verifySerialization(Withdrawn(90))
serializationTestKit.verifySerialization(AccountClosed)
serializationTestKit.verifySerialization(EmptyAccount)
serializationTestKit.verifySerialization(OpenedAccount(100))
serializationTestKit.verifySerialization(ClosedAccount)
}
}
}

View file

@ -10,6 +10,7 @@ import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
import akka.cluster.sharding.typed.scaladsl.EventSourcedEntity
import akka.persistence.typed.ExpectingReply
import akka.persistence.typed.scaladsl.Effect
import akka.serialization.jackson.CborSerializable
/**
* Bank account example illustrating:
@ -23,7 +24,7 @@ object AccountExampleWithCommandHandlersInState {
//#account-entity
object AccountEntity {
// Command
sealed trait AccountCommand[Reply] extends ExpectingReply[Reply]
sealed trait AccountCommand[Reply] extends ExpectingReply[Reply] with CborSerializable
final case class CreateAccount(override val replyTo: ActorRef[OperationResult])
extends AccountCommand[OperationResult]
final case class Deposit(amount: BigDecimal, override val replyTo: ActorRef[OperationResult])
@ -35,14 +36,14 @@ object AccountExampleWithCommandHandlersInState {
extends AccountCommand[OperationResult]
// Reply
sealed trait AccountCommandReply
sealed trait AccountCommandReply extends CborSerializable
sealed trait OperationResult extends AccountCommandReply
case object Confirmed extends OperationResult
final case class Rejected(reason: String) extends OperationResult
final case class CurrentBalance(balance: BigDecimal) extends AccountCommandReply
// Event
sealed trait AccountEvent
sealed trait AccountEvent extends CborSerializable
case object AccountCreated extends AccountEvent
case class Deposited(amount: BigDecimal) extends AccountEvent
case class Withdrawn(amount: BigDecimal) extends AccountEvent
@ -54,7 +55,7 @@ object AccountExampleWithCommandHandlersInState {
type ReplyEffect = akka.persistence.typed.scaladsl.ReplyEffect[AccountEvent, Account]
// State
sealed trait Account {
sealed trait Account extends CborSerializable {
def applyCommand(cmd: AccountCommand[_]): ReplyEffect
def applyEvent(event: AccountEvent): Account
}

View file

@ -11,6 +11,7 @@ import akka.cluster.sharding.typed.scaladsl.EventSourcedEntity
import akka.persistence.typed.ExpectingReply
import akka.persistence.typed.scaladsl.Effect
import akka.persistence.typed.scaladsl.ReplyEffect
import akka.serialization.jackson.CborSerializable
/**
* Bank account example illustrating:
@ -26,7 +27,7 @@ object AccountExampleWithEventHandlersInState {
object AccountEntity {
// Command
//#reply-command
sealed trait AccountCommand[Reply <: AccountCommandReply] extends ExpectingReply[Reply]
sealed trait AccountCommand[Reply <: AccountCommandReply] extends ExpectingReply[Reply] with CborSerializable
//#reply-command
final case class CreateAccount(override val replyTo: ActorRef[OperationResult])
extends AccountCommand[OperationResult]
@ -42,7 +43,7 @@ object AccountExampleWithEventHandlersInState {
// Reply
//#reply-command
sealed trait AccountCommandReply
sealed trait AccountCommandReply extends CborSerializable
sealed trait OperationResult extends AccountCommandReply
case object Confirmed extends OperationResult
final case class Rejected(reason: String) extends OperationResult
@ -50,7 +51,7 @@ object AccountExampleWithEventHandlersInState {
final case class CurrentBalance(balance: BigDecimal) extends AccountCommandReply
// Event
sealed trait AccountEvent
sealed trait AccountEvent extends CborSerializable
case object AccountCreated extends AccountEvent
case class Deposited(amount: BigDecimal) extends AccountEvent
case class Withdrawn(amount: BigDecimal) extends AccountEvent
@ -59,7 +60,7 @@ object AccountExampleWithEventHandlersInState {
val Zero = BigDecimal(0)
// State
sealed trait Account {
sealed trait Account extends CborSerializable {
def applyEvent(event: AccountEvent): Account
}
case object EmptyAccount extends Account {

View file

@ -10,6 +10,8 @@ import akka.cluster.sharding.typed.scaladsl.EntityTypeKey
import akka.cluster.sharding.typed.scaladsl.EventSourcedEntity
import akka.persistence.typed.ExpectingReply
import akka.persistence.typed.scaladsl.Effect
import akka.serialization.jackson.CborSerializable
import akka.serialization.jackson.CborSerializable
/**
* Bank account example illustrating:
@ -23,7 +25,7 @@ object AccountExampleWithOptionState {
//#account-entity
object AccountEntity {
// Command
sealed trait AccountCommand[Reply] extends ExpectingReply[Reply]
sealed trait AccountCommand[Reply] extends ExpectingReply[Reply] with CborSerializable
final case class CreateAccount(override val replyTo: ActorRef[OperationResult])
extends AccountCommand[OperationResult]
final case class Deposit(amount: BigDecimal, override val replyTo: ActorRef[OperationResult])
@ -35,14 +37,14 @@ object AccountExampleWithOptionState {
extends AccountCommand[OperationResult]
// Reply
sealed trait AccountCommandReply
sealed trait AccountCommandReply extends CborSerializable
sealed trait OperationResult extends AccountCommandReply
case object Confirmed extends OperationResult
final case class Rejected(reason: String) extends OperationResult
final case class CurrentBalance(balance: BigDecimal) extends AccountCommandReply
// Event
sealed trait AccountEvent
sealed trait AccountEvent extends CborSerializable
case object AccountCreated extends AccountEvent
case class Deposited(amount: BigDecimal) extends AccountEvent
case class Withdrawn(amount: BigDecimal) extends AccountEvent
@ -54,7 +56,7 @@ object AccountExampleWithOptionState {
type ReplyEffect = akka.persistence.typed.scaladsl.ReplyEffect[AccountEvent, Option[Account]]
// State
sealed trait Account {
sealed trait Account extends CborSerializable {
def applyCommand(cmd: AccountCommand[_]): ReplyEffect
def applyEvent(event: AccountEvent): Account
}

View file

@ -21,6 +21,7 @@ object HelloWorldEventSourcedEntityExampleSpec {
akka.remote.artery.canonical.hostname = 127.0.0.1
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
akka.persistence.journal.inmem.test-serialization = on
""")
}
@ -50,5 +51,12 @@ class HelloWorldEventSourcedEntityExampleSpec
probe.expectMessage(Greeting("Bob", 2))
}
"verifySerialization" in {
val probe = createTestProbe[Greeting]()
serializationTestKit.verifySerialization(Greet("Alice")(probe.ref))
serializationTestKit.verifySerialization(Greeting("Alice", 1))
serializationTestKit.verifySerialization(KnownPeople(Set.empty).add("Alice").add("Bob"))
}
}
}

View file

@ -9,6 +9,7 @@ import scala.concurrent.duration._
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.serialization.jackson.CborSerializable
object HelloWorldPersistentEntityExample {
@ -48,16 +49,16 @@ object HelloWorldPersistentEntityExample {
object HelloWorld {
// Command
trait Command
trait Command extends CborSerializable
final case class Greet(whom: String)(val replyTo: ActorRef[Greeting]) extends Command
// Response
final case class Greeting(whom: String, numberOfPeople: Int)
final case class Greeting(whom: String, numberOfPeople: Int) extends CborSerializable
// Event
final case class Greeted(whom: String)
final case class Greeted(whom: String) extends CborSerializable
// State
private final case class KnownPeople(names: Set[String]) {
final case class KnownPeople(names: Set[String]) extends CborSerializable {
def add(name: String): KnownPeople = copy(names = names + name)
def numberOfPeople: Int = names.size

View file

@ -85,7 +85,9 @@ abstract class ClusterShardingCustomShardAllocationSpecConfig(val mode: String)
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb-shared"
@ -101,7 +103,9 @@ abstract class ClusterShardingCustomShardAllocationSpecConfig(val mode: String)
akka.cluster.sharding.state-store-mode = "$mode"
akka.cluster.sharding.rebalance-interval = 1 s
#akka.cluster.sharding.retry-interval = 5 s
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
object PersistentClusterShardingCustomShardAllocationSpecConfig

View file

@ -7,8 +7,8 @@ package akka.cluster.sharding
import java.io.File
import akka.cluster.sharding.ShardRegion.Passivate
import scala.concurrent.duration._
import org.apache.commons.io.FileUtils
import com.typesafe.config.ConfigFactory
import akka.actor._
@ -21,14 +21,15 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import akka.util.ccompat._
@ccompatUsedUntil213
object ClusterShardingFailureSpec {
case class Get(id: String)
case class Add(id: String, i: Int)
case class Value(id: String, n: Int)
case class Get(id: String) extends CborSerializable
case class Add(id: String, i: Int) extends CborSerializable
case class Value(id: String, n: Int) extends CborSerializable
class Entity extends Actor {
var n = 0
@ -56,7 +57,9 @@ abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiN
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.classic.log-remote-lifecycle-events = off
@ -81,7 +84,14 @@ abstract class ClusterShardingFailureSpecConfig(val mode: String) extends MultiN
dir = target/ClusterShardingFailureSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
# using Java serialization for these messages because test is sending them
# to other nodes, which isn't normal usage.
akka.actor.serialization-bindings {
"${classOf[ShardRegion.Passivate].getName}" = java-test
}
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
testTransport(on = true)
}

View file

@ -11,13 +11,14 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import akka.serialization.jackson.CborSerializable
object ClusterShardingGetStateSpec {
case object Stop
case class Ping(id: Long)
case object Pong
case object Stop extends CborSerializable
case class Ping(id: Long) extends CborSerializable
case object Pong extends CborSerializable
class ShardedActor extends Actor with ActorLogging {
log.info(self.path.toString)
@ -45,7 +46,7 @@ object ClusterShardingGetStateSpecConfig extends MultiNodeConfig {
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString("""
commonConfig(ConfigFactory.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
@ -59,6 +60,12 @@ object ClusterShardingGetStateSpecConfig extends MultiNodeConfig {
dir = target/ClusterShardingGetStateSpec/sharding-ddata
map-size = 10 MiB
}
# using Java serialization for these messages because test is sending them
# to other nodes, which isn't normal usage.
akka.actor.serialization-bindings {
"${ShardRegion.GetShardRegionState.getClass.getName}" = java-test
"${classOf[ShardRegion.CurrentShardRegionState].getName}" = java-test
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
nodeConfig(first, second)(ConfigFactory.parseString("""akka.cluster.roles=["shard"]"""))

View file

@ -10,13 +10,14 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit.{ TestDuration, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import akka.serialization.jackson.CborSerializable
object ClusterShardingGetStatsSpec {
case object Stop
case class Ping(id: Long)
case object Pong
case object Stop extends CborSerializable
case class Ping(id: Long) extends CborSerializable
case object Pong extends CborSerializable
class ShardedActor extends Actor with ActorLogging {
log.info(s"entity started {}", self.path)
@ -60,7 +61,6 @@ object ClusterShardingGetStatsSpecConfig extends MultiNodeConfig {
dir = target/ClusterShardingGetStatsSpec/sharding-ddata
map-size = 10 MiB
}
akka.actor.warn-about-java-serializer-usage=false
""").withFallback(MultiNodeClusterSpec.clusterConfig))
nodeConfig(first, second, third)(ConfigFactory.parseString("""akka.cluster.roles=["shard"]"""))

View file

@ -46,7 +46,9 @@ abstract class ClusterShardingGracefulShutdownSpecConfig(val mode: String) exten
val first = role("first")
val second = role("second")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
@ -65,7 +67,9 @@ abstract class ClusterShardingGracefulShutdownSpecConfig(val mode: String) exten
dir = target/ClusterShardingGracefulShutdownSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
object PersistentClusterShardingGracefulShutdownSpecConfig

View file

@ -7,6 +7,7 @@ package akka.cluster.sharding
import java.io.File
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
@ -20,12 +21,13 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import com.typesafe.config.ConfigFactory
import org.apache.commons.io.FileUtils
object ClusterShardingLeavingSpec {
case class Ping(id: String)
case class Ping(id: String) extends CborSerializable
class Entity extends Actor {
def receive = {
@ -33,8 +35,8 @@ object ClusterShardingLeavingSpec {
}
}
case object GetLocations
case class Locations(locations: Map[String, ActorRef])
case object GetLocations extends CborSerializable
case class Locations(locations: Map[String, ActorRef]) extends CborSerializable
class ShardLocations extends Actor {
var locations: Locations = _
@ -59,7 +61,9 @@ abstract class ClusterShardingLeavingSpecConfig(val mode: String) extends MultiN
val third = role("third")
val fourth = role("fourth")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.classic.log-remote-lifecycle-events = off
@ -79,7 +83,9 @@ abstract class ClusterShardingLeavingSpecConfig(val mode: String) extends MultiN
dir = target/ClusterShardingLeavingSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
object PersistentClusterShardingLeavingSpecConfig extends ClusterShardingLeavingSpecConfig("persistence")

View file

@ -41,7 +41,9 @@ abstract class ClusterShardingMinMembersSpecConfig(val mode: String) extends Mul
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
@ -62,7 +64,9 @@ abstract class ClusterShardingMinMembersSpecConfig(val mode: String) extends Mul
map-size = 10 MiB
}
akka.cluster.min-nr-of-members = 3
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
}
object PersistentClusterShardingMinMembersSpecConfig extends ClusterShardingMinMembersSpecConfig("persistence")

View file

@ -57,7 +57,9 @@ abstract class ClusterShardingRememberEntitiesNewExtractorSpecConfig(val mode: S
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.actor.provider = "cluster"
akka.cluster.auto-down-unreachable-after = 0s
akka.remote.classic.log-remote-lifecycle-events = off
@ -76,7 +78,9 @@ abstract class ClusterShardingRememberEntitiesNewExtractorSpecConfig(val mode: S
dir = target/ShardingRememberEntitiesNewExtractorSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
val roleConfig = ConfigFactory.parseString("""
akka.cluster.roles = [sharding]

View file

@ -51,7 +51,9 @@ abstract class ClusterShardingRememberEntitiesSpecConfig(val mode: String) exten
val second = role("second")
val third = role("third")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.cluster.auto-down-unreachable-after = 0s
@ -71,7 +73,9 @@ abstract class ClusterShardingRememberEntitiesSpecConfig(val mode: String) exten
dir = target/ShardingRememberEntitiesSpec/sharding-ddata
map-size = 10 MiB
}
""").withFallback(MultiNodeClusterSpec.clusterConfig))
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
nodeConfig(third)(ConfigFactory.parseString(s"""
akka.cluster.sharding.distributed-data.durable.lmdb {

View file

@ -9,9 +9,9 @@ import akka.cluster.sharding.ShardCoordinator.Internal.{ HandOff, ShardStopped }
import akka.cluster.sharding.ShardRegion.Passivate
import akka.cluster.sharding.ShardRegion.GetCurrentRegions
import akka.cluster.sharding.ShardRegion.CurrentRegions
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor._
import akka.cluster.{ Cluster, MultiNodeClusterSpec }
@ -129,7 +129,9 @@ abstract class ClusterShardingSpecConfig(val mode: String, val entityRecoveryStr
val fifth = role("fifth")
val sixth = role("sixth")
commonConfig(ConfigFactory.parseString(s"""
commonConfig(
ConfigFactory
.parseString(s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.remote.log-remote-lifecycle-events = off
@ -165,7 +167,21 @@ abstract class ClusterShardingSpecConfig(val mode: String, val entityRecoveryStr
}
}
akka.testconductor.barrier-timeout = 70s
""").withFallback(MultiNodeClusterSpec.clusterConfig))
# using Java serialization for the messages here because would be to much (unrelated)
# to show full Jackson serialization in docs (requires annotations because of envelope and such)
akka.actor.serialization-bindings {
"${ClusterShardingSpec.Increment.getClass.getName}" = java-test
"${ClusterShardingSpec.Decrement.getClass.getName}" = java-test
"${classOf[ClusterShardingSpec.Get].getName}" = java-test
"${classOf[ClusterShardingSpec.EntityEnvelope].getName}" = java-test
"${ClusterShardingSpec.Stop.getClass.getName}" = java-test
"${classOf[ClusterShardingSpec.CounterChanged].getName}" = java-test
}
""")
.withFallback(SharedLeveldbJournal.configToEnableJavaSerializationForTest)
.withFallback(MultiNodeClusterSpec.clusterConfig))
nodeConfig(sixth) {
ConfigFactory.parseString("""akka.cluster.roles = ["frontend"]""")
}

View file

@ -5,6 +5,7 @@
package akka.cluster.sharding
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Address
@ -16,13 +17,14 @@ import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.util.ccompat._
@ccompatUsedUntil213
object MultiDcClusterShardingSpec {
sealed trait EntityMsg {
sealed trait EntityMsg extends CborSerializable {
def id: String
}
final case class Ping(id: String) extends EntityMsg

View file

@ -1,7 +1,6 @@
akka {
actor {
serialize-messages = on
warn-about-java-serializer-usage = off
serialize-messages = off
}
}

View file

@ -8,6 +8,7 @@ import java.util.concurrent.atomic.AtomicReference
import scala.concurrent.Future
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
@ -22,20 +23,21 @@ import akka.event.Logging
import akka.coordination.lease.LeaseSettings
import akka.coordination.lease.scaladsl.Lease
import akka.pattern.ask
import akka.testkit.JavaSerializable
import akka.util.Timeout
object TestLeaseActor {
def props(): Props =
Props(new TestLeaseActor)
sealed trait LeaseRequest
sealed trait LeaseRequest extends JavaSerializable
final case class Acquire(owner: String) extends LeaseRequest
final case class Release(owner: String) extends LeaseRequest
final case class Create(leaseName: String, ownerName: String)
final case class Create(leaseName: String, ownerName: String) extends JavaSerializable
final case object GetRequests
final case class LeaseRequests(requests: List[LeaseRequest])
final case class ActionRequest(request: LeaseRequest, result: Any) // boolean of Failure
final case object GetRequests extends JavaSerializable
final case class LeaseRequests(requests: List[LeaseRequest]) extends JavaSerializable
final case class ActionRequest(request: LeaseRequest, result: Any) extends JavaSerializable // boolean of Failure
}
class TestLeaseActor extends Actor with ActorLogging {

View file

@ -57,7 +57,7 @@ object ClusterClientSpec extends MultiNodeConfig {
testTransport(on = true)
case class Reply(msg: Any, node: Address)
case class Reply(msg: Any, node: Address) extends JavaSerializable
class TestService(testActor: ActorRef) extends Actor {
def receive = {

View file

@ -43,7 +43,7 @@ object ClusterSingletonManagerLeaseSpec extends MultiNodeConfig {
nodeConfig(first, second, third)(ConfigFactory.parseString("akka.cluster.roles = [worker]"))
object ImportantSingleton {
case class Response(msg: Any, address: Address)
case class Response(msg: Any, address: Address) extends JavaSerializable
def props(): Props = Props(new ImportantSingleton())
}

View file

@ -6,6 +6,7 @@ package akka.cluster.singleton
import language.postfixOps
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.Actor
import akka.actor.ActorLogging
@ -23,6 +24,7 @@ import akka.testkit.TestEvent._
import akka.actor.Identify
import akka.actor.ActorIdentity
import akka.actor.ActorSelection
import akka.serialization.jackson.CborSerializable
object ClusterSingletonManagerSpec extends MultiNodeConfig {
val controller = role("controller")
@ -45,15 +47,15 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
//#singleton-message-classes
object PointToPointChannel {
case object UnregistrationOk
case object UnregistrationOk extends CborSerializable
//#singleton-message-classes
case object RegisterConsumer
case object UnregisterConsumer
case object RegistrationOk
case object UnexpectedRegistration
case object UnexpectedUnregistration
case object Reset
case object ResetOk
case object RegisterConsumer extends CborSerializable
case object UnregisterConsumer extends CborSerializable
case object RegistrationOk extends CborSerializable
case object UnexpectedRegistration extends CborSerializable
case object UnexpectedUnregistration extends CborSerializable
case object Reset extends CborSerializable
case object ResetOk extends CborSerializable
//#singleton-message-classes
}
//#singleton-message-classes
@ -105,10 +107,10 @@ object ClusterSingletonManagerSpec extends MultiNodeConfig {
//#singleton-message-classes
object Consumer {
case object End
case object GetCurrent
case object Ping
case object Pong
case object End extends CborSerializable
case object GetCurrent extends CborSerializable
case object Ping extends CborSerializable
case object Pong extends CborSerializable
}
//#singleton-message-classes

View file

@ -7,13 +7,12 @@ package akka.cluster.singleton
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.{ Actor, ActorLogging, Address, PoisonPill, Props }
import akka.cluster.Cluster
import akka.testkit.ImplicitSender
import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.cluster.ClusterSettings
import akka.serialization.jackson.CborSerializable
object MultiDcSingletonManagerSpec extends MultiNodeConfig {
val controller = role("controller")
@ -60,8 +59,8 @@ class MultiDcSingleton extends Actor with ActorLogging {
}
}
object MultiDcSingleton {
case object Ping
case class Pong(fromDc: String, fromAddress: Address, roles: Set[String])
case object Ping extends CborSerializable
case class Pong(fromDc: String, fromAddress: Address, roles: Set[String]) extends CborSerializable
}
abstract class MultiDcSingletonManagerSpec

View file

@ -1,7 +1,6 @@
akka {
actor {
serialize-messages = on
warn-about-java-serializer-usage = off
serialize-messages = off
}
}

View file

@ -6,16 +6,27 @@ package akka.cluster.singleton
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.{ Actor, ActorLogging, ActorRef, ExtendedActorSystem, PoisonPill, Props }
import akka.cluster.TestLease.{ AcquireReq, ReleaseReq }
import akka.cluster.{ Cluster, MemberStatus, TestLease, TestLeaseExt }
import akka.testkit.{ AkkaSpec, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.Promise
import scala.concurrent.duration._
import scala.util.Success
import akka.actor.Actor
import akka.actor.ActorLogging
import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem
import akka.actor.PoisonPill
import akka.actor.Props
import akka.cluster.Cluster
import akka.cluster.MemberStatus
import akka.cluster.TestLease
import akka.cluster.TestLease.AcquireReq
import akka.cluster.TestLease.ReleaseReq
import akka.cluster.TestLeaseExt
import akka.testkit.AkkaSpec
import akka.testkit.TestException
import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory
class ImportantSingleton(lifeCycleProbe: ActorRef) extends Actor with ActorLogging {
override def preStart(): Unit = {
@ -129,7 +140,7 @@ class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
testLeaseExt.getTestLease(leaseNameFor(settings))
} // allow singleton manager to create the lease
probe.expectNoMessage(shortDuration)
testLease.initialPromise.failure(new RuntimeException("no lease for you"))
testLease.initialPromise.failure(TestException("no lease for you"))
probe.expectNoMessage(shortDuration)
}
@ -146,7 +157,7 @@ class ClusterSingletonLeaseSpec extends AkkaSpec(ConfigFactory.parseString("""
singletonProbe.expectNoMessage(shortDuration)
val nextResponse = Promise[Boolean]
testLease.setNextAcquireResult(nextResponse.future)
testLease.initialPromise.failure(new RuntimeException("no lease for you"))
testLease.initialPromise.failure(TestException("no lease for you"))
testLease.probe.expectMsg(AcquireReq(leaseOwner))
singletonProbe.expectNoMessage(shortDuration)
nextResponse.complete(Success(true))

View file

@ -26,7 +26,8 @@ object ClusterSingletonRestart2Spec {
}
}
class ClusterSingletonRestart2Spec extends AkkaSpec("""
class ClusterSingletonRestart2Spec
extends AkkaSpec("""
akka.loglevel = INFO
akka.cluster.roles = [singleton]
akka.actor.provider = akka.cluster.ClusterActorRefProvider
@ -42,6 +43,10 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
port = 0
}
}
akka.actor.serialization-bindings {
# there is no serializer for UniqueAddress, not intended to be sent as a standalone message
"akka.cluster.UniqueAddress" = jackson-cbor
}
""") {
val sys1 = ActorSystem(system.name, system.settings.config)

View file

@ -6,10 +6,11 @@ package akka.cluster.typed
import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors
import akka.serialization.jackson.CborSerializable
object MultiDcClusterActors {
case class Pong(dc: String)
sealed trait PingProtocol
case class Pong(dc: String) extends CborSerializable
sealed trait PingProtocol extends CborSerializable
case class Ping(ref: ActorRef[Pong]) extends PingProtocol
case object NoMore extends PingProtocol

View file

@ -29,7 +29,6 @@ public class ClusterApiTest extends JUnitSuite {
+ "akka.coordinated-shutdown.run-by-actor-system-terminate = off \n"
+ "akka.actor { \n"
+ " serialize-messages = off \n"
+ " allow-java-serialization = off \n"
+ "}");
ActorSystem<?> system1 =

View file

@ -27,7 +27,6 @@ object ClusterApiSpec {
akka.coordinated-shutdown.run-by-actor-system-terminate = off
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
# generous timeout for cluster forming probes
akka.actor.testkit.typed.default-timeout = 10s

View file

@ -26,7 +26,6 @@ object ClusterSingletonApiSpec {
akka.actor {
provider = cluster
serialize-messages = off
allow-java-serialization = off
serializers {
test = "akka.cluster.typed.ClusterSingletonApiSpec$$PingSerializer"

View file

@ -24,7 +24,6 @@ object ClusterSingletonPersistenceSpec {
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"

View file

@ -48,7 +48,6 @@ object RemoteContextAskSpec {
loglevel = debug
actor {
provider = cluster
warn-about-java-serializer-usage = off
serialize-creators = off
serializers {
test = "akka.cluster.typed.RemoteContextAskSpecSerializer"

View file

@ -20,7 +20,6 @@ object RemoteDeployNotAllowedSpec {
loglevel = warning
actor {
provider = cluster
warn-about-java-serializer-usage = off
serialize-creators = off
}
remote.classic.netty.tcp.port = 0

View file

@ -13,8 +13,8 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.{ ExtendedActorSystem, ActorSystem => UntypedActorSystem }
import akka.serialization.SerializerWithStringManifest
import com.typesafe.config.ConfigFactory
import scala.concurrent.Promise
import akka.actor.typed.scaladsl.adapter._
class PingSerializer(system: ExtendedActorSystem) extends SerializerWithStringManifest {
@ -37,7 +37,6 @@ object RemoteMessageSpec {
loglevel = debug
actor {
provider = cluster
warn-about-java-serializer-usage = off
serialize-creators = off
serializers {
test = "akka.cluster.typed.PingSerializer"

View file

@ -31,7 +31,6 @@ object ClusterReceptionistSpec {
akka.actor {
provider = cluster
serialize-messages = off
allow-java-serialization = true
serializers {
test = "akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$PingSerializer"
}

View file

@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit.TestEvent._
import akka.testkit._
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures
object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
@ -27,7 +28,14 @@ object ClusterDeathWatchMultiJvmSpec extends MultiNodeConfig {
val fourth = role("fourth")
val fifth = role("fifth")
commonConfig(debugConfig(on = false).withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
commonConfig(
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
"""))
.withFallback(MultiNodeClusterSpec.clusterConfigWithFailureDetectorPuppet))
deployOn(fourth, """/hello.remote = "@first@" """)

View file

@ -7,6 +7,7 @@ package akka.cluster
import java.util.UUID
import language.implicitConversions
import org.scalatest.{ Canceled, Outcome, Suite }
import org.scalatest.exceptions.TestCanceledException
import com.typesafe.config.Config
@ -18,7 +19,6 @@ import akka.testkit.TestEvent._
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
import akka.event.Logging.ErrorLevel
import akka.util.ccompat._
import scala.concurrent.duration._
import scala.collection.immutable
import java.util.concurrent.ConcurrentHashMap
@ -26,9 +26,10 @@ import java.util.concurrent.ConcurrentHashMap
import akka.remote.DefaultFailureDetectorRegistry
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
import akka.util.ccompat._
import scala.concurrent.Await
import akka.serialization.jackson.CborSerializable
@ccompatUsedUntil213
object MultiNodeClusterSpec {
@ -77,9 +78,9 @@ object MultiNodeClusterSpec {
// sometimes we need to coordinate test shutdown with messages instead of barriers
object EndActor {
case object SendEnd
case object End
case object EndAck
case object SendEnd extends CborSerializable
case object End extends CborSerializable
case object EndAck extends CborSerializable
}
class EndActor(testActor: ActorRef, target: Option[Address]) extends Actor {

View file

@ -26,13 +26,18 @@ object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
val seed2 = role("seed2")
commonConfig(
debugConfig(on = false).withFallback(ConfigFactory.parseString("""
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster.auto-down-unreachable-after = 2s
akka.cluster.retry-unsuccessful-join-after = 3s
akka.cluster.allow-weakly-up-members = off
akka.remote.retry-gate-closed-for = 45s
akka.remote.log-remote-lifecycle-events = INFO
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
}

View file

@ -27,10 +27,15 @@ object RestartNode3MultiJvmSpec extends MultiNodeConfig {
val third = role("third")
commonConfig(
debugConfig(on = false).withFallback(ConfigFactory.parseString("""
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster.auto-down-unreachable-after = off
akka.cluster.allow-weakly-up-members = off
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
testTransport(on = true)
}

View file

@ -32,11 +32,16 @@ object RestartNodeMultiJvmSpec extends MultiNodeConfig {
val third = role("third")
commonConfig(
debugConfig(on = false).withFallback(ConfigFactory.parseString("""
debugConfig(on = false)
.withFallback(ConfigFactory.parseString("""
akka.cluster.auto-down-unreachable-after = 5s
akka.cluster.allow-weakly-up-members = off
#akka.remote.use-passive-connections = off
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
"""))
.withFallback(MultiNodeClusterSpec.clusterConfig))
/**
* This was used together with sleep in EndpointReader before deliverAndAck

View file

@ -18,6 +18,7 @@ import akka.actor.Props
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable
import akka.stream.ActorMaterializer
import akka.stream.RemoteStreamRefActorTerminatedException
import akka.stream.SinkRef
@ -43,8 +44,9 @@ object StreamRefSpec extends MultiNodeConfig {
testTransport(on = true)
case class RequestLogs(streamId: Int)
case class LogsOffer(streamId: Int, sourceRef: SourceRef[String])
case class RequestLogs(streamId: Int) extends CborSerializable
// Using Java serialization until issue #27304 is fixed
case class LogsOffer(streamId: Int, sourceRef: SourceRef[String]) extends JavaSerializable
object DataSource {
def props(streamLifecycleProbe: ActorRef): Props =
@ -84,8 +86,9 @@ object StreamRefSpec extends MultiNodeConfig {
}
case class PrepareUpload(id: String)
case class MeasurementsSinkReady(id: String, sinkRef: SinkRef[String])
case class PrepareUpload(id: String) extends CborSerializable
// Using Java serialization until issue #27304 is fixed
case class MeasurementsSinkReady(id: String, sinkRef: SinkRef[String]) extends JavaSerializable
object DataReceiver {
def props(streamLifecycleProbe: ActorRef): Props =

View file

@ -165,6 +165,9 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
}
}
}
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
"""))
class Settings(conf: Config) {

View file

@ -13,12 +13,13 @@ import akka.actor.Address
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.Terminated
import akka.remote.artery.QuarantinedEvent
import akka.remote.RARP
import akka.remote.artery.QuarantinedEvent
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import com.github.ghik.silencer.silent
import com.typesafe.config.ConfigFactory
@ -50,8 +51,8 @@ object SurviveNetworkInstabilityMultiJvmSpec extends MultiNodeConfig {
}
}
case class Targets(refs: Set[ActorRef])
case object TargetsRegistered
case class Targets(refs: Set[ActorRef]) extends CborSerializable
case object TargetsRegistered extends CborSerializable
class Watcher extends Actor {
var targets = Set.empty[ActorRef]

View file

@ -16,8 +16,9 @@ import scala.concurrent.Await
object ClusterConsistentHashingGroupMultiJvmSpec extends MultiNodeConfig {
case object Get
final case class Collected(messages: Set[Any])
// using Java serialization because of `Any` in `Collected` (don't want to spend time on rewriting test)
case object Get extends JavaSerializable
final case class Collected(messages: Set[Any]) extends JavaSerializable
class Destination extends Actor {
var receivedMessages = Set.empty[Any]

View file

@ -50,11 +50,10 @@ object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString(s"""
akka.actor {
allow-java-serialization = off
serialize-creators = off
serialize-messages = off
serialization-bindings {
"akka.cluster.routing.ClusterRoundRobinMultiJvmSpec$$Reply" = test-message-serializer
"akka.cluster.routing.ClusterRoundRobinMultiJvmSpec$$Reply" = java-test
}
deployment {

View file

@ -36,7 +36,7 @@ object UseRoleIgnoredMultiJvmSpec extends MultiNodeConfig {
}
}
final case class Reply(routeeType: RouteeType, ref: ActorRef)
final case class Reply(routeeType: RouteeType, ref: ActorRef) extends JavaSerializable
sealed trait RouteeType extends Serializable
object PoolRoutee extends RouteeType

View file

@ -1,7 +1,6 @@
akka {
actor {
serialize-messages = off
warn-about-java-serializer-usage = off
}
}

View file

@ -13,6 +13,7 @@ import akka.cluster.ddata.Replicator._
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.serialization.jackson.CborSerializable
import akka.testkit._
import com.typesafe.config.ConfigFactory
@ -33,7 +34,8 @@ object ReplicatorDeltaSpec extends MultiNodeConfig {
case class Highest(n: Int, delta: Option[Highest] = None)
extends DeltaReplicatedData
with RequiresCausalDeliveryOfDeltas
with ReplicatedDelta {
with ReplicatedDelta
with CborSerializable {
type T = Highest
type D = Highest

View file

@ -28,9 +28,7 @@ object ReplicatorMapDeltaSpec extends MultiNodeConfig {
akka.log-dead-letters-during-shutdown = off
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
#akka.remote.artery.enabled = on
"""))
testTransport(on = true)

View file

@ -25,7 +25,6 @@ object ReplicatorORSetDeltaSpec extends MultiNodeConfig {
akka.log-dead-letters-during-shutdown = off
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
"""))

View file

@ -1,5 +1,4 @@
akka.actor {
serialize-messages = on
warn-about-java-serializer-usage = off
serialize-messages = off
}

View file

@ -37,7 +37,6 @@ class ReplicatedDataSerializerSpec
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
""")))
with WordSpecLike

View file

@ -41,7 +41,6 @@ class ReplicatorMessageSerializerSpec
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
allow-java-serialization = off
}
""")))
with WordSpecLike

View file

@ -195,6 +195,9 @@ Java
See @ref:[reference configuration](general/configuration.md#config-akka-cluster) for further descriptions of the settings.
When using a pool of remote deployed routees you must ensure that all parameters of the `Props` can
be @ref:[serialized](serialization.md).
### Router Example with Pool of Remote Deployed Routees
Let's take a look at how to use a cluster aware router on single master node that creates
@ -253,4 +256,4 @@ akka.actor.deployment {
```
The easiest way to run **Router Example with Pool of Routees** example yourself is to try the
@scala[@extref[Akka Cluster Sample with Scala](samples:akka-samples-cluster-scala)]@java[@extref[Akka Cluster Sample with Java](samples:akka-samples-cluster-java)].
It contains instructions on how to run the **Router Example with Pool of Routees** sample.
It contains instructions on how to run the **Router Example with Pool of Routees** sample.

View file

@ -6,6 +6,10 @@ The core of Akka Cluster is the cluster membership, to keep track of what nodes
their health. There are several @ref:[Higher level Cluster tools](cluster-usage.md#higher-level-cluster-tools) that are built
on top of the cluster membership.
You need to enable @ref:[serialization](serialization.md) for your actor messages.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
## Dependency
To use Akka Cluster, you must add the following dependency in your project:

View file

@ -1091,7 +1091,7 @@ it must add
@@snip [PersistenceSerializerDocSpec.scala](/akka-docs/src/test/scala/docs/persistence/PersistenceSerializerDocSpec.scala) { #custom-serializer-config }
to the application configuration. If not specified, a default serializer is used.
to the application configuration. If not specified, an exception will be throw when trying to persist events or snapshots.
For more advanced schema evolution techniques refer to the @ref:[Persistence - Schema Evolution](persistence-schema-evolution.md) documentation.

View file

@ -193,6 +193,64 @@ Classic remoting is deprecated but can be used in `2.6.` Explicitly disable Arte
specific to classic remoting needs to be moved to `akka.remote.classic`. To see which configuration options
are specific to classic search for them in: [`akka-remote/reference.conf`](/akka-remote/src/main/resources/reference.conf)
## Java Serialization
Java serialization is known to be slow and [prone to attacks](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995)
of various kinds - it never was designed for high throughput messaging after all.
One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck.
From Akka 2.6.0 the Akka serialization with Java serialization is disabled by default and Akka
itself doesn't use Java serialization for any of its internal messages.
For compatibility with older systems that rely on Java serialization it can be enabled with the following configuration:
```ruby
akka.actor.allow-java-serialization = on
```
Akka will still log warning when Java serialization is used and to silent that you may add:
```ruby
akka.actor.warn-about-java-serializer-usage = off
```
### Rolling update
You can replace Java serialization, if you use that, with for example the new
@ref:[Serialization with Jackson](../serialization-jackson.md) and still be able to perform a rolling updates
without bringing down the entire cluster.
The procedure for changing from Java serialization to Jackson would look like:
1. Rolling update from 2.5.24 (or later) to 2.6.0
* Use config `allow-java-serialization=on`.
* Roll out the change.
* Java serialization will be used as before.
* This step is optional and you could combine it with next step if you like, but could be good to
make one change at a time.
1. Rolling update to support deserialization but not enable serialization
* Change message classes by adding the marker interface and possibly needed annotations as
described in @ref:[Serialization with Jackson](../serialization-jackson.md).
* Test the system with the new serialization in a new test cluster (no rolling update).
* Remove the binding for the marker interface, so that Jackson is not used for serialization yet.
* Roll out the change.
* Java serialization is still used, but this version is prepared for next roll out.
1. Rolling update to enable serialization with Jackson.
* Add the binding to the marker interface to the Jackson serializer.
* Roll out the change.
* Old nodes will still send messages with Java serialization, and that can still be deserialized by new nodes.
* New nodes will send messages with Jackson serialization, and old node can deserialize those because they were
prepared in previous roll out.
1. Rolling update to disable Java serialization
* Remove `allow-java-serialization` config, to use the default `allow-java-serialization=off`.
* Remove `.warn-about-java-serializer-usage` config if you had changed that, to use the default `.warn-about-java-serializer-usage=on`.
* Roll out the change.
### Java serialization in consistent hashing
When using a consistent hashing router keys that were not bytes or a String are serialized.
You might have to add a serializer for you hash keys, unless one of the default serializer are not
handling that type and it was previously "accidentally" serialized with Java serialization.
## Configuration and behavior changes

View file

@ -86,10 +86,8 @@ underlying module that allows for Cluster, it is still useful to understand deta
@@@ note
This page describes the remoting subsystem, codenamed *Artery* that will eventually replace the
@ref:[classic remoting implementation](remoting.md). Artery is ready to use in production, but the
module is still marked @ref:[may change](common/may-change.md) because some configuration will be
changed when the API becomes stable in Akka 2.6.0.
This page describes the remoting subsystem, codenamed *Artery* that has replaced the
@ref:[classic remoting implementation](remoting.md).
@@@
@ -101,6 +99,10 @@ sent messages to, watched, etc.
Every `ActorRef` contains hostname and port information and can be passed around even on the network. This means
that on a network every `ActorRef` is a unique identifier of an actor on that network.
You need to enable @ref:[serialization](serialization.md) for your actor messages.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
Remoting is not a server-client technology. All systems using remoting can contact any other system on the network
if they possess an `ActorRef` pointing to those system. This means that every system that is remoting enabled
acts as a "server" to which arbitrary systems on the same network can connect to.
@ -254,8 +256,9 @@ Best practice is that Akka remoting nodes should only be accessible from the adj
enabled with mutual authentication there is still a risk that an attacker can gain access to a valid certificate by
compromising any node with certificates issued by the same internal PKI tree.
It is also security best-practice to @ref[disable the Java serializer](serialization.md#disable-java-serializer) because of
its multiple [known attack surfaces](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995).
By default, @ref[Java serialization](serialization.md#java-serialization) is disabled in Akka.
That is also security best-practice because of its multiple
[known attack surfaces](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995).
<a id="remote-tls"></a>
### Configuring SSL/TLS for Akka Remoting
@ -388,9 +391,10 @@ as a marker trait to user-defined messages.
Untrusted mode does not give full protection against attacks by itself.
It makes it slightly harder to perform malicious or unintended actions but
it should be complemented with @ref:[disabled Java serializer](#disabling-the-java-serializer)
it should be noted that @ref:[Java serialization](serialization.md#java-serialization)
should still not be enabled.
Additional protection can be achieved when running in an untrusted network by
network security (e.g. firewalls).
network security (e.g. firewalls) and/or enabling @ref:[TLS with mutual authentication](#remote-tls).
@@@
@ -546,10 +550,9 @@ This is how the curve looks like for `acceptable-heartbeat-pause` configured to
## Serialization
When using remoting for actors you must ensure that the `props` and `messages` used for
those actors are serializable. Failing to do so will cause the system to behave in an unintended way.
For more information please see @ref:[Serialization](serialization.md).
You need to enable @ref:[serialization](serialization.md) for your actor messages.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
<a id="remote-bytebuffer-serialization"></a>
### ByteBuffer based serialization
@ -588,11 +591,6 @@ Scala
Java
: @@snip [ByteBufferSerializerDocTest.java](/akka-docs/src/test/java/jdocs/actor/ByteBufferSerializerDocTest.java) { #bytebufserializer-with-manifest }
<a id="disable-java-serializer"></a>
### Disabling the Java Serializer
It is highly recommended that you @ref[disable Java serialization](serialization.md#disable-java-serializer).
## Routers with Remote Destinations
It is absolutely feasible to combine remoting with @ref:[Routing](routing.md).
@ -604,6 +602,9 @@ A pool of remote deployed routees can be configured as:
This configuration setting will clone the actor defined in the `Props` of the `remotePool` 10
times and deploy it evenly distributed across the two given target nodes.
When using a pool of remote deployed routees you must ensure that all parameters of the `Props` can
be @ref:[serialized](serialization.md).
A group of remote actors can be configured as:
@@snip [RouterDocSpec.scala](/akka-docs/src/test/scala/docs/routing/RouterDocSpec.scala) { #config-remote-round-robin-group-artery }

View file

@ -100,6 +100,10 @@ network and/or Akka configuration will have to be changed as described in
@@@
You need to enable @ref:[serialization](serialization.md) for your actor messages.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
## Types of Remote Interaction
Akka has two ways of using remoting:
@ -203,6 +207,9 @@ Java
The actor class `SampleActor` has to be available to the runtimes using it, i.e. the classloader of the
actor systems has to have a JAR containing the class.
When using remote deployment of actors you must ensure that all parameters of the `Props` can
be @ref:[serialized](serialization.md).
@@@ note
In order to ensure serializability of `Props` when passing constructor
@ -359,15 +366,9 @@ This is how the curve looks like for `acceptable-heartbeat-pause` configured to
## Serialization
When using remoting for actors you must ensure that the `props` and `messages` used for
those actors are serializable. Failing to do so will cause the system to behave in an unintended way.
For more information please see @ref:[Serialization](serialization.md).
<a id="disable-java-serializer"></a>
### Disabling the Java Serializer
It is highly recommended that you @ref[disable Java serialization](serialization.md#disable-java-serializer).
You need to enable @ref:[serialization](serialization.md) for your actor messages.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
## Routers with Remote Destinations
@ -380,6 +381,9 @@ A pool of remote deployed routees can be configured as:
This configuration setting will clone the actor defined in the `Props` of the `remotePool` 10
times and deploy it evenly distributed across the two given target nodes.
When using a pool of remote deployed routees you must ensure that all parameters of the `Props` can
be @ref:[serialized](serialization.md).
A group of remote actors can be configured as:
@@snip [RouterDocSpec.scala](/akka-docs/src/test/scala/docs/routing/RouterDocSpec.scala) { #config-remote-round-robin-group }
@ -454,8 +458,9 @@ Best practice is that Akka remoting nodes should only be accessible from the adj
enabled with mutual authentication there is still a risk that an attacker can gain access to a valid certificate by
compromising any node with certificates issued by the same internal PKI tree.
It is also security best-practice to [disable the Java serializer](#disable-java-serializer) because of
its multiple [known attack surfaces](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995).
By default, @ref[Java serialization](serialization.md#java-serialization) is disabled in Akka.
That is also security best-practice because of its multiple
[known attack surfaces](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995).
<a id="remote-tls"></a>
### Configuring SSL/TLS for Akka Remoting
@ -564,9 +569,10 @@ as a marker trait to user-defined messages.
Untrusted mode does not give full protection against attacks by itself.
It makes it slightly harder to perform malicious or unintended actions but
it should be complemented with [disabled Java serializer](#disable-java-serializer).
it should be noted that @ref:[Java serialization](serialization.md#java-serialization)
should still not be enabled.
Additional protection can be achieved when running in an untrusted network by
network security (e.g. firewalls) and/or enabling [TLS with mutual authentication](#remote-tls).
network security (e.g. firewalls) and/or enabling @ref:[TLS with mutual authentication](#remote-tls).
@@@

View file

@ -20,7 +20,7 @@ Java deserialization is [known to be vulnerable](https://community.hpe.com/t5/Se
Akka Remoting uses Java serialiser as default configuration which makes it vulnerable in its default form. The documentation of how to disable Java serializer was not complete. The documentation of how to enable mutual authentication was missing (only described in reference.conf).
To protect against such attacks the system should be updated to Akka *2.4.17* or later and be configured with
@ref:[disabled Java serializer](../remoting.md#disable-java-serializer). Additional protection can be achieved when running in an
[disabled Java serializer](https://doc.akka.io/docs/akka/2.5/remoting.html#disable-java-serializer). Additional protection can be achieved when running in an
untrusted network by enabling @ref:[TLS with mutual authentication](../remoting.md#remote-tls).
Please subscribe to the [akka-security](https://groups.google.com/forum/#!forum/akka-security) mailing list to be notified promptly about future security issues.

View file

@ -18,7 +18,7 @@ to ensure that a fix can be provided without delay.
## Security Related Documentation
* @ref:[Disabling the Java Serializer](../remoting-artery.md#disable-java-serializer)
* @ref:[Java Serialization](../serialization.md#java-serialization)
* @ref:[Remote deployment whitelist](../remoting.md#remote-deployment-whitelist)
* @ref:[Remote Security](../remoting-artery.md#remote-security)

View file

@ -12,15 +12,24 @@ To use Serialization, you must add the following dependency in your project:
## Introduction
The messages that Akka actors send to each other are JVM objects (e.g. instances of Scala case classes). Message passing between actors that live on the same JVM is straightforward. It is done via reference passing. However, messages that have to escape the JVM to reach an actor running on a different host have to undergo some form of serialization (i.e. the objects have to be converted to and from byte arrays).
The messages that Akka actors send to each other are JVM objects @scala[(e.g. instances of Scala case classes)]. Message passing between actors that live on the same JVM is straightforward. It is done via reference passing. However, messages that have to escape the JVM to reach an actor running on a different host have to undergo some form of serialization (i.e. the objects have to be converted to and from byte arrays).
Akka itself uses Protocol Buffers to serialize internal messages (i.e. cluster gossip messages). However, the serialization mechanism in Akka allows you to write custom serializers and to define which serializer to use for what.
The serialization mechanism in Akka allows you to write custom serializers and to define which serializer to use for what.
@ref:[Serialization with Jackson](serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
[Google Protocol Buffers](https://developers.google.com/protocol-buffers/) is good if you want
more control over the schema evolution of your messages, but it requires more work to develop and
maintain the mapping between serialized representation and domain representation.
Akka itself uses Protocol Buffers to serialize internal messages (for example cluster gossip messages).
## Usage
### Configuration
For Akka to know which `Serializer` to use for what, you need edit your [Configuration](),
For Akka to know which `Serializer` to use for what, you need edit your configuration,
in the "akka.actor.serializers"-section you bind names to implementations of the `akka.serialization.Serializer`
you wish to use, like this:
@ -35,8 +44,8 @@ You only need to specify the name of an interface or abstract base class of the
messages. In case of ambiguity, i.e. the message implements several of the
configured classes, the most specific configured class will be used, i.e. the
one of which all other candidates are superclasses. If this condition cannot be
met, because e.g. `java.io.Serializable` and `MyOwnSerializable` both apply
and neither is a subtype of the other, a warning will be issued.
met, because e.g. two marker interfaces that have been configured for serialization
both apply and neither is a subtype of the other, a warning will be issued.
@@@ note
@ -47,13 +56,10 @@ you would need to reference it as `Wrapper$Message` instead of `Wrapper.Message`
@@@
Akka provides serializers for `java.io.Serializable` and [protobuf](http://code.google.com/p/protobuf/)
Akka provides serializers for several primitive types and [protobuf](http://code.google.com/p/protobuf/)
`com.google.protobuf.GeneratedMessage` by default (the latter only if
depending on the akka-remote module), so normally you don't need to add
configuration for that; since `com.google.protobuf.GeneratedMessage`
implements `java.io.Serializable`, protobuf messages will always be
serialized using the protobuf protocol unless specifically overridden. In order
to disable a default serializer, see @ref:[Disabling the Java Serializer](remoting-artery.md#disable-java-serializer)
configuration for that if you send raw protobuf messages as actor messages.
### Verification
@ -89,7 +95,13 @@ Scala
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #programmatic }
For more information, have a look at the `ScalaDoc` for `akka.serialization._`
The manifest is a type hint so that the same serializer can be used for different classes.
Note that when deserializing from bytes the manifest and the identifier of the serializer are needed.
It is important to use the serializer identifier in this way to support rolling updates, where the
`serialization-bindings` for a class may have changed from one serializer to another. Therefore the three parts
consisting of the bytes, the serializer id, and the manifest should always be transferred or stored together so that
they can be deserialized with different `serialization-bindings` configuration.
## Customization
@ -117,7 +129,7 @@ classes. The manifest parameter in @scala[`fromBinary`]@java[`fromBinaryJava`] i
was serialized. In `fromBinary` you can match on the class and deserialize the
bytes to different objects.
Then you only need to fill in the blanks, bind it to a name in your [Configuration]() and then
Then you only need to fill in the blanks, bind it to a name in your configuration and then
list which classes that should be serialized using it.
<a id="string-manifest-serializer"></a>
@ -145,17 +157,17 @@ Scala
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #my-own-serializer2 }
You must also bind it to a name in your [Configuration]() and then list which classes
You must also bind it to a name in your configuration and then list which classes
that should be serialized using it.
It's recommended to throw `java.io.NotSerializableException` in `fromBinary`
if the manifest is unknown. This makes it possible to introduce new message types and
It's recommended to throw `IllegalArgumentException` or ``java.io.NotSerializableException` in
`fromBinary` if the manifest is unknown. This makes it possible to introduce new message types and
send them to nodes that don't know about them. This is typically needed when performing
rolling upgrades, i.e. running a cluster with mixed versions for while.
`NotSerializableException` is treated as a transient problem in the TCP based remoting
Those exceptions are treated as a transient problem in the classic remoting
layer. The problem will be logged and message is dropped. Other exceptions will tear down
the TCP connection because it can be an indication of corrupt bytes from the underlying
transport.
transport. Artery TCP handles all deserialization exceptions as transient problems.
### Serializing ActorRefs
@ -186,16 +198,13 @@ address part of an actors path determines how that actor is communicated with
Storing a local actor path might be the right choice if the retrieval happens
in the same logical context, but it is not enough when deserializing it on a
different network host: for that it would need to include the systems remote
transport address. An actor system is not limited to having just one remote
transport per se, which makes this question a bit more interesting. To find out
the appropriate address to use when sending to `remoteAddr` you can use
`ActorRefProvider.getExternalAddressFor(remoteAddr)` like this:
transport address.
Scala
: @@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #external-address }
: @@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #external-address-default }
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #external-address }
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #external-address-default }
@@@ note
@ -212,12 +221,6 @@ include the unique id.
@@@
This requires that you know at least which type of address will be supported by
the system which will deserialize the resulting actor reference; if you have no
concrete address handy you can create a dummy one for the right protocol using
@scala[`Address(protocol, "", "", 0)`]@java[`new Address(protocol, "", "", 0)`] (assuming that the actual transport used is as
lenient as Akkas RemoteActorRefProvider).
There is also a default remote address which is the one used by cluster support
(and typical systems have just this one); you can get it like this:
@ -227,56 +230,42 @@ Scala
Java
: @@snip [SerializationDocTest.java](/akka-docs/src/test/java/jdocs/serialization/SerializationDocTest.java) { #external-address-default }
Another solution is to encapsulate your serialization code in `Serialization.withTransportInformation`.
It ensures the actorRefs are serialized using systems default address when
no other address is available.
### Deep serialization of Actors
The recommended approach to do deep serialization of internal actor state is to use Akka @ref:[Persistence](persistence.md).
<a id="disable-java-serializer"></a>
## Disabling the Java Serializer
## Java serialization
Java serialization is known to be slow and [prone to attacks](https://community.hpe.com/t5/Security-Research/The-perils-of-Java-deserialization/ba-p/6838995)
of various kinds - it never was designed for high throughput messaging after all. However, it is very
convenient to use, thus it remained the default serialization mechanism that Akka used to
serialize user messages as well as some of its internal messages in previous versions.
of various kinds - it never was designed for high throughput messaging after all.
One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck.
@@@ note
Akka does not use Java Serialization for any of its internal messages.
It is highly encouraged to disable java serialization, so please plan to do so at the earliest possibility you have in your project.
Akka serialization with Java serialization is disabled by default and Akka itself doesn't use Java serialization
for any of its internal messages. It is highly discouraged to enable Java serialization in production.
One may think that network bandwidth and latency limit the performance of remote messaging, but serialization is a more typical bottleneck.
@@@
For user messages, the default serializer, implemented using Java serialization, remains available and enabled.
We do however recommend to disable it entirely and utilise a proper serialization library instead in order effectively utilise
the improved performance and ability for rolling deployments using Artery. Libraries that we recommend to use include,
but are not limited to, [Kryo](https://github.com/EsotericSoftware/kryo) by using the [akka-kryo-serialization](https://github.com/romix/akka-kryo-serialization) library or [Google Protocol Buffers](https://developers.google.com/protocol-buffers/) if you want
more control over the schema evolution of your messages.
In order to completely disable Java Serialization in your Actor system you need to add the following configuration to
your `application.conf`:
```ruby
akka.actor.allow-java-serialization = off
```
This will completely disable the use of `akka.serialization.JavaSerialization` by the
Akka Serialization extension, instead `DisabledJavaSerializer` will
be inserted which will fail explicitly if attempts to use java serialization are made.
The log messages emitted by such serializer SHOULD be treated as potential
The log messages emitted by the disabled Java serializer in production SHOULD be treated as potential
attacks which the serializer prevented, as they MAY indicate an external operator
attempting to send malicious messages intending to use java serialization as attack vector.
The attempts are logged with the SECURITY marker.
Please note that this option does not stop you from manually invoking java serialization.
@@@
## Serialization compatibility
However, for early prototyping it is very convenient to use. For that reason and for compatibility with
older systems that rely on Java serialization it can be enabled with the following configuration:
```ruby
akka.actor.allow-java-serialization = on
```
Akka will still log warning when Java serialization is used and to silent that you may add:
```ruby
akka.actor.warn-about-java-serializer-usage = off
```
### Java serialization compatibility
It is not safe to mix major Scala versions when using the Java serialization as Scala does not guarantee compatibility
and this could lead to very surprising errors.
@ -308,8 +297,8 @@ It must still be possible to deserialize the events that were stored with the ol
## External Akka Serializers
[Akka-quickser by Roman Levenstein](https://github.com/romix/akka-quickser-serialization)
* [Akka-quickser by Roman Levenstein](https://github.com/romix/akka-quickser-serialization)
[Akka-kryo by Roman Levenstein](https://github.com/romix/akka-kryo-serialization)
* [Akka-kryo by Roman Levenstein](https://github.com/romix/akka-kryo-serialization)
[Twitter Chill Scala extensions for Kryo (based on Akka Version 2.3.x but due to backwards compatibility of the Serializer Interface this extension also works with 2.4.x)](https://github.com/twitter/chill)
* [Twitter Chill Scala extensions for Kryo (based on Akka Version 2.3.x but due to backwards compatibility of the Serializer Interface this extension also works with 2.4.x)](https://github.com/twitter/chill)

View file

@ -15,6 +15,10 @@ To use Akka Cluster Typed, you must add the following dependency in your project
For an introduction to Akka Cluster concepts see @ref:[Cluster Specification](../common/cluster.md). This documentation shows how to use the typed
Cluster API.
You need to enable @ref:[serialization](../serialization.md) for your actor messages.
@ref:[Serialization with Jackson](../serialization-jackson.md) is a good choice in many cases and our
recommendation if you don't have other preference.
## Examples
All of the examples below assume the following imports:

View file

@ -4,8 +4,8 @@
The section about @ref:[Changing Behavior](persistence.md#changing-behavior) described how commands and events
can be handled differently depending on the state. One can take that one step further and define the event
handler inside the state classes. In @ref:[next section the command handlers](#command-handlers-in-the-state) are
also defined in the state.
handler inside the state classes. @scala[In @ref:[next section the command handlers](#command-handlers-in-the-state) are
also defined in the state.]
The state can be seen as your domain object and it should contain the core business logic. Then it's a matter
of taste if event handlers and command handlers should be defined in the state or be kept outside it.
@ -24,6 +24,7 @@ in the concrete `EmptyAccount`, `OpenedAccount`, and `ClosedAccount`.]
@java[Notice how the `eventHandler` delegates to methods in the concrete `Account` (state) classes;
`EmptyAccount`, `OpenedAccount`, and `ClosedAccount`.]
@@@ div { .group-scala }
## Command handlers in the state
We can take the previous bank account example one step further by handling the commands in the state too.
@ -31,13 +32,10 @@ We can take the previous bank account example one step further by handling the c
Scala
: @@snip [AccountExampleWithCommandHandlersInState.scala](/akka-cluster-sharding-typed/src/test/scala/docs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.scala) { #account-entity }
Java
: @@snip [AccountExampleWithCommandHandlersInState.java](/akka-cluster-sharding-typed/src/test/java/jdocs/akka/cluster/sharding/typed/AccountExampleWithCommandHandlersInState.java) { #account-entity }
Notice how the command handler is delegating to `applyCommand` in the `Account` (state), which is implemented
in the concrete `EmptyAccount`, `OpenedAccount`, and `ClosedAccount`.
@scala[Notice how the command handler is delegating to `applyCommand` in the `Account` (state), which is implemented
in the concrete `EmptyAccount`, `OpenedAccount`, and `ClosedAccount`.]
@java[Notice how the command handler delegates to methods in the concrete `Account` (state) classes;
`EmptyAccount`, `OpenedAccount`, and `ClosedAccount`.]
@@@
## Optional initial state

Some files were not shown because too many files have changed in this diff Show more