2011-06-07 06:36:21 +05:30
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2011-06-07 06:36:21 +05:30
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
package akka.serialization
|
|
|
|
|
|
2012-06-21 16:09:14 +02:00
|
|
|
import language.postfixOps
|
2012-02-07 15:51:41 +01:00
|
|
|
import akka.testkit.{ AkkaSpec, EventFilter }
|
2011-12-08 16:07:03 +01:00
|
|
|
import akka.actor._
|
2013-03-05 16:19:54 +01:00
|
|
|
import akka.dispatch.sysmsg._
|
2011-12-08 16:07:03 +01:00
|
|
|
import java.io._
|
2012-06-29 16:06:26 +02:00
|
|
|
import scala.concurrent.Await
|
2011-12-27 17:30:05 +01:00
|
|
|
import akka.util.Timeout
|
2012-09-21 14:50:06 +02:00
|
|
|
import scala.concurrent.duration._
|
2014-01-31 11:14:13 +01:00
|
|
|
import scala.beans.BeanInfo
|
2013-01-25 14:24:23 +13:00
|
|
|
import com.typesafe.config._
|
2012-01-18 10:18:51 +01:00
|
|
|
import akka.pattern.ask
|
2014-01-31 11:14:13 +01:00
|
|
|
import org.apache.commons.codec.binary.Hex.encodeHex
|
2016-05-26 11:58:13 +02:00
|
|
|
import java.nio.ByteOrder
|
|
|
|
|
import java.nio.ByteBuffer
|
2016-07-06 18:48:15 +09:00
|
|
|
import akka.actor.NoSerializationVerificationNeeded
|
|
|
|
|
import test.akka.serialization.NoVerification
|
2011-12-21 11:25:40 +01:00
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
object SerializationTests {
|
2011-11-22 13:04:10 +01:00
|
|
|
|
2015-01-30 16:05:36 +01:00
|
|
|
val serializeConf = s"""
|
2011-11-22 13:04:10 +01:00
|
|
|
akka {
|
|
|
|
|
actor {
|
2013-08-23 14:39:21 +02:00
|
|
|
serialize-messages = off
|
2011-11-22 13:04:10 +01:00
|
|
|
serializers {
|
2017-02-17 16:41:25 +01:00
|
|
|
test = "akka.serialization.NoopSerializer"
|
2011-11-22 13:04:10 +01:00
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
|
2011-11-22 13:04:10 +01:00
|
|
|
serialization-bindings {
|
2015-01-30 16:05:36 +01:00
|
|
|
"akka.serialization.SerializationTests$$Person" = java
|
|
|
|
|
"akka.serialization.SerializationTests$$Address" = java
|
2013-01-25 14:24:23 +13:00
|
|
|
"akka.serialization.TestSerializable" = test
|
2015-01-30 16:05:36 +01:00
|
|
|
"akka.serialization.SerializationTests$$PlainMessage" = test
|
|
|
|
|
"akka.serialization.SerializationTests$$A" = java
|
|
|
|
|
"akka.serialization.SerializationTests$$B" = test
|
|
|
|
|
"akka.serialization.SerializationTests$$D" = test
|
2011-11-22 13:04:10 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-02-06 21:12:26 +01:00
|
|
|
"""
|
2011-11-22 13:04:10 +01:00
|
|
|
|
2011-06-07 06:36:21 +05:30
|
|
|
@BeanInfo
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Address(no: String, street: String, city: String, zip: String) { def this() = this("", "", "", "") }
|
2011-06-07 06:36:21 +05:30
|
|
|
@BeanInfo
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Person(name: String, age: Int, address: Address) { def this() = this("", 0, null) }
|
2011-06-07 06:36:21 +05:30
|
|
|
|
2014-03-07 13:20:01 +01:00
|
|
|
final case class Record(id: Int, person: Person)
|
2012-02-03 17:32:32 +01:00
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
class SimpleMessage(s: String) extends TestSerializable
|
2012-02-03 17:32:32 +01:00
|
|
|
|
|
|
|
|
class ExtendedSimpleMessage(s: String, i: Int) extends SimpleMessage(s)
|
|
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
trait AnotherInterface extends TestSerializable
|
2012-02-03 17:32:32 +01:00
|
|
|
|
|
|
|
|
class AnotherMessage extends AnotherInterface
|
|
|
|
|
|
|
|
|
|
class ExtendedAnotherMessage extends AnotherMessage
|
|
|
|
|
|
|
|
|
|
class PlainMessage
|
|
|
|
|
|
|
|
|
|
class ExtendedPlainMessage extends PlainMessage
|
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
class Both(s: String) extends SimpleMessage(s) with Serializable
|
|
|
|
|
|
|
|
|
|
trait A
|
|
|
|
|
trait B
|
|
|
|
|
class C extends B with A
|
|
|
|
|
class D extends A
|
|
|
|
|
class E extends D
|
|
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
val verifySerializabilityConf = """
|
|
|
|
|
akka {
|
|
|
|
|
actor {
|
|
|
|
|
serialize-messages = on
|
|
|
|
|
serialize-creators = on
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
"""
|
|
|
|
|
|
|
|
|
|
class FooActor extends Actor {
|
|
|
|
|
def receive = {
|
2014-01-16 15:16:35 +01:00
|
|
|
case s: String ⇒ sender() ! s
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
improve AbstractActor, #21717
* Receive class that wraps PartialFunction, to avoid
scary scala types
* move AbstractActorContext to AbstractActor.ActorContext
* converting docs, many, many UntypedActor
* removing UntypedActor docs
* add unit test for ReceiveBuilder
* MiMa filters
* consistent use of getContext(), self(), sender()
* rename cross references
* migration guide
* skip samples for now
* improve match type safetyi, add matchUnchecked
* the `? extends P` caused code like this to compile:
`match(String.class, (Integer i) -> {})`
* added matchUnchecked, since it can still be useful (um, convenient)
to be able to do:
`matchUnchecked(List.class, (List<String> list) -> {})`
* eleminate some scala.Option
* preRestart
* findChild
* ActorIdentity.getActorRef
2016-12-13 10:59:29 +01:00
|
|
|
class FooAbstractActor extends AbstractActor {
|
|
|
|
|
override def createReceive(): AbstractActor.Receive =
|
|
|
|
|
receiveBuilder().build()
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class NonSerializableActor(system: ActorSystem) extends Actor {
|
|
|
|
|
def receive = {
|
2014-01-16 15:16:35 +01:00
|
|
|
case s: String ⇒ sender() ! s
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def mostlyReferenceSystem: ActorSystem = {
|
|
|
|
|
val referenceConf = ConfigFactory.defaultReference()
|
2017-01-25 08:06:57 +01:00
|
|
|
// we are checking the old Java serialization formats here
|
|
|
|
|
val mostlyReferenceConf = ConfigFactory.parseString("""
|
|
|
|
|
akka.actor.enable-additional-serialization-bindings = off
|
|
|
|
|
""").withFallback(AkkaSpec.testConf.withFallback(referenceConf))
|
2013-01-25 14:24:23 +13:00
|
|
|
ActorSystem("SerializationSystem", mostlyReferenceConf)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
val systemMessageMultiSerializerConf = """
|
|
|
|
|
akka {
|
|
|
|
|
actor {
|
|
|
|
|
serializers {
|
2017-02-17 16:41:25 +01:00
|
|
|
test = "akka.serialization.NoopSerializer"
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
serialization-bindings {
|
2013-03-05 16:19:54 +01:00
|
|
|
"akka.dispatch.sysmsg.SystemMessage" = test
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
"""
|
|
|
|
|
|
|
|
|
|
val systemMessageClasses = List[Class[_]](
|
|
|
|
|
classOf[Create],
|
|
|
|
|
classOf[Recreate],
|
|
|
|
|
classOf[Suspend],
|
|
|
|
|
classOf[Resume],
|
|
|
|
|
classOf[Terminate],
|
|
|
|
|
classOf[Supervise],
|
|
|
|
|
classOf[Watch],
|
|
|
|
|
classOf[Unwatch],
|
2013-03-05 16:19:54 +01:00
|
|
|
classOf[Failed],
|
2013-01-25 14:24:23 +13:00
|
|
|
NoMessage.getClass)
|
2011-06-07 06:36:21 +05:30
|
|
|
}
|
|
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) {
|
|
|
|
|
import SerializationTests._
|
2011-06-07 06:36:21 +05:30
|
|
|
|
2011-11-24 18:53:18 +01:00
|
|
|
val ser = SerializationExtension(system)
|
2011-11-16 17:18:36 +01:00
|
|
|
import ser._
|
2011-06-07 06:36:21 +05:30
|
|
|
|
2011-11-22 13:04:10 +01:00
|
|
|
val addr = Address("120", "Monroe Street", "Santa Clara", "95050")
|
|
|
|
|
val person = Person("debasish ghosh", 25, Address("120", "Monroe Street", "Santa Clara", "95050"))
|
|
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
"Serialization" must {
|
|
|
|
|
|
2011-11-22 13:04:10 +01:00
|
|
|
"have correct bindings" in {
|
2015-01-16 11:09:59 +01:00
|
|
|
ser.bindings.collectFirst { case (c, s) if c == addr.getClass ⇒ s.getClass } should ===(Some(classOf[JavaSerializer]))
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.bindings.collectFirst { case (c, s) if c == classOf[PlainMessage] ⇒ s.getClass } should ===(Some(classOf[NoopSerializer]))
|
2011-11-22 13:04:10 +01:00
|
|
|
}
|
|
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
"serialize Address" in {
|
2012-09-06 03:17:51 +02:00
|
|
|
assert(deserialize(serialize(addr).get, classOf[Address]).get === addr)
|
2011-06-07 06:36:21 +05:30
|
|
|
}
|
|
|
|
|
|
2011-10-11 16:05:48 +02:00
|
|
|
"serialize Person" in {
|
2012-09-06 03:17:51 +02:00
|
|
|
assert(deserialize(serialize(person).get, classOf[Person]).get === person)
|
2011-06-07 06:36:21 +05:30
|
|
|
}
|
2011-10-11 16:05:48 +02:00
|
|
|
|
|
|
|
|
"serialize record with default serializer" in {
|
|
|
|
|
val r = Record(100, person)
|
2012-09-06 03:17:51 +02:00
|
|
|
assert(deserialize(serialize(r).get, classOf[Record]).get === r)
|
2011-06-07 06:36:21 +05:30
|
|
|
}
|
2011-11-01 11:20:02 +01:00
|
|
|
|
2011-12-08 16:07:03 +01:00
|
|
|
"not serialize ActorCell" in {
|
|
|
|
|
val a = system.actorOf(Props(new Actor {
|
|
|
|
|
def receive = {
|
|
|
|
|
case o: ObjectOutputStream ⇒
|
2012-09-06 03:17:51 +02:00
|
|
|
try o.writeObject(this) catch { case _: NotSerializableException ⇒ testActor ! "pass" }
|
2011-12-08 16:07:03 +01:00
|
|
|
}
|
|
|
|
|
}))
|
|
|
|
|
a ! new ObjectOutputStream(new ByteArrayOutputStream())
|
|
|
|
|
expectMsg("pass")
|
2011-12-14 00:06:36 +01:00
|
|
|
system.stop(a)
|
2011-12-08 16:07:03 +01:00
|
|
|
}
|
|
|
|
|
|
2011-11-01 11:20:02 +01:00
|
|
|
"serialize DeadLetterActorRef" in {
|
|
|
|
|
val outbuf = new ByteArrayOutputStream()
|
|
|
|
|
val out = new ObjectOutputStream(outbuf)
|
2011-11-30 15:16:20 +01:00
|
|
|
val a = ActorSystem("SerializeDeadLeterActorRef", AkkaSpec.testConf)
|
|
|
|
|
try {
|
|
|
|
|
out.writeObject(a.deadLetters)
|
|
|
|
|
out.flush()
|
|
|
|
|
out.close()
|
|
|
|
|
|
|
|
|
|
val in = new ObjectInputStream(new ByteArrayInputStream(outbuf.toByteArray))
|
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750
- PropertyMaster is the only place in Akka which calls
ClassLoader.getClass (apart from kernel, which might be special)
- all PropertyMaster methods (there are only three) take a ClassManifest
of what is to be constructed, and they verify that the obtained object
is actually compatible with the required type
Other stuff:
- noticed that I had forgotten to change to ExtendedActorSystem when
constructing Extensions by ExtensionKey (damn you, reflection!)
- moved Serializer.currentSystem into JavaSerializer, because that’s the
only one needing it (it’s only used in readResolve() methods)
- Serializers are constructed now with one-arg constructor taking
ExtendedActorSystem (if that exists, otherwise no-arg as before), to
allow JavaSerializer to do its magic; possibly necessary for others as
well
- Removed all Option[ClassLoader] signatures
- made it so that the ActorSystem will try context class loader, then
the class loader which loaded the class actually calling into
ActorSystem.apply, then the loader which loaded ActorSystemImpl
- for the second of the above I added a (reflectively accessed hopefully
safe) facility for getting caller Class[_] objects by using
sun.reflect.Reflection; this is optional an defaults to None, e.g. on
Android, which means that getting the caller’s classloader is done on
a best effort basis (there’s nothing we can do because a StackTrace
does not contain actual Class[_] objects).
- refactored DurableMailbox to contain the owner val and use that
instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
|
|
|
JavaSerializer.currentSystem.withValue(a.asInstanceOf[ActorSystemImpl]) {
|
2011-11-30 15:16:20 +01:00
|
|
|
val deadLetters = in.readObject().asInstanceOf[DeadLetterActorRef]
|
2015-01-16 11:09:59 +01:00
|
|
|
(deadLetters eq a.deadLetters) should ===(true)
|
2011-11-30 15:16:20 +01:00
|
|
|
}
|
|
|
|
|
} finally {
|
2013-05-02 17:12:36 +02:00
|
|
|
shutdown(a)
|
2011-11-01 11:20:02 +01:00
|
|
|
}
|
|
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
"resolve serializer by direct interface" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[SimpleMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-06 21:12:26 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"resolve serializer by interface implemented by super class" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[ExtendedSimpleMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-06 21:12:26 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"resolve serializer by indirect interface" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[AnotherMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-06 21:12:26 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"resolve serializer by indirect interface implemented by super class" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[ExtendedAnotherMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
"resolve serializer for message with binding" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[PlainMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
"resolve serializer for message extending class with with binding" in {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[ExtendedPlainMessage]).getClass should ===(classOf[NoopSerializer])
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-07 15:51:41 +01:00
|
|
|
"give warning for message with several bindings" in {
|
|
|
|
|
EventFilter.warning(start = "Multiple serializers found", occurrences = 1) intercept {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[Both]).getClass should (be(classOf[NoopSerializer]) or be(classOf[JavaSerializer]))
|
2012-02-07 15:51:41 +01:00
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-07 15:51:41 +01:00
|
|
|
"resolve serializer in the order of the bindings" in {
|
2015-01-16 11:09:59 +01:00
|
|
|
ser.serializerFor(classOf[A]).getClass should ===(classOf[JavaSerializer])
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[B]).getClass should ===(classOf[NoopSerializer])
|
2012-02-07 15:51:41 +01:00
|
|
|
EventFilter.warning(start = "Multiple serializers found", occurrences = 1) intercept {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[C]).getClass should (be(classOf[NoopSerializer]) or be(classOf[JavaSerializer]))
|
2012-02-07 15:11:16 +01:00
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
"resolve serializer in the order of most specific binding first" in {
|
2015-01-16 11:09:59 +01:00
|
|
|
ser.serializerFor(classOf[A]).getClass should ===(classOf[JavaSerializer])
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(classOf[D]).getClass should ===(classOf[NoopSerializer])
|
|
|
|
|
ser.serializerFor(classOf[E]).getClass should ===(classOf[NoopSerializer])
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-02-06 21:12:26 +01:00
|
|
|
"throw java.io.NotSerializableException when no binding" in {
|
|
|
|
|
intercept[java.io.NotSerializableException] {
|
|
|
|
|
ser.serializerFor(classOf[Actor])
|
|
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
|
|
|
|
|
2012-09-27 00:59:33 +02:00
|
|
|
"use ByteArraySerializer for byte arrays" in {
|
2012-09-27 01:10:07 +02:00
|
|
|
val byteSerializer = ser.serializerFor(classOf[Array[Byte]])
|
2013-12-17 14:25:56 +01:00
|
|
|
byteSerializer.getClass should be theSameInstanceAs classOf[ByteArraySerializer]
|
2012-09-27 01:10:07 +02:00
|
|
|
|
2012-09-27 11:34:04 +02:00
|
|
|
for (a ← Seq("foo".getBytes("UTF-8"), null: Array[Byte], Array[Byte]()))
|
2013-12-17 14:25:56 +01:00
|
|
|
byteSerializer.fromBinary(byteSerializer.toBinary(a)) should be theSameInstanceAs a
|
2012-09-27 01:10:07 +02:00
|
|
|
|
|
|
|
|
intercept[IllegalArgumentException] {
|
|
|
|
|
byteSerializer.toBinary("pigdog")
|
2016-05-26 11:58:13 +02:00
|
|
|
}.getMessage should ===(s"${classOf[ByteArraySerializer].getName} only serializes byte arrays, not [java.lang.String]")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"support ByteBuffer serialization for byte arrays" in {
|
|
|
|
|
val byteSerializer = ser.serializerFor(classOf[Array[Byte]]).asInstanceOf[ByteBufferSerializer]
|
|
|
|
|
|
|
|
|
|
val byteBuffer = ByteBuffer.allocate(128).order(ByteOrder.LITTLE_ENDIAN)
|
|
|
|
|
val str = "abcdef"
|
|
|
|
|
val payload = str.getBytes("UTF-8")
|
|
|
|
|
byteSerializer.toBinary(payload, byteBuffer)
|
|
|
|
|
byteBuffer.position() should ===(payload.length)
|
|
|
|
|
byteBuffer.flip()
|
|
|
|
|
val deserialized = byteSerializer.fromBinary(byteBuffer, "").asInstanceOf[Array[Byte]]
|
|
|
|
|
byteBuffer.remaining() should ===(0)
|
|
|
|
|
new String(deserialized, "UTF-8") should ===(str)
|
|
|
|
|
|
|
|
|
|
intercept[IllegalArgumentException] {
|
|
|
|
|
byteSerializer.toBinary("pigdog", byteBuffer)
|
|
|
|
|
}.getMessage should ===(s"${classOf[ByteArraySerializer].getName} only serializes byte arrays, not [java.lang.String]")
|
2012-09-27 00:59:33 +02:00
|
|
|
}
|
2011-06-07 06:36:21 +05:30
|
|
|
}
|
|
|
|
|
}
|
2011-12-27 17:30:05 +01:00
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
class VerifySerializabilitySpec extends AkkaSpec(SerializationTests.verifySerializabilityConf) {
|
|
|
|
|
import SerializationTests._
|
2011-12-27 20:07:21 +01:00
|
|
|
implicit val timeout = Timeout(5 seconds)
|
2011-12-27 17:30:05 +01:00
|
|
|
|
2011-12-27 20:07:21 +01:00
|
|
|
"verify config" in {
|
2015-01-16 11:09:59 +01:00
|
|
|
system.settings.SerializeAllCreators should ===(true)
|
|
|
|
|
system.settings.SerializeAllMessages should ===(true)
|
2011-12-27 20:07:21 +01:00
|
|
|
}
|
2011-12-27 17:30:05 +01:00
|
|
|
|
2011-12-27 20:07:21 +01:00
|
|
|
"verify creators" in {
|
2011-12-27 17:30:05 +01:00
|
|
|
val a = system.actorOf(Props[FooActor])
|
2011-12-27 20:07:21 +01:00
|
|
|
system stop a
|
2012-02-06 14:19:59 +01:00
|
|
|
|
improve AbstractActor, #21717
* Receive class that wraps PartialFunction, to avoid
scary scala types
* move AbstractActorContext to AbstractActor.ActorContext
* converting docs, many, many UntypedActor
* removing UntypedActor docs
* add unit test for ReceiveBuilder
* MiMa filters
* consistent use of getContext(), self(), sender()
* rename cross references
* migration guide
* skip samples for now
* improve match type safetyi, add matchUnchecked
* the `? extends P` caused code like this to compile:
`match(String.class, (Integer i) -> {})`
* added matchUnchecked, since it can still be useful (um, convenient)
to be able to do:
`matchUnchecked(List.class, (List<String> list) -> {})`
* eleminate some scala.Option
* preRestart
* findChild
* ActorIdentity.getActorRef
2016-12-13 10:59:29 +01:00
|
|
|
val b = system.actorOf(Props(new FooAbstractActor))
|
2012-02-06 14:19:59 +01:00
|
|
|
system stop b
|
|
|
|
|
|
2013-05-29 16:13:10 +02:00
|
|
|
intercept[IllegalArgumentException] {
|
2012-02-06 14:19:59 +01:00
|
|
|
val d = system.actorOf(Props(new NonSerializableActor(system)))
|
|
|
|
|
}
|
|
|
|
|
|
2011-12-27 20:07:21 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"verify messages" in {
|
|
|
|
|
val a = system.actorOf(Props[FooActor])
|
2015-01-16 11:09:59 +01:00
|
|
|
Await.result(a ? "pigdog", timeout.duration) should ===("pigdog")
|
2012-02-06 14:19:59 +01:00
|
|
|
|
2012-08-15 21:46:05 +02:00
|
|
|
EventFilter[NotSerializableException](occurrences = 1) intercept {
|
|
|
|
|
a ! (new AnyRef)
|
2011-12-27 17:30:05 +01:00
|
|
|
}
|
2011-12-27 20:07:21 +01:00
|
|
|
system stop a
|
2011-12-27 17:30:05 +01:00
|
|
|
}
|
|
|
|
|
}
|
2012-02-03 17:32:32 +01:00
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
class ReferenceSerializationSpec extends AkkaSpec(SerializationTests.mostlyReferenceSystem) {
|
|
|
|
|
import SerializationTests._
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
|
|
|
|
def serializerMustBe(toSerialize: Class[_], expectedSerializer: Class[_]) =
|
2015-01-16 11:09:59 +01:00
|
|
|
ser.serializerFor(toSerialize).getClass should ===(expectedSerializer)
|
2013-01-25 14:24:23 +13:00
|
|
|
|
|
|
|
|
"Serialization settings from reference.conf" must {
|
|
|
|
|
|
|
|
|
|
"declare Serializable classes to be use JavaSerializer" in {
|
|
|
|
|
serializerMustBe(classOf[Serializable], classOf[JavaSerializer])
|
|
|
|
|
serializerMustBe(classOf[String], classOf[JavaSerializer])
|
|
|
|
|
for (smc ← systemMessageClasses) {
|
|
|
|
|
serializerMustBe(smc, classOf[JavaSerializer])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"declare Array[Byte] to use ByteArraySerializer" in {
|
|
|
|
|
serializerMustBe(classOf[Array[Byte]], classOf[ByteArraySerializer])
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"not support serialization for other classes" in {
|
|
|
|
|
intercept[NotSerializableException] { ser.serializerFor(classOf[Object]) }
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-20 15:27:32 +01:00
|
|
|
"serialize function with JavaSerializer" in {
|
|
|
|
|
val f = (i: Int) ⇒ i + 1
|
|
|
|
|
val serializer = ser.serializerFor(f.getClass)
|
|
|
|
|
serializer.getClass should ===(classOf[JavaSerializer])
|
|
|
|
|
val bytes = ser.serialize(f).get
|
|
|
|
|
val f2 = ser.deserialize(bytes, serializer.identifier, "").get.asInstanceOf[Function1[Int, Int]]
|
|
|
|
|
f2(3) should ===(4)
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class SerializationCompatibilitySpec extends AkkaSpec(SerializationTests.mostlyReferenceSystem) {
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
|
|
|
|
|
|
|
|
|
"Cross-version serialization compatibility" must {
|
2016-10-31 15:40:47 +01:00
|
|
|
def verify(obj: SystemMessage, asExpected: String): Unit = {
|
|
|
|
|
val bytes = javax.xml.bind.DatatypeConverter.parseHexBinary(asExpected)
|
|
|
|
|
val stream = new ObjectInputStream(new ByteArrayInputStream(bytes))
|
|
|
|
|
val read = stream.readObject()
|
|
|
|
|
read should ===(obj)
|
|
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
|
|
|
|
|
"be preserved for the Create SystemMessage" in {
|
2013-04-30 09:06:13 +02:00
|
|
|
// Using null as the cause to avoid a large serialized message and JDK differences
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Create(Some(null)),
|
2014-09-05 13:14:20 +02:00
|
|
|
if (scala.util.Properties.versionNumberString.startsWith("2.10.")) {
|
|
|
|
|
"aced00057372001b616b6b612e64697370617463682e7379736d73672e4372656174650000000000" +
|
|
|
|
|
"0000010200014c00076661696c75726574000e4c7363616c612f4f7074696f6e3b78707372000a73" +
|
|
|
|
|
"63616c612e536f6d65e2a09f87fc0836ae0200014c0001787400124c6a6176612f6c616e672f4f62" +
|
|
|
|
|
"6a6563743b7872000c7363616c612e4f7074696f6ee36024a8328a45e9020000787070"
|
|
|
|
|
} else {
|
|
|
|
|
"aced00057372001b616b6b612e64697370617463682e7379736d73672e4372656174650000000000" +
|
|
|
|
|
"0000010200014c00076661696c75726574000e4c7363616c612f4f7074696f6e3b78707372000a73" +
|
|
|
|
|
"63616c612e536f6d651122f2695ea18b740200014c0001787400124c6a6176612f6c616e672f4f62" +
|
|
|
|
|
"6a6563743b7872000c7363616c612e4f7074696f6efe6937fddb0e6674020000787070"
|
|
|
|
|
})
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Recreate SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Recreate(null),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001d616b6b612e64697370617463682e7379736d73672e5265637265617465000000" +
|
|
|
|
|
"00000000010200014c000563617573657400154c6a6176612f6c616e672f5468726f7761626c653b" +
|
|
|
|
|
"787070")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Suspend SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Suspend(),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001c616b6b612e64697370617463682e7379736d73672e53757370656e6400000000" +
|
|
|
|
|
"000000010200007870")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Resume SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Resume(null),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001b616b6b612e64697370617463682e7379736d73672e526573756d650000000000" +
|
|
|
|
|
"0000010200014c000f63617573656442794661696c7572657400154c6a6176612f6c616e672f5468" +
|
|
|
|
|
"726f7761626c653b787070")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Terminate SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Terminate(),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001e616b6b612e64697370617463682e7379736d73672e5465726d696e6174650000" +
|
|
|
|
|
"0000000000010200007870")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Supervise SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Supervise(null, true),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001e616b6b612e64697370617463682e7379736d73672e5375706572766973650000" +
|
|
|
|
|
"0000000000010200025a00056173796e634c00056368696c647400154c616b6b612f6163746f722f" +
|
2013-05-02 21:10:33 +02:00
|
|
|
"4163746f725265663b78700170")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Watch SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Watch(null, null),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001a616b6b612e64697370617463682e7379736d73672e5761746368000000000000" +
|
|
|
|
|
"00010200024c00077761746368656574001d4c616b6b612f6163746f722f496e7465726e616c4163" +
|
2013-05-02 21:10:33 +02:00
|
|
|
"746f725265663b4c00077761746368657271007e000178707070")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the Unwatch SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Unwatch(null, null),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001c616b6b612e64697370617463682e7379736d73672e556e776174636800000000" +
|
|
|
|
|
"000000010200024c0007776174636865657400154c616b6b612f6163746f722f4163746f72526566" +
|
2013-05-02 21:10:33 +02:00
|
|
|
"3b4c00077761746368657271007e000178707070")
|
2013-01-28 17:56:29 +01:00
|
|
|
}
|
2013-01-28 23:20:43 +01:00
|
|
|
"be preserved for the NoMessage SystemMessage" in {
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
NoMessage,
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001f616b6b612e64697370617463682e7379736d73672e4e6f4d6573736167652400" +
|
|
|
|
|
"000000000000010200007870")
|
2013-03-05 16:19:54 +01:00
|
|
|
}
|
|
|
|
|
"be preserved for the Failed SystemMessage" in {
|
2013-04-30 09:06:13 +02:00
|
|
|
// Using null as the cause to avoid a large serialized message and JDK differences
|
2016-06-02 14:06:57 +02:00
|
|
|
verify(
|
|
|
|
|
Failed(null, cause = null, uid = 0),
|
2013-04-24 13:29:11 +02:00
|
|
|
"aced00057372001b616b6b612e64697370617463682e7379736d73672e4661696c65640000000000" +
|
|
|
|
|
"0000010200034900037569644c000563617573657400154c6a6176612f6c616e672f5468726f7761" +
|
|
|
|
|
"626c653b4c00056368696c647400154c616b6b612f6163746f722f4163746f725265663b78700000" +
|
2013-05-02 21:10:33 +02:00
|
|
|
"00007070")
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
2016-12-20 15:27:32 +01:00
|
|
|
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class OverriddenSystemMessageSerializationSpec extends AkkaSpec(SerializationTests.systemMessageMultiSerializerConf) {
|
|
|
|
|
import SerializationTests._
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
|
|
|
|
|
|
|
|
|
"Overridden SystemMessage serialization" must {
|
|
|
|
|
|
|
|
|
|
"resolve to a single serializer" in {
|
|
|
|
|
EventFilter.warning(start = "Multiple serializers found", occurrences = 0) intercept {
|
|
|
|
|
for (smc ← systemMessageClasses) {
|
2017-02-17 16:41:25 +01:00
|
|
|
ser.serializerFor(smc).getClass should ===(classOf[NoopSerializer])
|
2013-01-25 14:24:23 +13:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-24 16:45:51 +02:00
|
|
|
class DefaultSerializationWarningSpec extends AkkaSpec(
|
|
|
|
|
ConfigFactory.parseString("akka.actor.warn-about-java-serializer-usage = on")) {
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
2016-07-06 18:48:15 +09:00
|
|
|
val messagePrefix = "Using the default Java serializer for class"
|
2015-09-24 16:45:51 +02:00
|
|
|
|
|
|
|
|
"Using the default Java serializer" must {
|
|
|
|
|
|
2016-03-07 20:51:26 +01:00
|
|
|
"log a warning when serializing classes outside of java.lang package" in {
|
2016-07-06 18:48:15 +09:00
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 1) intercept {
|
2016-03-07 20:51:26 +01:00
|
|
|
ser.serializerFor(classOf[java.math.BigDecimal])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"not log warning when serializing classes from java.lang package" in {
|
2016-07-06 18:48:15 +09:00
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 0) intercept {
|
2016-03-07 20:51:26 +01:00
|
|
|
ser.serializerFor(classOf[java.lang.String])
|
2015-09-24 16:45:51 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2016-07-06 18:48:15 +09:00
|
|
|
class NoVerificationWarningSpec extends AkkaSpec(
|
|
|
|
|
ConfigFactory.parseString(
|
|
|
|
|
"akka.actor.warn-about-java-serializer-usage = on\n" +
|
|
|
|
|
"akka.actor.warn-on-no-serialization-verification = on")) {
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
|
|
|
|
val messagePrefix = "Using the default Java serializer for class"
|
|
|
|
|
|
|
|
|
|
"When warn-on-no-serialization-verification = on, using the default Java serializer" must {
|
|
|
|
|
|
|
|
|
|
"log a warning on classes without extending NoSerializationVerificationNeeded" in {
|
|
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 1) intercept {
|
|
|
|
|
ser.serializerFor(classOf[java.math.BigDecimal])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"still log warning on classes extending NoSerializationVerificationNeeded" in {
|
|
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 1) intercept {
|
|
|
|
|
ser.serializerFor(classOf[NoVerification])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class NoVerificationWarningOffSpec extends AkkaSpec(
|
|
|
|
|
ConfigFactory.parseString(
|
|
|
|
|
"akka.actor.warn-about-java-serializer-usage = on\n" +
|
|
|
|
|
"akka.actor.warn-on-no-serialization-verification = off")) {
|
|
|
|
|
|
|
|
|
|
val ser = SerializationExtension(system)
|
|
|
|
|
val messagePrefix = "Using the default Java serializer for class"
|
|
|
|
|
|
|
|
|
|
"When warn-on-no-serialization-verification = off, using the default Java serializer" must {
|
|
|
|
|
|
|
|
|
|
"log a warning on classes without extending NoSerializationVerificationNeeded" in {
|
|
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 1) intercept {
|
|
|
|
|
ser.serializerFor(classOf[java.math.BigDecimal])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"not log warning on classes extending NoSerializationVerificationNeeded" in {
|
|
|
|
|
EventFilter.warning(start = messagePrefix, occurrences = 0) intercept {
|
|
|
|
|
ser.serializerFor(classOf[NoVerification])
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-03-28 23:45:48 +01:00
|
|
|
protected[akka] trait TestSerializable
|
2012-02-03 17:32:32 +01:00
|
|
|
|
2017-02-17 16:41:25 +01:00
|
|
|
protected[akka] class NoopSerializer extends Serializer {
|
2012-02-03 17:32:32 +01:00
|
|
|
def includeManifest: Boolean = false
|
|
|
|
|
|
|
|
|
|
def identifier = 9999
|
|
|
|
|
|
|
|
|
|
def toBinary(o: AnyRef): Array[Byte] = {
|
|
|
|
|
Array.empty[Byte]
|
|
|
|
|
}
|
|
|
|
|
|
Bye-bye ReflectiveAccess, introducing PropertyMaster, see #1750
- PropertyMaster is the only place in Akka which calls
ClassLoader.getClass (apart from kernel, which might be special)
- all PropertyMaster methods (there are only three) take a ClassManifest
of what is to be constructed, and they verify that the obtained object
is actually compatible with the required type
Other stuff:
- noticed that I had forgotten to change to ExtendedActorSystem when
constructing Extensions by ExtensionKey (damn you, reflection!)
- moved Serializer.currentSystem into JavaSerializer, because that’s the
only one needing it (it’s only used in readResolve() methods)
- Serializers are constructed now with one-arg constructor taking
ExtendedActorSystem (if that exists, otherwise no-arg as before), to
allow JavaSerializer to do its magic; possibly necessary for others as
well
- Removed all Option[ClassLoader] signatures
- made it so that the ActorSystem will try context class loader, then
the class loader which loaded the class actually calling into
ActorSystem.apply, then the loader which loaded ActorSystemImpl
- for the second of the above I added a (reflectively accessed hopefully
safe) facility for getting caller Class[_] objects by using
sun.reflect.Reflection; this is optional an defaults to None, e.g. on
Android, which means that getting the caller’s classloader is done on
a best effort basis (there’s nothing we can do because a StackTrace
does not contain actual Class[_] objects).
- refactored DurableMailbox to contain the owner val and use that
instead of declaring that in all subclasses
2012-02-09 11:56:43 +01:00
|
|
|
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null
|
2012-02-03 17:32:32 +01:00
|
|
|
}
|
2013-01-25 14:24:23 +13:00
|
|
|
|
|
|
|
|
@SerialVersionUID(1)
|
2014-03-07 13:20:01 +01:00
|
|
|
protected[akka] final case class FakeThrowable(msg: String) extends Throwable(msg) with Serializable {
|
2013-01-25 14:24:23 +13:00
|
|
|
override def fillInStackTrace = null
|
|
|
|
|
}
|