throw java.io.NotSerializableException during deserialization (#22821)

* throw java.io.NotSerializableException during deserialization

* formatting fix

* add annotation to all fromBinary methods

* remove annotation

* add annotation to remaining methods in Serialization.scala
This commit is contained in:
Nafer Sanabria 2017-08-08 06:15:18 -05:00 committed by Patrik Nordwall
parent e4b8256009
commit ad1ffeda2b
6 changed files with 37 additions and 12 deletions

View file

@ -179,6 +179,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* using the optional type hint to the Serializer. * using the optional type hint to the Serializer.
* Returns either the resulting object or throws an exception if deserialization fails. * Returns either the resulting object or throws an exception if deserialization fails.
*/ */
@throws(classOf[NotSerializableException])
def deserializeByteBuffer(buf: ByteBuffer, serializerId: Int, manifest: String): AnyRef = { def deserializeByteBuffer(buf: ByteBuffer, serializerId: Int, manifest: String): AnyRef = {
val serializer = try getSerializerById(serializerId) catch { val serializer = try getSerializerById(serializerId) catch {
case _: NoSuchElementException throw new NotSerializableException( case _: NoSuchElementException throw new NotSerializableException(
@ -220,6 +221,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* *
* Throws java.io.NotSerializableException if no `serialization-bindings` is configured for the class. * Throws java.io.NotSerializableException if no `serialization-bindings` is configured for the class.
*/ */
@throws(classOf[NotSerializableException])
def serializerFor(clazz: Class[_]): Serializer = def serializerFor(clazz: Class[_]): Serializer =
serializerMap.get(clazz) match { serializerMap.get(clazz) match {
case null // bindings are ordered from most specific to least specific case null // bindings are ordered from most specific to least specific

View file

@ -4,7 +4,7 @@ package akka.serialization
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com> * Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
*/ */
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, NotSerializableException, ObjectOutputStream }
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.concurrent.Callable import java.util.concurrent.Callable
@ -57,6 +57,7 @@ trait Serializer {
* Produces an object from an array of bytes, with an optional type-hint; * Produces an object from an array of bytes, with an optional type-hint;
* the class should be loaded using ActorSystem.dynamicAccess. * the class should be loaded using ActorSystem.dynamicAccess.
*/ */
@throws(classOf[NotSerializableException])
def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef
/** /**
@ -67,6 +68,7 @@ trait Serializer {
/** /**
* Java API: deserialize with type hint * Java API: deserialize with type hint
*/ */
@throws(classOf[NotSerializableException])
final def fromBinary(bytes: Array[Byte], clazz: Class[_]): AnyRef = fromBinary(bytes, Option(clazz)) final def fromBinary(bytes: Array[Byte], clazz: Class[_]): AnyRef = fromBinary(bytes, Option(clazz))
} }
@ -135,6 +137,7 @@ abstract class SerializerWithStringManifest extends Serializer {
* and message is dropped. Other exceptions will tear down the TCP connection * and message is dropped. Other exceptions will tear down the TCP connection
* because it can be an indication of corrupt bytes from the underlying transport. * because it can be an indication of corrupt bytes from the underlying transport.
*/ */
@throws(classOf[NotSerializableException])
def fromBinary(bytes: Array[Byte], manifest: String): AnyRef def fromBinary(bytes: Array[Byte], manifest: String): AnyRef
final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = { final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = {
@ -194,6 +197,7 @@ trait ByteBufferSerializer {
* Produces an object from a `ByteBuffer`, with an optional type-hint; * Produces an object from a `ByteBuffer`, with an optional type-hint;
* the class should be loaded using ActorSystem.dynamicAccess. * the class should be loaded using ActorSystem.dynamicAccess.
*/ */
@throws(classOf[NotSerializableException])
def fromBinary(buf: ByteBuffer, manifest: String): AnyRef def fromBinary(buf: ByteBuffer, manifest: String): AnyRef
} }
@ -257,6 +261,8 @@ object BaseSerializer {
* the JSerializer (also possible with empty constructor). * the JSerializer (also possible with empty constructor).
*/ */
abstract class JSerializer extends Serializer { abstract class JSerializer extends Serializer {
@throws(classOf[NotSerializableException])
final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef = final def fromBinary(bytes: Array[Byte], manifest: Option[Class[_]]): AnyRef =
fromBinaryJava(bytes, manifest.orNull) fromBinaryJava(bytes, manifest.orNull)
@ -315,6 +321,7 @@ class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
bos.toByteArray bos.toByteArray
} }
@throws(classOf[NotSerializableException])
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes)) val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes))
val obj = JavaSerializer.currentSystem.withValue(system) { in.readObject } val obj = JavaSerializer.currentSystem.withValue(system) { in.readObject }
@ -344,11 +351,13 @@ final case class DisabledJavaSerializer(system: ExtendedActorSystem) extends Ser
throw IllegalSerialization throw IllegalSerialization
} }
@throws(classOf[NotSerializableException])
override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = { override def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
log.warning(LogMarker.Security, "Incoming message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set!") log.warning(LogMarker.Security, "Incoming message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set!")
throw IllegalDeserialization throw IllegalDeserialization
} }
@throws(classOf[NotSerializableException])
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
// we don't capture the manifest or mention it in the log as the default setting for includeManifest is set to false. // we don't capture the manifest or mention it in the log as the default setting for includeManifest is set to false.
log.warning(LogMarker.Security, "Incoming message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set!") log.warning(LogMarker.Security, "Incoming message attempted to use Java Serialization even though `akka.actor.allow-java-serialization = off` was set!")
@ -376,6 +385,7 @@ class NullSerializer extends Serializer {
def includeManifest: Boolean = false def includeManifest: Boolean = false
def identifier = 0 def identifier = 0
def toBinary(o: AnyRef): Array[Byte] = nullAsBytes def toBinary(o: AnyRef): Array[Byte] = nullAsBytes
@throws(classOf[NotSerializableException])
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null
} }
@ -392,6 +402,8 @@ class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerialize
case other throw new IllegalArgumentException( case other throw new IllegalArgumentException(
s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]") s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]")
} }
@throws(classOf[NotSerializableException])
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = bytes def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = bytes
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = override def toBinary(o: AnyRef, buf: ByteBuffer): Unit =
@ -402,6 +414,7 @@ class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerialize
s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]") s"${getClass.getName} only serializes byte arrays, not [${other.getClass.getName}]")
} }
@throws(classOf[NotSerializableException])
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
val bytes = new Array[Byte](buf.remaining()) val bytes = new Array[Byte](buf.remaining())
buf.get(bytes) buf.get(bytes)

View file

@ -6,6 +6,8 @@ package jdocs.persistence;
import docs.persistence.ExampleJsonMarshaller; import docs.persistence.ExampleJsonMarshaller;
import docs.persistence.proto.FlightAppModels; import docs.persistence.proto.FlightAppModels;
import java.io.NotSerializableException;
import java.nio.charset.Charset; import java.nio.charset.Charset;
import spray.json.JsObject; import spray.json.JsObject;
@ -72,7 +74,7 @@ public class PersistenceSchemaEvolutionDocTest {
return o.getClass().getName(); return o.getClass().getName();
} }
@Override public Object fromBinary(byte[] bytes, String manifest) { @Override public Object fromBinary(byte[] bytes, String manifest) throws NotSerializableException{
if (seatReservedManifest.equals(manifest)) { if (seatReservedManifest.equals(manifest)) {
// use generated protobuf serializer // use generated protobuf serializer
try { try {
@ -81,7 +83,7 @@ public class PersistenceSchemaEvolutionDocTest {
throw new IllegalArgumentException(e.getMessage()); throw new IllegalArgumentException(e.getMessage());
} }
} else { } else {
throw new IllegalArgumentException("Unable to handle manifest: " + manifest); throw new NotSerializableException("Unable to handle manifest: " + manifest);
} }
} }
@ -208,13 +210,13 @@ public class PersistenceSchemaEvolutionDocTest {
} }
// deserialize the object, using the manifest to indicate which logic to apply // deserialize the object, using the manifest to indicate which logic to apply
@Override public Object fromBinary(byte[] bytes, String manifest) { @Override public Object fromBinary(byte[] bytes, String manifest) throws NotSerializableException {
if (personManifest.equals(manifest)) { if (personManifest.equals(manifest)) {
String nameAndSurname = new String(bytes, utf8); String nameAndSurname = new String(bytes, utf8);
String[] parts = nameAndSurname.split("[|]"); String[] parts = nameAndSurname.split("[|]");
return new Person(parts[0], parts[1]); return new Person(parts[0], parts[1]);
} else { } else {
throw new IllegalArgumentException( throw new NotSerializableException(
"Unable to deserialize from bytes, manifest was: " + manifest + "Unable to deserialize from bytes, manifest was: " + manifest +
"! Bytes length: " + bytes.length); "! Bytes length: " + bytes.length);
} }
@ -412,12 +414,12 @@ public class PersistenceSchemaEvolutionDocTest {
} }
} }
@Override public Object fromBinary(byte[] bytes, String manifest) { @Override public Object fromBinary(byte[] bytes, String manifest) throws NotSerializableException {
if (oldPayloadClassName.equals(manifest)) if (oldPayloadClassName.equals(manifest))
return new SamplePayload(new String(bytes, utf8)); return new SamplePayload(new String(bytes, utf8));
else if (myPayloadClassName.equals(manifest)) else if (myPayloadClassName.equals(manifest))
return new SamplePayload(new String(bytes, utf8)); return new SamplePayload(new String(bytes, utf8));
else throw new IllegalArgumentException("unexpected manifest [" + manifest + "]"); else throw new NotSerializableException("unexpected manifest [" + manifest + "]");
} }
} }
//#string-serializer-handle-rename //#string-serializer-handle-rename

View file

@ -4,8 +4,11 @@
package docs.persistence package docs.persistence
import java.io.NotSerializableException
import scala.language.reflectiveCalls import scala.language.reflectiveCalls
import java.nio.charset.Charset import java.nio.charset.Charset
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.persistence.journal.{ EventAdapter, EventSeq } import akka.persistence.journal.{ EventAdapter, EventSeq }
import akka.serialization.{ SerializationExtension, SerializerWithStringManifest } import akka.serialization.{ SerializationExtension, SerializerWithStringManifest }
@ -13,6 +16,7 @@ import akka.testkit.TestKit
import com.typesafe.config._ import com.typesafe.config._
import org.scalatest.WordSpec import org.scalatest.WordSpec
import spray.json.JsObject import spray.json.JsObject
import scala.concurrent.duration._ import scala.concurrent.duration._
import docs.persistence.proto.FlightAppModels import docs.persistence.proto.FlightAppModels
@ -82,7 +86,7 @@ class ProtobufReadOptional {
// use generated protobuf serializer // use generated protobuf serializer
seatReserved(FlightAppModels.SeatReserved.parseFrom(bytes)) seatReserved(FlightAppModels.SeatReserved.parseFrom(bytes))
case _ => case _ =>
throw new IllegalArgumentException("Unable to handle manifest: " + manifest) throw new NotSerializableException("Unable to handle manifest: " + manifest)
} }
override def toBinary(o: AnyRef): Array[Byte] = o match { override def toBinary(o: AnyRef): Array[Byte] = o match {
@ -197,7 +201,7 @@ object SimplestCustomSerializer {
val nameAndSurname = new String(bytes, Utf8) val nameAndSurname = new String(bytes, Utf8)
val Array(name, surname) = nameAndSurname.split("[|]") val Array(name, surname) = nameAndSurname.split("[|]")
Person(name, surname) Person(name, surname)
case _ => throw new IllegalArgumentException( case _ => throw new NotSerializableException(
s"Unable to deserialize from bytes, manifest was: $manifest! Bytes length: " + s"Unable to deserialize from bytes, manifest was: $manifest! Bytes length: " +
bytes.length) bytes.length)
} }
@ -317,7 +321,7 @@ class RenamedEventAwareSerializer extends SerializerWithStringManifest {
manifest match { manifest match {
case OldPayloadClassName => SamplePayload(new String(bytes, Utf8)) case OldPayloadClassName => SamplePayload(new String(bytes, Utf8))
case MyPayloadClassName => SamplePayload(new String(bytes, Utf8)) case MyPayloadClassName => SamplePayload(new String(bytes, Utf8))
case other => throw new Exception(s"unexpected manifest [$other]") case other => throw new NotSerializableException(s"unexpected manifest [$other]")
} }
} }
//#string-serializer-handle-rename //#string-serializer-handle-rename

View file

@ -3,6 +3,8 @@
*/ */
package akka.remote.serialization package akka.remote.serialization
import java.io.NotSerializableException
import akka.actor.{ ActorRef, Address, ExtendedActorSystem } import akka.actor.{ ActorRef, Address, ExtendedActorSystem }
import akka.protobuf.MessageLite import akka.protobuf.MessageLite
import akka.remote.RemoteWatcher.ArteryHeartbeatRsp import akka.remote.RemoteWatcher.ArteryHeartbeatRsp
@ -91,7 +93,7 @@ private[akka] final class ArteryMessageSerializer(val system: ExtendedActorSyste
case ClassManifestCompressionAdvertisementAckManifest deserializeCompressionTableAdvertisementAck(bytes, ClassManifestCompressionAdvertisementAck) case ClassManifestCompressionAdvertisementAckManifest deserializeCompressionTableAdvertisementAck(bytes, ClassManifestCompressionAdvertisementAck)
case ArteryHeartbeatManifest RemoteWatcher.ArteryHeartbeat case ArteryHeartbeatManifest RemoteWatcher.ArteryHeartbeat
case ArteryHeartbeatRspManifest deserializeArteryHeartbeatRsp(bytes, ArteryHeartbeatRsp) case ArteryHeartbeatRspManifest deserializeArteryHeartbeatRsp(bytes, ArteryHeartbeatRsp)
case _ throw new IllegalArgumentException(s"Manifest '$manifest' not defined for ArteryControlMessageSerializer (serializer id $identifier)") case _ throw new NotSerializableException(s"Manifest '$manifest' not defined for ArteryControlMessageSerializer (serializer id $identifier)")
} }
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._

View file

@ -4,6 +4,8 @@
package akka.remote.serialization package akka.remote.serialization
import java.io.NotSerializableException
import akka.actor._ import akka.actor._
import akka.remote.{ RemoteWatcher, UniqueAddress } import akka.remote.{ RemoteWatcher, UniqueAddress }
import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp } import akka.remote.artery.OutboundHandshake.{ HandshakeReq, HandshakeRsp }
@ -55,7 +57,7 @@ class ArteryMessageSerializerSpec extends AkkaSpec {
} }
"reject deserialization with invalid manifest" in { "reject deserialization with invalid manifest" in {
intercept[IllegalArgumentException] { intercept[NotSerializableException] {
val serializer = new ArteryMessageSerializer(system.asInstanceOf[ExtendedActorSystem]) val serializer = new ArteryMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
serializer.fromBinary(Array.empty[Byte], "INVALID") serializer.fromBinary(Array.empty[Byte], "INVALID")
} }