catch NotSerializableException from deserialization, #20641
* to be able to introduce new messages and still support rolling upgrades, i.e. a cluster of mixed versions * note that it's only catching NotSerializableException, which we already use for unknown serializer ids and class manifests * note that it is not catching for system messages, since that could result in infinite resending
This commit is contained in:
parent
1a12e950ff
commit
e494ec2183
16 changed files with 64 additions and 15 deletions
|
|
@ -120,6 +120,15 @@ abstract class SerializerWithStringManifest extends 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.
|
||||||
|
*
|
||||||
|
* It's recommended to throw `java.io.NotSerializableException` in `fromBinary`
|
||||||
|
* if the manifest is unknown. This makes it possible to introduce new message
|
||||||
|
* types and send them to nodes that don't know about them. This is typically
|
||||||
|
* needed when performing rolling upgrades, i.e. running a cluster with mixed
|
||||||
|
* versions for while. `NotSerializableException` is treated as a transient
|
||||||
|
* problem in the TCP based remoting layer. The problem will be logged
|
||||||
|
* and message is dropped. Other exceptions will tear down the TCP connection
|
||||||
|
* because it can be an indication of corrupt bytes from the underlying transport.
|
||||||
*/
|
*/
|
||||||
def fromBinary(bytes: Array[Byte], manifest: String): AnyRef
|
def fromBinary(bytes: Array[Byte], manifest: String): AnyRef
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ import akka.protobuf.{ ByteString, MessageLite }
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
|
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
|
||||||
|
|
@ -66,7 +67,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
|
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = manifest match {
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = manifest match {
|
||||||
case MetricsGossipEnvelopeManifest ⇒ metricsGossipEnvelopeFromBinary(bytes)
|
case MetricsGossipEnvelopeManifest ⇒ metricsGossipEnvelopeFromBinary(bytes)
|
||||||
case _ ⇒ throw new IllegalArgumentException(
|
case _ ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@ import akka.serialization.Serialization
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
import akka.protobuf.MessageLite
|
import akka.protobuf.MessageLite
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Protobuf serializer of ClusterSharding messages.
|
* INTERNAL API: Protobuf serializer of ClusterSharding messages.
|
||||||
|
|
@ -159,7 +160,7 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,7 @@ import akka.serialization.SerializationExtension
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
import akka.cluster.client.ClusterReceptionist
|
import akka.cluster.client.ClusterReceptionist
|
||||||
import akka.cluster.client.protobuf.msg.{ ClusterClientMessages ⇒ cm }
|
import akka.cluster.client.protobuf.msg.{ ClusterClientMessages ⇒ cm }
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Serializer of ClusterClient messages.
|
* INTERNAL API: Serializer of ClusterClient messages.
|
||||||
|
|
@ -54,7 +55,7 @@ private[akka] class ClusterClientMessageSerializer(val system: ExtendedActorSyst
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@ import akka.actor.ActorRef
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Protobuf serializer of DistributedPubSubMediator messages.
|
* INTERNAL API: Protobuf serializer of DistributedPubSubMediator messages.
|
||||||
|
|
@ -72,7 +73,7 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.cluster.singleton.ClusterSingletonManager.Internal.TakeOverFromMe
|
||||||
import akka.serialization.BaseSerializer
|
import akka.serialization.BaseSerializer
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Serializer of ClusterSingleton messages.
|
* INTERNAL API: Serializer of ClusterSingleton messages.
|
||||||
|
|
@ -56,7 +57,7 @@ private[akka] class ClusterSingletonMessageSerializer(val system: ExtendedActorS
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -18,6 +18,7 @@ import scala.annotation.tailrec
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration.Deadline
|
import scala.concurrent.duration.Deadline
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protobuf serializer of cluster messages.
|
* Protobuf serializer of cluster messages.
|
||||||
|
|
@ -107,7 +108,7 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = clazz match {
|
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = clazz match {
|
||||||
case Some(c) ⇒ fromBinaryMap.get(c.asInstanceOf[Class[ClusterMessage]]) match {
|
case Some(c) ⇒ fromBinaryMap.get(c.asInstanceOf[Class[ClusterMessage]]) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(s"Unimplemented deserialization of message class $c in ClusterSerializer")
|
case None ⇒ throw new NotSerializableException(s"Unimplemented deserialization of message class $c in ClusterSerializer")
|
||||||
}
|
}
|
||||||
case _ ⇒ throw new IllegalArgumentException("Need a cluster message class to be able to deserialize bytes in ClusterSerializer")
|
case _ ⇒ throw new IllegalArgumentException("Need a cluster message class to be able to deserialize bytes in ClusterSerializer")
|
||||||
}
|
}
|
||||||
|
|
@ -175,8 +176,7 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
} else {
|
} else {
|
||||||
// old remote node
|
// old remote node
|
||||||
uniqueAddress.getUid.toLong
|
uniqueAddress.getUid.toLong
|
||||||
}
|
})
|
||||||
)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private val memberStatusToInt = scala.collection.immutable.HashMap[MemberStatus, Int](
|
private val memberStatusToInt = scala.collection.immutable.HashMap[MemberStatus, Int](
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,7 @@ import akka.protobuf.ByteString
|
||||||
import akka.util.ByteString.UTF_8
|
import akka.util.ByteString.UTF_8
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import akka.cluster.UniqueAddress
|
import akka.cluster.UniqueAddress
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protobuf serializer of ReplicatedData.
|
* Protobuf serializer of ReplicatedData.
|
||||||
|
|
@ -126,7 +127,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,7 @@ import scala.annotation.tailrec
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -235,7 +236,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(f) ⇒ f(bytes)
|
case Some(f) ⇒ f(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -145,6 +145,16 @@ This is how a ``SerializerWithStringManifest`` looks like:
|
||||||
You must also bind it to a name in your :ref:`configuration` and then list which classes
|
You must also bind it to a name in your :ref:`configuration` and then list which classes
|
||||||
that should be serialized using it.
|
that should be serialized using it.
|
||||||
|
|
||||||
|
It's recommended to throw ``java.io.NotSerializableException`` in ``fromBinary``
|
||||||
|
if the manifest is unknown. This makes it possible to introduce new message types and
|
||||||
|
send them to nodes that don't know about them. This is typically needed when performing
|
||||||
|
rolling upgrades, i.e. running a cluster with mixed versions for while.
|
||||||
|
``NotSerializableException`` is treated as a transient problem in the TCP based remoting
|
||||||
|
layer. The problem will be logged and message is dropped. Other exceptions will tear down
|
||||||
|
the TCP connection because it can be an indication of corrupt bytes from the underlying
|
||||||
|
transport.
|
||||||
|
|
||||||
|
|
||||||
Serializing ActorRefs
|
Serializing ActorRefs
|
||||||
---------------------
|
---------------------
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -135,6 +135,15 @@ This is how a ``SerializerWithStringManifest`` looks like:
|
||||||
You must also bind it to a name in your :ref:`configuration` and then list which classes
|
You must also bind it to a name in your :ref:`configuration` and then list which classes
|
||||||
that should be serialized using it.
|
that should be serialized using it.
|
||||||
|
|
||||||
|
It's recommended to throw ``java.io.NotSerializableException`` in ``fromBinary``
|
||||||
|
if the manifest is unknown. This makes it possible to introduce new message types and
|
||||||
|
send them to nodes that don't know about them. This is typically needed when performing
|
||||||
|
rolling upgrades, i.e. running a cluster with mixed versions for while.
|
||||||
|
``NotSerializableException`` is treated as a transient problem in the TCP based remoting
|
||||||
|
layer. The problem will be logged and message is dropped. Other exceptions will tear down
|
||||||
|
the TCP connection because it can be an indication of corrupt bytes from the underlying
|
||||||
|
transport.
|
||||||
|
|
||||||
Serializing ActorRefs
|
Serializing ActorRefs
|
||||||
---------------------
|
---------------------
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ import scala.concurrent.duration
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import scala.language.existentials
|
import scala.language.existentials
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Marker trait for all protobuf-serializable messages in `akka.persistence`.
|
* Marker trait for all protobuf-serializable messages in `akka.persistence`.
|
||||||
|
|
@ -71,7 +72,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer
|
||||||
case AtLeastOnceDeliverySnapshotClass ⇒ atLeastOnceDeliverySnapshot(mf.AtLeastOnceDeliverySnapshot.parseFrom(bytes))
|
case AtLeastOnceDeliverySnapshotClass ⇒ atLeastOnceDeliverySnapshot(mf.AtLeastOnceDeliverySnapshot.parseFrom(bytes))
|
||||||
case PersistentStateChangeEventClass ⇒ stateChange(mf.PersistentStateChangeEvent.parseFrom(bytes))
|
case PersistentStateChangeEventClass ⇒ stateChange(mf.PersistentStateChangeEvent.parseFrom(bytes))
|
||||||
case PersistentFSMSnapshotClass ⇒ persistentFSMSnapshot(mf.PersistentFSMSnapshot.parseFrom(bytes))
|
case PersistentFSMSnapshotClass ⇒ persistentFSMSnapshot(mf.PersistentFSMSnapshot.parseFrom(bytes))
|
||||||
case _ ⇒ throw new IllegalArgumentException(s"Can't deserialize object of type ${c}")
|
case _ ⇒ throw new NotSerializableException(s"Can't deserialize object of type ${c}")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -984,7 +984,18 @@ private[remote] class EndpointReader(
|
||||||
if (msg.reliableDeliveryEnabled) {
|
if (msg.reliableDeliveryEnabled) {
|
||||||
ackedReceiveBuffer = ackedReceiveBuffer.receive(msg)
|
ackedReceiveBuffer = ackedReceiveBuffer.receive(msg)
|
||||||
deliverAndAck()
|
deliverAndAck()
|
||||||
} else msgDispatch.dispatch(msg.recipient, msg.recipientAddress, msg.serializedMessage, msg.senderOption)
|
} else try
|
||||||
|
msgDispatch.dispatch(msg.recipient, msg.recipientAddress, msg.serializedMessage, msg.senderOption)
|
||||||
|
catch {
|
||||||
|
case e: NotSerializableException ⇒
|
||||||
|
val sm = msg.serializedMessage
|
||||||
|
log.warning(
|
||||||
|
"Serializer not defined for message with serializer id [{}] and manifest [{}]. " +
|
||||||
|
"Transient association error (association remains live). {}",
|
||||||
|
sm.getSerializerId,
|
||||||
|
if (sm.hasMessageManifest) sm.getMessageManifest.toStringUtf8 else "",
|
||||||
|
e.getMessage)
|
||||||
|
}
|
||||||
|
|
||||||
case None ⇒
|
case None ⇒
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import akka.protobuf.ByteString
|
||||||
import akka.remote.{ ContainerFormats, RemoteWatcher }
|
import akka.remote.{ ContainerFormats, RemoteWatcher }
|
||||||
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
||||||
|
|
||||||
|
|
@ -152,7 +153,7 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||||
fromBinaryMap.get(manifest) match {
|
fromBinaryMap.get(manifest) match {
|
||||||
case Some(deserializer) ⇒ deserializer(bytes)
|
case Some(deserializer) ⇒ deserializer(bytes)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new NotSerializableException(
|
||||||
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,6 +12,7 @@ import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
object MiscMessageSerializerSpec {
|
object MiscMessageSerializerSpec {
|
||||||
val serializationTestOverrides =
|
val serializationTestOverrides =
|
||||||
|
|
@ -103,7 +104,7 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC
|
||||||
}
|
}
|
||||||
|
|
||||||
"reject deserialization with invalid manifest" in {
|
"reject deserialization with invalid manifest" in {
|
||||||
intercept[IllegalArgumentException] {
|
intercept[NotSerializableException] {
|
||||||
val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
||||||
serializer.fromBinary(Array.empty[Byte], "INVALID")
|
serializer.fromBinary(Array.empty[Byte], "INVALID")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue