Config to exclude class prefix from serialize-messages, #24273 (#27517)

* Config to exclude class prefix from serialize-messages, #24273

* Adding no-serialization-verification-needed-class-prefix, with "akka." included
  by default
* This is important now when we disable Java serialization by default.
  There will be many complaints about Akka internal classes that are not marked with
  NoSerializationVerificationNeeded, and we can't really sprinkle that everywhere.
* Not removing NoSerializationVerificationNeeded usage in Akka classes because that
  may break bin compat, even though it's needed any more.

* fix DisabledJavaSerializerWarningSpec
This commit is contained in:
Patrik Nordwall 2019-09-26 22:51:12 +02:00 committed by GitHub
parent 4f9a4b5403
commit a7c43cf573
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
100 changed files with 105 additions and 235 deletions

View file

@ -29,8 +29,7 @@ import static org.junit.Assert.assertSame;
public class LoggingAdapterTest extends JUnitSuite {
private static final Config config =
ConfigFactory.parseString("akka.loglevel = DEBUG\n" + "akka.actor.serialize-messages = off");
private static final Config config = ConfigFactory.parseString("akka.loglevel = DEBUG\n");
@Rule
public AkkaJUnitActorSystemResource actorSystemResource =

View file

@ -1,10 +1,6 @@
akka {
# for the akka.actor.ExtensionSpec
library-extensions += "akka.actor.InstanceCountingExtension"
actor {
serialize-messages = off
}
}
# FIXME Some test depend on this setting when running on windows.

View file

@ -30,7 +30,6 @@ object ActorCreationPerfSpec {
#force-gc = on
#report-metrics = on
}
akka.actor.serialize-messages = off
""")
final case class Create(number: Int, props: () => Props)

View file

@ -26,11 +26,7 @@ object ActorLifeCycleSpec {
}
class ActorLifeCycleSpec
extends AkkaSpec("akka.actor.serialize-messages=off")
with BeforeAndAfterEach
with ImplicitSender
with DefaultTimeout {
class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender with DefaultTimeout {
import ActorLifeCycleSpec._
"An Actor" must {

View file

@ -96,17 +96,13 @@ object ActorWithStashSpec {
var expectedException: TestLatch = null
}
val testConf = """
akka.actor.serialize-messages = off
"""
}
@silent
class JavaActorWithStashSpec extends StashJavaAPI with JUnitSuiteLike
@silent
class ActorWithStashSpec extends AkkaSpec(ActorWithStashSpec.testConf) with DefaultTimeout with BeforeAndAfterEach {
class ActorWithStashSpec extends AkkaSpec with DefaultTimeout with BeforeAndAfterEach {
import ActorWithStashSpec._
override def atStartup: Unit = {

View file

@ -15,9 +15,7 @@ import com.github.ghik.silencer.silent
import scala.concurrent.duration._
import scala.concurrent.Await
class LocalDeathWatchSpec extends AkkaSpec("""
akka.actor.serialize-messages = on
""") with ImplicitSender with DefaultTimeout with DeathWatchSpec
class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec
object DeathWatchSpec {
class Watcher(target: ActorRef, testActor: ActorRef) extends Actor {

View file

@ -253,10 +253,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
"log events and transitions if asked to do so" in {
import akka.util.ccompat.JavaConverters._
val config = ConfigFactory
.parseMap(Map(
"akka.loglevel" -> "DEBUG",
"akka.actor.serialize-messages" -> "off",
"akka.actor.debug.fsm" -> true).asJava)
.parseMap(Map("akka.loglevel" -> "DEBUG", "akka.actor.debug.fsm" -> true).asJava)
.withFallback(system.settings.config)
val fsmEventSystem = ActorSystem("fsmEvent", config)
try {

View file

@ -41,6 +41,7 @@ class FunctionRefSpec extends AkkaSpec("""
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
akka.actor.serialize-messages = on
akka.actor.no-serialization-verification-needed-class-prefix = []
""") with ImplicitSender {
import FunctionRefSpec._

View file

@ -31,6 +31,7 @@ object PropsCreationSpec {
class PropsCreationSpec extends AkkaSpec("""
# test is using Java serialization and relies on serialize-creators=on
akka.actor.serialize-creators = on
akka.actor.no-serialization-verification-needed-class-prefix = []
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off
""") {

View file

@ -19,7 +19,7 @@ import akka.pattern.ask
import com.github.ghik.silencer.silent
@silent
class RestartStrategySpec extends AkkaSpec("akka.actor.serialize-messages = off") with DefaultTimeout {
class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
override def atStartup: Unit = {
system.eventStream.publish(Mute(EventFilter[Exception]("Crashing...")))

View file

@ -28,7 +28,6 @@ object SchedulerSpec {
ConfigFactory.parseString("""
akka.scheduler.implementation = akka.actor.LightArrayRevolverScheduler
akka.scheduler.ticks-per-wheel = 32
akka.actor.serialize-messages = off
""").withFallback(AkkaSpec.testConf)
}

View file

@ -85,7 +85,6 @@ object SupervisorHierarchySpec {
type = "akka.actor.SupervisorHierarchySpec$MyDispatcherConfigurator"
}
akka.loglevel = INFO
akka.actor.serialize-messages = off
akka.actor.debug.fsm = on
""")

