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
|
|
@ -16,7 +16,7 @@ import scala.concurrent.duration.FiniteDuration
|
|||
|
||||
/**
|
||||
* INTERNAL API: Use `BalancingPool` instead of this dispatcher directly.
|
||||
*
|
||||
*
|
||||
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
||||
* that all actors using the same instance of this dispatcher can process all messages that have been sent to one of the actors. I.e. the
|
||||
* actors belong to a pool of actors, and to the client there is no guarantee about which actor instance actually processes a given message.
|
||||
|
|
|
|||
|
|
@ -120,6 +120,15 @@ abstract class SerializerWithStringManifest extends Serializer {
|
|||
/**
|
||||
* Produces an object from an array of bytes, with an optional type-hint;
|
||||
* 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
|
||||
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import akka.protobuf.{ ByteString, MessageLite }
|
|||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
||||
import akka.serialization.SerializerWithStringManifest
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
case MetricsGossipEnvelopeManifest ⇒ metricsGossipEnvelopeFromBinary(bytes)
|
||||
case _ ⇒ throw new IllegalArgumentException(
|
||||
case _ ⇒ throw new NotSerializableException(
|
||||
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.SerializerWithStringManifest
|
||||
import akka.protobuf.MessageLite
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.serialization.SerializationExtension
|
|||
import akka.serialization.SerializerWithStringManifest
|
||||
import akka.cluster.client.ClusterReceptionist
|
||||
import akka.cluster.client.protobuf.msg.{ ClusterClientMessages ⇒ cm }
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import akka.actor.ActorRef
|
|||
import akka.serialization.SerializationExtension
|
||||
import scala.collection.immutable.TreeMap
|
||||
import akka.serialization.SerializerWithStringManifest
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import akka.cluster.singleton.ClusterSingletonManager.Internal.TakeOverFromMe
|
|||
import akka.serialization.BaseSerializer
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.serialization.SerializerWithStringManifest
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ import scala.annotation.tailrec
|
|||
import scala.collection.JavaConverters._
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Deadline
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
case Some(c) ⇒ fromBinaryMap.get(c.asInstanceOf[Class[ClusterMessage]]) match {
|
||||
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")
|
||||
}
|
||||
|
|
@ -175,8 +176,7 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
|||
} else {
|
||||
// old remote node
|
||||
uniqueAddress.getUid.toLong
|
||||
}
|
||||
)
|
||||
})
|
||||
}
|
||||
|
||||
private val memberStatusToInt = scala.collection.immutable.HashMap[MemberStatus, Int](
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import akka.protobuf.ByteString
|
|||
import akka.util.ByteString.UTF_8
|
||||
import scala.collection.immutable.TreeMap
|
||||
import akka.cluster.UniqueAddress
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* Protobuf serializer of ReplicatedData.
|
||||
|
|
@ -126,7 +127,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -27,6 +27,7 @@ import scala.annotation.tailrec
|
|||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||
import akka.cluster.ddata.DurableStore.DurableDataEnvelope
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -235,7 +236,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
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
|
||||
---------------------
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
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
|
||||
---------------------
|
||||
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import scala.concurrent.duration
|
|||
import akka.actor.Actor
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.language.existentials
|
||||
import java.io.NotSerializableException
|
||||
|
||||
/**
|
||||
* 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 PersistentStateChangeEventClass ⇒ stateChange(mf.PersistentStateChangeEvent.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) {
|
||||
ackedReceiveBuffer = ackedReceiveBuffer.receive(msg)
|
||||
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 ⇒
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.protobuf.ByteString
|
|||
import akka.remote.{ ContainerFormats, RemoteWatcher }
|
||||
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
||||
import java.util.Optional
|
||||
import java.io.NotSerializableException
|
||||
|
||||
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 =
|
||||
fromBinaryMap.get(manifest) match {
|
||||
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}]")
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -12,6 +12,7 @@ import com.typesafe.config.ConfigFactory
|
|||
|
||||
import scala.util.control.NoStackTrace
|
||||
import java.util.Optional
|
||||
import java.io.NotSerializableException
|
||||
|
||||
object MiscMessageSerializerSpec {
|
||||
val serializationTestOverrides =
|
||||
|
|
@ -103,7 +104,7 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC
|
|||
}
|
||||
|
||||
"reject deserialization with invalid manifest" in {
|
||||
intercept[IllegalArgumentException] {
|
||||
intercept[NotSerializableException] {
|
||||
val serializer = new MiscMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
||||
serializer.fromBinary(Array.empty[Byte], "INVALID")
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue