* 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:
parent
a4f090b622
commit
3efc1c2877
191 changed files with 4041 additions and 2321 deletions
|
|
@ -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)
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
@ -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()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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] =
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
}
|
||||
}
|
||||
|
|
@ -3,8 +3,7 @@ akka {
|
|||
library-extensions += "akka.actor.InstanceCountingExtension"
|
||||
|
||||
actor {
|
||||
serialize-messages = on
|
||||
warn-about-java-serializer-usage = off
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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) = {
|
||||
|
|
|
|||
|
|
@ -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._
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
"""
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
{
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -41,7 +41,6 @@ class ORSetSerializationBenchmark {
|
|||
akka.remote.artery.canonical.port = 0
|
||||
akka.actor {
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
}
|
||||
""")
|
||||
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -1,7 +1,6 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = on
|
||||
warn-about-java-serializer-usage = off
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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])
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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() {
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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() {}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"]"""))
|
||||
|
|
|
|||
|
|
@ -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"]"""))
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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"]""")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -1,7 +1,6 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = on
|
||||
warn-about-java-serializer-usage = off
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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())
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -1,7 +1,6 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = on
|
||||
warn-about-java-serializer-usage = off
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -26,7 +26,6 @@ object ClusterSingletonApiSpec {
|
|||
akka.actor {
|
||||
provider = cluster
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
|
||||
serializers {
|
||||
test = "akka.cluster.typed.ClusterSingletonApiSpec$$PingSerializer"
|
||||
|
|
|
|||
|
|
@ -24,7 +24,6 @@ object ClusterSingletonPersistenceSpec {
|
|||
|
||||
akka.actor {
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
}
|
||||
|
||||
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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@" """)
|
||||
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -1,7 +1,6 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
warn-about-java-serializer-usage = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -25,7 +25,6 @@ object ReplicatorORSetDeltaSpec extends MultiNodeConfig {
|
|||
akka.log-dead-letters-during-shutdown = off
|
||||
akka.actor {
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
}
|
||||
"""))
|
||||
|
||||
|
|
|
|||
|
|
@ -1,5 +1,4 @@
|
|||
akka.actor {
|
||||
serialize-messages = on
|
||||
warn-about-java-serializer-usage = off
|
||||
serialize-messages = off
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -37,7 +37,6 @@ class ReplicatedDataSerializerSpec
|
|||
akka.remote.artery.canonical.port = 0
|
||||
akka.actor {
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
}
|
||||
""")))
|
||||
with WordSpecLike
|
||||
|
|
|
|||
|
|
@ -41,7 +41,6 @@ class ReplicatorMessageSerializerSpec
|
|||
akka.remote.artery.canonical.port = 0
|
||||
akka.actor {
|
||||
serialize-messages = off
|
||||
allow-java-serialization = off
|
||||
}
|
||||
""")))
|
||||
with WordSpecLike
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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:
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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 }
|
||||
|
|
|
|||
|
|
@ -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).
|
||||
|
||||
@@@
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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 actor’s 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 system’s 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 Akka’s 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)
|
||||
|
|
|
|||
|
|
@ -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:
|
||||
|
|
|
|||
|
|
@ -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
Loading…
Add table
Add a link
Reference in a new issue