View file

@ -20,7 +20,6 @@ import scala.util.control.NonFatal
object SupervisorMiscSpec {
val config = """
akka.actor.serialize-messages = off
pinned-dispatcher {
executor = thread-pool-executor
type = PinnedDispatcher

View file

@ -97,11 +97,10 @@ object SupervisorSpec {
}
val config = ConfigFactory.parseString("""
akka.actor.serialize-messages = off
error-mailbox {
mailbox-type = "akka.actor.SupervisorSpec$Mailbox"
}
""")
error-mailbox {
mailbox-type = "akka.actor.SupervisorSpec$Mailbox"
}
""")
}
class SupervisorSpec

View file

@ -11,10 +11,7 @@ import scala.concurrent.duration._
import akka.testkit.{ AkkaSpec, DefaultTimeout, EventFilter, ImplicitSender }
import akka.pattern.ask
class SupervisorTreeSpec
extends AkkaSpec("akka.actor.serialize-messages = off")
with ImplicitSender
with DefaultTimeout {
class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout {
"In a 3 levels deep supervisor tree (linked in the constructor) we" must {

View file

@ -39,7 +39,6 @@ object TypedActorSpec {
# 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
"""
class CyclicIterator[T](val items: immutable.Seq[T]) extends Iterator[T] {

View file

@ -17,7 +17,10 @@ object DispatchSpec {
}
}
}
class DispatchSpec extends AkkaSpec("akka.actor.serialize-messages = on") with DefaultTimeout {
class DispatchSpec extends AkkaSpec("""
akka.actor.serialize-messages = on
akka.actor.no-serialization-verification-needed-class-prefix = []
""") with DefaultTimeout {
import DispatchSpec._
"The dispatcher" should {

View file

@ -35,6 +35,8 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
getBoolean("akka.actor.serialize-messages") should ===(false)
settings.SerializeAllMessages should ===(false)
settings.NoSerializationVerificationNeededClassPrefix should ===(Set("akka."))
getInt("akka.scheduler.ticks-per-wheel") should ===(512)
getDuration("akka.scheduler.tick-duration", TimeUnit.MILLISECONDS) should ===(10L)
getString("akka.scheduler.implementation") should ===("akka.actor.LightArrayRevolverScheduler")

View file

@ -10,7 +10,6 @@ import com.typesafe.config.ConfigFactory
object ForkJoinPoolStarvationSpec {
val config = ConfigFactory.parseString("""
|akka.actor.serialize-messages = off
|actorhang {
| task-dispatcher {
| mailbox-type = "akka.dispatch.SingleConsumerOnlyUnboundedMailbox"

View file

@ -263,7 +263,6 @@ class SingleConsumerOnlyMailboxSpec extends MailboxSpec {
object SingleConsumerOnlyMailboxVerificationSpec {
case object Ping
val mailboxConf = ConfigFactory.parseString("""
akka.actor.serialize-messages = off
test-unbounded-dispatcher {
mailbox-type = "akka.dispatch.SingleConsumerOnlyUnboundedMailbox"
throughput = 1

View file

@ -15,7 +15,6 @@ object EventStreamSpec {
val config = ConfigFactory.parseString("""
akka {
actor.serialize-messages = off
stdout-loglevel = WARNING
loglevel = INFO
loggers = ["akka.event.EventStreamSpec$MyLog", "%s"]
@ -24,7 +23,6 @@ object EventStreamSpec {
val configUnhandled = ConfigFactory.parseString("""
akka {
actor.serialize-messages = off
stdout-loglevel = WARNING
loglevel = WARNING
actor.debug.unhandled = on

View file

@ -66,6 +66,7 @@ object LoggerSpec {
loggers = ["akka.event.LoggerSpec$$TestLogger1"]
actor {
serialize-messages = on
no-serialization-verification-needed-class-prefix = []
serialization-bindings {
"akka.event.Logging$$LogEvent" = bytes
"java.io.Serializable" = java
@ -79,9 +80,6 @@ object LoggerSpec {
stdout-loglevel = "WARNING"
loglevel = "WARNING"
loggers = ["akka.event.LoggerSpec$TestLogger1"]
actor {
serialize-messages = off
}
}
""").withFallback(AkkaSpec.testConf)

View file

@ -28,7 +28,6 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterAll {
import LoggingReceiveSpec._
val config = ConfigFactory.parseString("""
akka.loglevel=DEBUG # test verifies debug
akka.actor.serialize-messages = off # debug noise from serialization
""").withFallback(AkkaSpec.testConf)
val appLogging =
ActorSystem("logging", ConfigFactory.parseMap(Map("akka.actor.debug.receive" -> true).asJava).withFallback(config))

View file

@ -11,7 +11,6 @@ import Tcp._
class CapacityLimitSpec extends AkkaSpec("""
akka.loglevel = ERROR
akka.io.tcp.max-channels = 4
akka.actor.serialize-creators = on
""") with TcpIntegrationSpecSupport {
"The TCP transport implementation" should {

View file

@ -15,7 +15,6 @@ import com.github.ghik.silencer.silent
class InetAddressDnsResolverSpec extends AkkaSpec("""
akka.io.dns.inet-address.positive-ttl = default
akka.io.dns.inet-address.negative-ttl = default
akka.actor.serialize-creators = on
""") { thisSpecs =>
"The DNS resolver default ttl's" must {
@ -124,7 +123,6 @@ class InetAddressDnsResolverSpec extends AkkaSpec("""
class InetAddressDnsResolverConfigSpec extends AkkaSpec("""
akka.io.dns.inet-address.positive-ttl = forever
akka.io.dns.inet-address.negative-ttl = never
akka.actor.serialize-creators = on
""") {
thisSpecs =>

View file

@ -45,7 +45,6 @@ class TcpConnectionSpec extends AkkaSpec("""
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.io.tcp.trace-logging = on
akka.io.tcp.register-timeout = 500ms
akka.actor.serialize-creators = on
""") with WithLogCapturing { thisSpecs =>
import TcpConnectionSpec._

View file

@ -21,7 +21,6 @@ class TcpIntegrationSpec extends AkkaSpec("""
akka.loglevel = debug
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.io.tcp.trace-logging = on
akka.actor.serialize-creators = on
""") with TcpIntegrationSpecSupport with TimeLimits with WithLogCapturing {
def verifyActorTermination(actor: ActorRef): Unit = {

View file

@ -19,7 +19,6 @@ import akka.io.TcpListenerSpec.RegisterChannel
class TcpListenerSpec extends AkkaSpec("""
akka.io.tcp.batch-accept-limit = 2
akka.actor.serialize-creators = on
""") {
"A TcpListener" must {

View file

@ -13,7 +13,6 @@ import akka.testkit.SocketUtil.temporaryServerAddresses
class UdpConnectedIntegrationSpec extends AkkaSpec("""
akka.loglevel = INFO
akka.actor.serialize-creators = on
""") with ImplicitSender {
val addresses = temporaryServerAddresses(5, udp = true)

View file

@ -17,8 +17,7 @@ import java.net.DatagramSocket
class UdpIntegrationSpec extends AkkaSpec("""
akka.loglevel = INFO
# tests expect to be able to mutate messages
akka.actor.serialize-messages = off
akka.actor.serialize-creators = on""") with ImplicitSender {
""") with ImplicitSender {
def bindUdp(handler: ActorRef): InetSocketAddress = {
val commander = TestProbe()

View file

@ -21,7 +21,7 @@ object PatternSpec {
}
}
class PatternSpec extends AkkaSpec("akka.actor.serialize-messages = off") {
class PatternSpec extends AkkaSpec {
implicit val ec = system.dispatcher
import PatternSpec._

View file

@ -19,7 +19,6 @@ 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
}

View file

@ -17,7 +17,6 @@ import akka.pattern.ask
object ResizerSpec {
val config = """
akka.actor.serialize-messages = off
akka.actor.deployment {
/router1 {
router = round-robin-pool

View file

@ -18,7 +18,6 @@ import akka.pattern.{ ask, pipe }
object RoutingSpec {
val config = """
akka.actor.serialize-messages = off
akka.actor.deployment {
/router1 {
router = round-robin-pool

View file

@ -9,10 +9,7 @@ import scala.concurrent.Await
import akka.actor.{ Actor, Props }
import akka.testkit.{ AkkaSpec, DefaultTimeout, ImplicitSender, TestLatch }
class SmallestMailboxSpec
extends AkkaSpec("akka.actor.serialize-messages = off")
with DefaultTimeout
with ImplicitSender {
class SmallestMailboxSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
"smallest mailbox pool" must {

View file

@ -19,6 +19,7 @@ class DisabledJavaSerializerWarningSpec extends AkkaSpec("""
akka.actor {
allow-java-serialization = off
serialize-messages = on
no-serialization-verification-needed-class-prefix = []
# this is by default on, but tests are running with off
warn-about-java-serializer-usage = on
}

View file

@ -55,8 +55,6 @@ object SerializationSetupSpec {
Some(ConfigFactory.parseString("""
akka {
actor {
serialize-messages = off
allow-java-serialization = on
# this is by default on, but tests are running with off, use defaults here

View file

@ -31,7 +31,6 @@ object SerializationTests {
val serializeConf = s"""
akka {
actor {
serialize-messages = off
serializers {
test = "akka.serialization.NoopSerializer"
test2 = "akka.serialization.NoopSerializer2"
@ -105,7 +104,7 @@ object SerializationTests {
class FooActor extends Actor {
def receive = {
case s: String => sender() ! s
case msg => sender() ! msg
}
}
@ -114,7 +113,7 @@ object SerializationTests {
receiveBuilder().build()
}
class NonSerializableActor(@unused system: ActorSystem) extends Actor {
class NonSerializableActor(@unused arg: AnyRef) extends Actor {
def receive = {
case s: String => sender() ! s
}
@ -292,11 +291,21 @@ class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerial
system.stop(b)
intercept[IllegalArgumentException] {
system.actorOf(Props(new NonSerializableActor(system)))
system.actorOf(Props(classOf[NonSerializableActor], new AnyRef))
}
}
"not verify akka creators" in {
EventFilter.warning(start = "ok", occurrences = 1).intercept {
// ActorSystem is not possible to serialize, but ok since it starts with "akka."
val a = system.actorOf(Props(classOf[NonSerializableActor], system))
// to verify that nothing is logged
system.log.warning("ok")
system.stop(a)
}
}
"verify messages" in {
val a = system.actorOf(Props[FooActor])
Await.result(a ? "pigdog", timeout.duration) should ===("pigdog")
@ -308,6 +317,18 @@ class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerial
}
system.stop(a)
}
"not verify akka messages" in {
val a = system.actorOf(Props[FooActor])
EventFilter.warning(start = "ok", occurrences = 1).intercept {
// ActorSystem is not possible to serialize, but ok since it starts with "akka."
val message = system
Await.result(a ? message, timeout.duration) should ===(message)
// to verify that nothing is logged
system.log.warning("ok")
}
system.stop(a)
}
}
class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyReferenceSystem) {

View file

@ -16,7 +16,6 @@ import org.scalatest.WordSpecLike
object ActorRefSerializationSpec {
def config = ConfigFactory.parseString("""
akka.actor {
serialize-messages = off
# test is verifying Java serialization of ActorRef
allow-java-serialization = on
warn-about-java-serializer-usage = off

View file

@ -119,6 +119,10 @@ akka {
# as marked with deploy.scope == LocalScope are exempt from verification.
serialize-creators = off
# If serialize-messages or serialize-creators are enabled classes that starts with
# a prefix listed here are not verified.
no-serialization-verification-needed-class-prefix = ["akka."]
# Timeout for send operations to top-level actors which are in the process
# of being started. This is only relevant if using a bounded mailbox or the
# CallingThreadDispatcher for a top-level actor.

View file

@ -424,6 +424,10 @@ object ActorSystem {
final val EnableAdditionalSerializationBindings: Boolean = true
final val SerializeAllMessages: Boolean = getBoolean("akka.actor.serialize-messages")
final val SerializeAllCreators: Boolean = getBoolean("akka.actor.serialize-creators")
final val NoSerializationVerificationNeededClassPrefix: Set[String] = {
import akka.util.ccompat.JavaConverters._
getStringList("akka.actor.no-serialization-verification-needed-class-prefix").asScala.toSet
}
final val LogLevel: String = getString("akka.loglevel")
final val StdoutLogLevel: String = getString("akka.stdout-loglevel")

View file

@ -256,7 +256,8 @@ private[akka] trait Children { this: ActorCell =>
name: String,
async: Boolean,
systemService: Boolean): ActorRef = {
if (cell.system.settings.SerializeAllCreators && !systemService && props.deploy.scope != LocalScope) {
val settings = cell.system.settings
if (settings.SerializeAllCreators && !systemService && props.deploy.scope != LocalScope) {
val oldInfo = Serialization.currentTransportInformation.value
try {
val ser = SerializationExtension(cell.system)
@ -266,7 +267,8 @@ private[akka] trait Children { this: ActorCell =>
props.args.forall(
arg =>
arg == null ||
arg.isInstanceOf[NoSerializationVerificationNeeded] || {
arg.isInstanceOf[NoSerializationVerificationNeeded] ||
settings.NoSerializationVerificationNeededClassPrefix.exists(arg.getClass.getName.startsWith) || {
val o = arg.asInstanceOf[AnyRef]
val serializer = ser.findSerializerFor(o)
val bytes = serializer.toBinary(o)

View file

@ -169,14 +169,18 @@ private[akka] trait Dispatch { this: ActorCell =>
unwrappedMessage match {
case _: NoSerializationVerificationNeeded => envelope
case msg =>
val deserializedMsg = try {
serializeAndDeserializePayload(msg)
} catch {
case NonFatal(e) => throw SerializationCheckFailedException(msg, e)
}
envelope.message match {
case dl: DeadLetter => envelope.copy(message = dl.copy(message = deserializedMsg))
case _ => envelope.copy(message = deserializedMsg)
if (system.settings.NoSerializationVerificationNeededClassPrefix.exists(msg.getClass.getName.startsWith))
envelope
else {
val deserializedMsg = try {
serializeAndDeserializePayload(msg)
} catch {
case NonFatal(e) => throw SerializationCheckFailedException(msg, e)
}
envelope.message match {
case dl: DeadLetter => envelope.copy(message = dl.copy(message = deserializedMsg))
case _ => envelope.copy(message = deserializedMsg)
}
}
}
}

View file

@ -39,9 +39,6 @@ class ORSetSerializationBenchmark {
akka.actor.provider=cluster
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
}
""")
private val system1 = ActorSystem("ORSetSerializationBenchmark", config)

View file

@ -1,6 +0,0 @@
akka {
actor {
serialize-messages = off
}
}

View file

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

View file

@ -5,7 +5,6 @@
package akka.cluster.sharding.typed
import scala.concurrent.duration.FiniteDuration
import akka.actor.NoSerializationVerificationNeeded
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.cluster.ClusterSettings.DataCenter
@ -267,8 +266,7 @@ final class ClusterShardingSettings(
val shardRegionQueryTimeout: FiniteDuration,
val stateStoreMode: ClusterShardingSettings.StateStoreMode,
val tuningParameters: ClusterShardingSettings.TuningParameters,
val coordinatorSingletonSettings: ClusterSingletonManagerSettings)
extends NoSerializationVerificationNeeded {
val coordinatorSingletonSettings: ClusterSingletonManagerSettings) {
import akka.cluster.sharding.typed.ClusterShardingSettings.StateStoreModeDData
import akka.cluster.sharding.typed.ClusterShardingSettings.StateStoreModePersistence

View file

@ -52,8 +52,6 @@ object ClusterShardingSpec {
akka.coordinated-shutdown.run-by-actor-system-terminate = off
akka.actor {
serialize-messages = off
serializers {
test = "akka.cluster.sharding.typed.scaladsl.ClusterShardingSpec$$Serializer"
}

View file

@ -1,6 +0,0 @@
akka {
actor {
serialize-messages = off
}
}

View file

@ -20,7 +20,6 @@ object InactiveEntityPassivationSpec {
akka.actor.provider = "cluster"
akka.remote.classic.netty.tcp.port = 0
akka.remote.artery.canonical.port = 0
akka.actor.serialize-messages = off
""")
val enabledConfig = ConfigFactory.parseString("""

View file

@ -29,7 +29,6 @@ object DistributedPubSubMediatorSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString("""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.actor.serialize-messages = off
akka.remote.log-remote-lifecycle-events = off
akka.cluster.auto-down-unreachable-after = 0s
akka.cluster.pub-sub.max-delta-elements = 500

View file

@ -22,7 +22,6 @@ object MultiDcSingletonManagerSpec extends MultiNodeConfig {
commonConfig(ConfigFactory.parseString("""
akka.actor.provider = "cluster"
akka.actor.serialize-creators = off
akka.remote.log-remote-lifecycle-events = off"""))
nodeConfig(controller) {

View file

@ -1,6 +0,0 @@
akka {
actor {
serialize-messages = off
}
}

View file

@ -20,7 +20,6 @@ object DistributedPubSubMediatorRouterSpec {
def config(routingLogic: String) = s"""
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.actor.serialize-messages = off
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port=0
akka.remote.log-remote-lifecycle-events = off

View file

@ -120,7 +120,7 @@ object Replicator {
replyTo: ActorRef[GetResponse[A]])
extends Command
@DoNotInherit sealed abstract class GetResponse[A <: ReplicatedData] extends NoSerializationVerificationNeeded {
@DoNotInherit sealed abstract class GetResponse[A <: ReplicatedData] {
def key: Key[A]
}
@ -180,8 +180,7 @@ object Replicator {
key: Key[A],
writeConsistency: WriteConsistency,
replyTo: ActorRef[UpdateResponse[A]])(val modify: Option[A] => A)
extends Command
with NoSerializationVerificationNeeded {
extends Command {
/**
* Modify value of local `Replicator` and replicate with given `writeConsistency`.
@ -200,7 +199,7 @@ object Replicator {
}
@DoNotInherit sealed abstract class UpdateResponse[A <: ReplicatedData] extends NoSerializationVerificationNeeded {
@DoNotInherit sealed abstract class UpdateResponse[A <: ReplicatedData] {
def key: Key[A]
}
final case class UpdateSuccess[A <: ReplicatedData](key: Key[A]) extends UpdateResponse[A] with DeadLetterSuppression
@ -312,9 +311,8 @@ object Replicator {
consistency: WriteConsistency,
replyTo: ActorRef[DeleteResponse[A]])
extends Command
with NoSerializationVerificationNeeded
sealed trait DeleteResponse[A <: ReplicatedData] extends NoSerializationVerificationNeeded {
sealed trait DeleteResponse[A <: ReplicatedData] {
def key: Key[A]
}
final case class DeleteSuccess[A <: ReplicatedData](key: Key[A]) extends DeleteResponse[A]

View file

@ -4,8 +4,6 @@
package akka.cluster.ddata.typed.scaladsl
import akka.actor.NoSerializationVerificationNeeded
import akka.cluster.{ ddata => dd }
import akka.cluster.ddata.Key
import akka.cluster.ddata.ReplicatedData
@ -145,7 +143,6 @@ object Replicator {
writeConsistency: WriteConsistency,
replyTo: ActorRef[UpdateResponse[A]])(val modify: Option[A] => A)
extends Command
with NoSerializationVerificationNeeded {}
type UpdateResponse[A <: ReplicatedData] = dd.Replicator.UpdateResponse[A]
type UpdateSuccess[A <: ReplicatedData] = dd.Replicator.UpdateSuccess[A]
@ -282,7 +279,6 @@ object Replicator {
consistency: WriteConsistency,
replyTo: ActorRef[DeleteResponse[A]])
extends Command
with NoSerializationVerificationNeeded
type DeleteResponse[A <: ReplicatedData] = dd.Replicator.DeleteResponse[A]
type DeleteSuccess[A <: ReplicatedData] = dd.Replicator.DeleteSuccess[A]

View file

@ -4,7 +4,6 @@
package akka.cluster.typed
import akka.actor.NoSerializationVerificationNeeded
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.singleton.{
@ -50,8 +49,7 @@ final class ClusterSingletonSettings(
val singletonIdentificationInterval: FiniteDuration,
val removalMargin: FiniteDuration,
val handOverRetryInterval: FiniteDuration,
val bufferSize: Int)
extends NoSerializationVerificationNeeded {
val bufferSize: Int) {
def withRole(role: String): ClusterSingletonSettings = copy(role = Some(role))
@ -270,8 +268,7 @@ final class ClusterSingletonManagerSettings(
val singletonName: String,
val role: Option[String],
val removalMargin: FiniteDuration,
val handOverRetryInterval: FiniteDuration)
extends NoSerializationVerificationNeeded {
val handOverRetryInterval: FiniteDuration) {
def withSingletonName(name: String): ClusterSingletonManagerSettings = copy(singletonName = name)

View file

@ -26,10 +26,7 @@ public class ClusterApiTest extends JUnitSuite {
+ "akka.remote.artery.canonical.hostname = 127.0.0.1 \n"
+ "akka.cluster.jmx.multi-mbeans-in-same-jvm = on \n"
+ "akka.coordinated-shutdown.terminate-actor-system = off \n"
+ "akka.coordinated-shutdown.run-by-actor-system-terminate = off \n"
+ "akka.actor { \n"
+ " serialize-messages = off \n"
+ "}");
+ "akka.coordinated-shutdown.run-by-actor-system-terminate = off \n");
ActorSystem<?> system1 =
ActorSystem.wrap(akka.actor.ActorSystem.create("ClusterApiTest", config));

View file

@ -26,9 +26,6 @@ object ClusterApiSpec {
akka.cluster.jmx.multi-mbeans-in-same-jvm = on
akka.coordinated-shutdown.terminate-actor-system = off
akka.coordinated-shutdown.run-by-actor-system-terminate = off
akka.actor {
serialize-messages = off
}
# generous timeout for cluster forming probes
akka.actor.testkit.typed.default-timeout = 10s
# disable this or we cannot be sure to observe node end state on the leaving side

View file

@ -26,7 +26,6 @@ object ClusterSingletonApiSpec {
val config = ConfigFactory.parseString(s"""
akka.actor {
provider = cluster
serialize-messages = off
serializers {
test = "akka.cluster.typed.ClusterSingletonApiSpec$$PingSerializer"

View file

@ -23,10 +23,6 @@ object ClusterSingletonPersistenceSpec {
akka.coordinated-shutdown.terminate-actor-system = off
akka.coordinated-shutdown.run-by-actor-system-terminate = off
akka.actor {
serialize-messages = off
}
akka.persistence.journal.plugin = "akka.persistence.journal.inmem"
""".stripMargin)

View file

@ -49,7 +49,6 @@ object RemoteContextAskSpec {
loglevel = debug
actor {
provider = cluster
serialize-creators = off
serializers {
test = "akka.cluster.typed.RemoteContextAskSpecSerializer"
}

View file

@ -21,7 +21,6 @@ object RemoteDeployNotAllowedSpec {
loglevel = warning
actor {
provider = cluster
serialize-creators = off
}
remote.classic.netty.tcp.port = 0
remote.artery {

View file

@ -37,7 +37,6 @@ object RemoteMessageSpec {
loglevel = debug
actor {
provider = cluster
serialize-creators = off
serializers {
test = "akka.cluster.typed.PingSerializer"
}

View file

@ -31,7 +31,6 @@ object ClusterReceptionistSpec {
akka.loglevel = DEBUG # issue #24960
akka.actor {
provider = cluster
serialize-messages = off
serializers {
test = "akka.cluster.typed.internal.receptionist.ClusterReceptionistSpec$$PingSerializer"
}

View file

@ -116,8 +116,6 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
exercise-actors = on
}
akka.actor.serialize-messages = off
akka.actor.serialize-creators = off
akka.actor.provider = cluster
akka.cluster {
failure-detector.acceptable-heartbeat-pause = 10s

View file

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

View file

@ -1,7 +0,0 @@
akka {
actor {
serialize-messages = off
}
}

View file

@ -1 +0,0 @@
akka.actor.serialize-messages=on

View file

@ -35,7 +35,6 @@ object PerformanceSpec extends MultiNodeConfig {
akka.remote.classic.log-frame-size-exceeding=1000b
akka.testconductor.barrier-timeout = 60 s
akka.cluster.distributed-data.gossip-interval = 1 s
akka.actor.serialize-messages = off
#akka.cluster.distributed-data.durable.keys = ["*"]
#akka.cluster.distributed-data.durable.lmdb.dir = target/PerformanceSpec-${System.currentTimeMillis}-ddata

View file

@ -26,9 +26,6 @@ object ReplicatorMapDeltaSpec extends MultiNodeConfig {
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.log-dead-letters-during-shutdown = off
akka.actor {
serialize-messages = off
}
"""))
testTransport(on = true)

View file

@ -23,9 +23,6 @@ object ReplicatorORSetDeltaSpec extends MultiNodeConfig {
akka.loglevel = INFO
akka.actor.provider = "cluster"
akka.log-dead-letters-during-shutdown = off
akka.actor {
serialize-messages = off
}
"""))
testTransport(on = true)

View file

@ -1,4 +0,0 @@
akka.actor {
serialize-messages = off
}

View file

@ -35,9 +35,6 @@ class ReplicatedDataSerializerSpec
akka.actor.provider=cluster
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
}
""")))
with WordSpecLike
with Matchers

View file

@ -39,9 +39,6 @@ class ReplicatorMessageSerializerSpec
akka.actor.provider=cluster
akka.remote.classic.netty.tcp.port=0
akka.remote.artery.canonical.port = 0
akka.actor {
serialize-messages = off
}
""")))
with WordSpecLike
with Matchers

View file

@ -61,22 +61,6 @@ Akka provides serializers for several primitive types and [protobuf](http://code
depending on the akka-remote module), so normally you don't need to add
configuration for that if you send raw protobuf messages as actor messages.
### Verification
Normally, messages sent between local actors (i.e. same JVM) do not undergo serialization. For testing, sometimes, it may be desirable to force serialization on all messages (both remote and local). If you want to do this in order to verify that your messages are serializable you can enable the following config option:
@@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #serialize-messages-config }
If you want to verify that your `Props` are serializable you can enable the following config option:
@@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #serialize-creators-config }
@@@ warning
We recommend having these config options turned on **only** when you're running tests. Turning these options on in production is pointless, as it would negatively impact the performance of local message passing without giving any gain.
@@@
### Programmatic
If you want to programmatically serialize/deserialize using Akka Serialization,
@ -275,3 +259,23 @@ It must still be possible to deserialize the events that were stored with the ol
* [Akka-kryo by Roman Levenstein](https://github.com/romix/akka-kryo-serialization)
* [Twitter Chill Scala extensions for Kryo](https://github.com/twitter/chill)
### Verification
Normally, messages sent between local actors (i.e. same JVM) do not undergo serialization. For testing, sometimes, it may be desirable to force serialization on all messages (both remote and local). If you want to do this in order to verify that your messages are serializable you can enable the following config option:
@@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #serialize-messages-config }
Certain messages can be excluded from verification by extending the marker @scala[trait]@java[interface]
`akka.actor.NoSerializationVerificationNeeded` or define a class name prefix in configuration
`akka.actor.no-serialization-verification-needed-class-prefix`.
If you want to verify that your `Props` are serializable you can enable the following config option:
@@snip [SerializationDocSpec.scala](/akka-docs/src/test/scala/docs/serialization/SerializationDocSpec.scala) { #serialize-creators-config }
@@@ warning
We recommend having these config options turned on **only** when you're running tests. Turning these options on in production is pointless, as it would negatively impact the performance of local message passing without giving any gain.
@@@

View file

@ -18,8 +18,6 @@ object JournalSpec {
val config: Config = ConfigFactory.parseString(s"""
akka.persistence.publish-plugin-commands = on
akka.actor {
serialize-messages = off
serialize-creators = off
serializers {
persistence-tck-test = "${classOf[TestSerializer].getName}"
}

View file

@ -111,8 +111,6 @@ object PerformanceSpec {
}
class PerformanceSpec extends ScalaTestWithActorTestKit(ConfigFactory.parseString(s"""
akka.actor.serialize-creators = off
akka.actor.serialize-messages = off
akka.persistence.publish-plugin-commands = on
akka.persistence.journal.plugin = "akka.persistence.journal.leveldb"
akka.persistence.journal.leveldb.dir = "target/journal-PerformanceSpec"

View file

@ -64,8 +64,9 @@ object PersistenceSpec {
.map(ConfigFactory.parseString(_))
.getOrElse(ConfigFactory.empty())
.withFallback(ConfigFactory.parseString(s"""
akka.actor.serialize-creators = ${serialization}
akka.actor.serialize-messages = ${serialization}
akka.actor.serialize-creators = $serialization
akka.actor.serialize-messages = $serialization
akka.actor.no-serialization-verification-needed-class-prefix = []
# test is using Java serialization and not priority to rewrite
akka.actor.allow-java-serialization = on
akka.actor.warn-about-java-serializer-usage = off

View file

@ -40,8 +40,6 @@ object LatencySpec extends MultiNodeConfig {
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor {
provider = remote
serialize-creators = false
serialize-messages = false
}
remote.artery {
enabled = on

View file

@ -39,8 +39,6 @@ object MaxThroughputSpec extends MultiNodeConfig {
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor {
provider = remote
serialize-creators = false
serialize-messages = false
serializers {
test = "akka.remote.artery.MaxThroughputSpec$$TestSerializer"

View file

@ -51,8 +51,6 @@ object AeronStreamLatencySpec extends MultiNodeConfig {
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor {
provider = remote
serialize-creators = false
serialize-messages = false
}
remote.artery {
enabled = off

View file

@ -42,8 +42,6 @@ object AeronStreamMaxThroughputSpec extends MultiNodeConfig {
testconductor.barrier-timeout = ${barrierTimeout.toSeconds}s
actor {
provider = remote
serialize-creators = false
serialize-messages = false
}
remote.artery.enabled = off
}

View file

@ -1,5 +0,0 @@
akka {
actor {
serialize-messages = off # FIXME try to enable again?
}
}

View file

@ -1,5 +0,0 @@
akka {
actor {
serialize-messages = off
}
}

View file

@ -23,7 +23,6 @@ object RemoteRouterSpec {
class RemoteRouterSpec extends AkkaSpec(s"""
akka.actor.provider = remote
akka.actor.serialize-messages = off
akka.remote.use-unsafe-remote-features-outside-cluster = on
akka.remote.classic.netty.tcp {
hostname = localhost

View file

@ -53,7 +53,6 @@ abstract class AbstractTransientSerializationErrorSpec(config: Config)
actor {
provider = remote
warn-about-java-serializer-usage = off
serialize-creators = off
serializers {
test = "akka.remote.TransientSerializationErrorSpec$TestSerializer"
}

View file

@ -12,7 +12,6 @@ object ArterySpecSupport {
akka {
actor {
provider = remote
serialize-creators = off
}
remote.warn-about-direct-use = off
remote.artery {

View file

@ -24,10 +24,7 @@ object RemoteMessageSerializationSpec {
}
}
class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec("""
akka.actor.serialize-messages = off
akka.actor.serialize-creators = off
""") with ImplicitSender {
class RemoteMessageSerializationSpec extends ArteryMultiNodeSpec with ImplicitSender {
val maxPayloadBytes = RARP(system).provider.remoteSettings.Artery.Advanced.MaximumFrameSize

View file

@ -49,7 +49,6 @@ object SendQueueSpec {
}
class SendQueueSpec extends AkkaSpec("""
akka.actor.serialize-messages = off
akka.stream.materializer.debug.fuzzing-mode = on
akka.stream.secret-test-fuzzing-warning-disable = yep
""") with ImplicitSender {

View file

@ -22,7 +22,6 @@ object CompressionIntegrationSpec {
loglevel = INFO
actor {
serialize-messages = off
serializers {
test-message = "akka.remote.artery.compress.TestMessageSerializer"
}

View file

@ -83,7 +83,6 @@ object RemotingSpec {
akka {
actor.provider = remote
actor.serialize-messages = off
# test is using Java serialization and not priority to rewrite
actor.allow-java-serialization = on
actor.warn-about-java-serializer-usage = off
@ -503,9 +502,6 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
val config = ConfigFactory
.parseString(
"""
# Additional internal serialization verification need so be off, otherwise it triggers two error messages
# instead of one: one for the internal check, and one for the actual remote send -- tripping off this test
akka.actor.serialize-messages = off
akka.remote.classic.enabled-transports = ["akka.remote.classic.test", "akka.remote.classic.netty.tcp"]
akka.remote.classic.test.local-address = "test://other-system@localhost:12347"
""")

View file

@ -19,7 +19,6 @@ object AkkaProtocolStressTest {
ConfigFactory.parseString("""
akka {
#loglevel = DEBUG
actor.serialize-messages = off
actor.provider = remote
remote.artery.enabled = off

View file

@ -27,7 +27,6 @@ object SystemMessageDeliveryStressTest {
#loglevel = DEBUG
remote.artery.enabled = false
actor.provider = remote
actor.serialize-messages = off
# test is using Java serialization and not priority to rewrite
actor.allow-java-serialization = on
actor.warn-about-java-serializer-usage = off

View file

@ -31,8 +31,6 @@ object AllowJavaSerializationOffSpec {
Some(ConfigFactory.parseString("""
akka {
actor {
serialize-messages = off
allow-java-serialization = on
# this is by default on, but tests are running with off, use defaults here

View file

@ -128,8 +128,6 @@ class DaemonMsgCreateSerializerAllowJavaSerializationSpec
class DaemonMsgCreateSerializerNoJavaSerializationSpec extends AkkaSpec("""
akka.actor.allow-java-serialization=off
akka.actor.serialize-messages=off
akka.actor.serialize-creators=off
""") with SerializationVerification {
import DaemonMsgCreateSerializerAllowJavaSerializationSpec.MyActor

View file

@ -75,7 +75,6 @@ abstract class AbstractSerializationTransportInformationSpec(config: Config)
actor {
provider = remote
warn-about-java-serializer-usage = off
serialize-creators = off
serializers {
test = "akka.remote.serialization.SerializationTransportInformationSpec$TestSerializer"
}

View file

@ -19,7 +19,6 @@ import scala.util.control.NoStackTrace
class FlowLogSpec extends StreamSpec("""
akka.loglevel = DEBUG # test verifies logging
akka.actor.serialize-messages = off
""") with ScriptedTest {
val logProbe = {

View file

@ -13,7 +13,6 @@ import akka.testkit.TestProbe
class FlowWithContextLogSpec extends StreamSpec("""
akka.loglevel = DEBUG # test verifies logging
akka.actor.serialize-messages = off
""") with ScriptedTest {
val logProbe = {

View file

@ -149,7 +149,6 @@ object StreamRefsSpec {
actor {
provider = remote
serialize-messages = off
default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"
}