* 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:
parent
4f9a4b5403
commit
a7c43cf573
100 changed files with 105 additions and 235 deletions
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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._
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
""") {
|
||||
|
|
|
|||
|
|
@ -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...")))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -85,7 +85,6 @@ object SupervisorHierarchySpec {
|
|||
type = "akka.actor.SupervisorHierarchySpec$MyDispatcherConfigurator"
|
||||
}
|
||||
akka.loglevel = INFO
|
||||
akka.actor.serialize-messages = off
|
||||
akka.actor.debug.fsm = on
|
||||
""")
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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] {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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 =>
|
||||
|
||||
|
|
|
|||
|
|
@ -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._
|
||||
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
|
|
|
|||
|
|
@ -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._
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -1,6 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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])
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,6 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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("""
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -1,6 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -49,7 +49,6 @@ object RemoteContextAskSpec {
|
|||
loglevel = debug
|
||||
actor {
|
||||
provider = cluster
|
||||
serialize-creators = off
|
||||
serializers {
|
||||
test = "akka.cluster.typed.RemoteContextAskSpecSerializer"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ object RemoteDeployNotAllowedSpec {
|
|||
loglevel = warning
|
||||
actor {
|
||||
provider = cluster
|
||||
serialize-creators = off
|
||||
}
|
||||
remote.classic.netty.tcp.port = 0
|
||||
remote.artery {
|
||||
|
|
|
|||
|
|
@ -37,7 +37,6 @@ object RemoteMessageSpec {
|
|||
loglevel = debug
|
||||
actor {
|
||||
provider = cluster
|
||||
serialize-creators = off
|
||||
serializers {
|
||||
test = "akka.cluster.typed.PingSerializer"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,7 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -1 +0,0 @@
|
|||
akka.actor.serialize-messages=on
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -1,4 +0,0 @@
|
|||
akka.actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
||||
@@@
|
||||
|
|
|
|||
|
|
@ -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}"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,5 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off # FIXME try to enable again?
|
||||
}
|
||||
}
|
||||
|
|
@ -1,5 +0,0 @@
|
|||
akka {
|
||||
actor {
|
||||
serialize-messages = off
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,7 +12,6 @@ object ArterySpecSupport {
|
|||
akka {
|
||||
actor {
|
||||
provider = remote
|
||||
serialize-creators = off
|
||||
}
|
||||
remote.warn-about-direct-use = off
|
||||
remote.artery {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ object CompressionIntegrationSpec {
|
|||
loglevel = INFO
|
||||
|
||||
actor {
|
||||
serialize-messages = off
|
||||
serializers {
|
||||
test-message = "akka.remote.artery.compress.TestMessageSerializer"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
""")
|
||||
|
|
|
|||
|
|
@ -19,7 +19,6 @@ object AkkaProtocolStressTest {
|
|||
ConfigFactory.parseString("""
|
||||
akka {
|
||||
#loglevel = DEBUG
|
||||
actor.serialize-messages = off
|
||||
actor.provider = remote
|
||||
remote.artery.enabled = off
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -149,7 +149,6 @@ object StreamRefsSpec {
|
|||
|
||||
actor {
|
||||
provider = remote
|
||||
serialize-messages = off
|
||||
|
||||
default-mailbox.mailbox-type = "akka.dispatch.UnboundedMailbox"
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue