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:
Patrik Nordwall 2016-12-16 11:36:04 +01:00
parent 1a12e950ff
commit e494ec2183
16 changed files with 64 additions and 15 deletions

View file

@ -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.

View file

@ -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

View file

@ -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}")
}

View file

@ -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}]")
}

View file

@ -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}]")
}

View file

@ -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}]")
}

View file

@ -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}]")
}

View file

@ -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](

View file

@ -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}]")
}

View file

@ -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}]")
}

View file

@ -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
---------------------

View file

@ -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
---------------------

View file

@ -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}")
}
}

View file

@ -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
}

View file

@ -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}]")
}

View file

@ -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")
}