Protobuf serializers for remote deployment #22332
This commit is contained in:
parent
3e4f44765c
commit
3643f18ded
24 changed files with 12700 additions and 314 deletions
File diff suppressed because it is too large
Load diff
|
|
@ -71,3 +71,28 @@ message Address {
|
||||||
required uint32 port = 3;
|
required uint32 port = 3;
|
||||||
optional string protocol = 4;
|
optional string protocol = 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/****************************************
|
||||||
|
* Router pool (for remote deployment
|
||||||
|
****************************************/
|
||||||
|
|
||||||
|
message AdaptiveLoadBalancingPool {
|
||||||
|
// omitted if default
|
||||||
|
optional MetricsSelector metricsSelector = 1;
|
||||||
|
required uint32 nrOfInstances = 2;
|
||||||
|
// omitted if default
|
||||||
|
optional string routerDispatcher = 3;
|
||||||
|
required bool usePoolDispatcher = 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
// couldn't figure out how to import Payload
|
||||||
|
message MetricsSelector {
|
||||||
|
required uint32 serializerId = 1;
|
||||||
|
required string manifest = 2;
|
||||||
|
required bytes data = 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
message MixMetricsSelector {
|
||||||
|
repeated MetricsSelector selectors = 1;
|
||||||
|
}
|
||||||
|
|
@ -94,6 +94,11 @@ akka.actor {
|
||||||
# Interface binding for remote cluster metrics messages.
|
# Interface binding for remote cluster metrics messages.
|
||||||
serialization-bindings {
|
serialization-bindings {
|
||||||
"akka.cluster.metrics.ClusterMetricsMessage" = akka-cluster-metrics
|
"akka.cluster.metrics.ClusterMetricsMessage" = akka-cluster-metrics
|
||||||
|
"akka.cluster.metrics.AdaptiveLoadBalancingPool" = akka-cluster-metrics
|
||||||
|
"akka.cluster.metrics.MixMetricsSelector" = akka-cluster-metrics
|
||||||
|
"akka.cluster.metrics.CpuMetricsSelector$" = akka-cluster-metrics
|
||||||
|
"akka.cluster.metrics.HeapMetricsSelector$" = akka-cluster-metrics
|
||||||
|
"akka.cluster.metrics.SystemLoadAverageMetricsSelector$" = akka-cluster-metrics
|
||||||
}
|
}
|
||||||
# Globally unique metrics extension serializer identifier.
|
# Globally unique metrics extension serializer identifier.
|
||||||
serialization-identifiers {
|
serialization-identifiers {
|
||||||
|
|
|
||||||
|
|
@ -7,17 +7,20 @@ package akka.cluster.metrics.protobuf
|
||||||
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
|
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
|
||||||
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
|
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
|
||||||
import java.{ lang ⇒ jl }
|
import java.{ lang ⇒ jl }
|
||||||
|
|
||||||
import akka.actor.{ Address, ExtendedActorSystem }
|
import akka.actor.{ Address, ExtendedActorSystem }
|
||||||
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages ⇒ cm }
|
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages ⇒ cm }
|
||||||
import akka.cluster.metrics.{ EWMA, Metric, MetricsGossip, MetricsGossipEnvelope, NodeMetrics }
|
import akka.cluster.metrics._
|
||||||
import akka.serialization.BaseSerializer
|
import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest }
|
||||||
import akka.util.ClassLoaderObjectInputStream
|
import akka.util.ClassLoaderObjectInputStream
|
||||||
import akka.protobuf.{ ByteString, MessageLite }
|
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 java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
|
import akka.dispatch.Dispatchers
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
|
* Protobuf serializer for [[akka.cluster.metrics.ClusterMetricsMessage]] types.
|
||||||
*/
|
*/
|
||||||
|
|
@ -26,16 +29,32 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
private final val BufferSize = 4 * 1024
|
private final val BufferSize = 4 * 1024
|
||||||
|
|
||||||
private val MetricsGossipEnvelopeManifest = "a"
|
private val MetricsGossipEnvelopeManifest = "a"
|
||||||
|
private val AdaptiveLoadBalancingPoolManifest = "b"
|
||||||
|
private val MixMetricsSelectorManifest = "c"
|
||||||
|
private val CpuMetricsSelectorManifest = "d"
|
||||||
|
private val HeapMetricsSelectorManifest = "e"
|
||||||
|
private val SystemLoadAverageMetricsSelectorManifest = "f"
|
||||||
|
|
||||||
|
private lazy val serialization = SerializationExtension(system)
|
||||||
|
|
||||||
override def manifest(obj: AnyRef): String = obj match {
|
override def manifest(obj: AnyRef): String = obj match {
|
||||||
case _: MetricsGossipEnvelope ⇒ MetricsGossipEnvelopeManifest
|
case _: MetricsGossipEnvelope ⇒ MetricsGossipEnvelopeManifest
|
||||||
|
case _: AdaptiveLoadBalancingPool ⇒ AdaptiveLoadBalancingPoolManifest
|
||||||
|
case _: MixMetricsSelector ⇒ MixMetricsSelectorManifest
|
||||||
|
case CpuMetricsSelector ⇒ CpuMetricsSelectorManifest
|
||||||
|
case HeapMetricsSelector ⇒ HeapMetricsSelectorManifest
|
||||||
|
case SystemLoadAverageMetricsSelector ⇒ SystemLoadAverageMetricsSelectorManifest
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
override def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||||
case m: MetricsGossipEnvelope ⇒
|
case m: MetricsGossipEnvelope ⇒ compress(metricsGossipEnvelopeToProto(m))
|
||||||
compress(metricsGossipEnvelopeToProto(m))
|
case alb: AdaptiveLoadBalancingPool ⇒ adaptiveLoadBalancingPoolToBinary(alb)
|
||||||
|
case mms: MixMetricsSelector ⇒ mixMetricSelectorToBinary(mms)
|
||||||
|
case CpuMetricsSelector ⇒ Array.emptyByteArray
|
||||||
|
case HeapMetricsSelector ⇒ Array.emptyByteArray
|
||||||
|
case SystemLoadAverageMetricsSelector ⇒ Array.emptyByteArray
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass} in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
@ -67,6 +86,11 @@ 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 AdaptiveLoadBalancingPoolManifest ⇒ adaptiveLoadBalancingPoolFromBinary(bytes)
|
||||||
|
case MixMetricsSelectorManifest ⇒ mixMetricSelectorFromBinary(bytes)
|
||||||
|
case CpuMetricsSelectorManifest ⇒ CpuMetricsSelector
|
||||||
|
case HeapMetricsSelectorManifest ⇒ HeapMetricsSelector
|
||||||
|
case SystemLoadAverageMetricsSelectorManifest ⇒ SystemLoadAverageMetricsSelector
|
||||||
case _ ⇒ throw new NotSerializableException(
|
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}")
|
||||||
}
|
}
|
||||||
|
|
@ -77,6 +101,48 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
|
case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def adaptiveLoadBalancingPoolToBinary(alb: AdaptiveLoadBalancingPool): Array[Byte] = {
|
||||||
|
val builder = cm.AdaptiveLoadBalancingPool.newBuilder()
|
||||||
|
if (alb.metricsSelector != MixMetricsSelector) {
|
||||||
|
builder.setMetricsSelector(metricsSelectorToProto(alb.metricsSelector))
|
||||||
|
}
|
||||||
|
if (alb.routerDispatcher != Dispatchers.DefaultDispatcherId) {
|
||||||
|
builder.setRouterDispatcher(alb.routerDispatcher)
|
||||||
|
}
|
||||||
|
builder.setNrOfInstances(alb.nrOfInstances)
|
||||||
|
builder.setUsePoolDispatcher(alb.usePoolDispatcher)
|
||||||
|
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def metricsSelectorToProto(selector: MetricsSelector): cm.MetricsSelector = {
|
||||||
|
val builder = cm.MetricsSelector.newBuilder()
|
||||||
|
val serializer = serialization.findSerializerFor(selector)
|
||||||
|
|
||||||
|
builder.setData(ByteString.copyFrom(serializer.toBinary(selector)))
|
||||||
|
.setSerializerId(serializer.identifier)
|
||||||
|
|
||||||
|
serializer match {
|
||||||
|
case ser2: SerializerWithStringManifest ⇒
|
||||||
|
val manifest = ser2.manifest(selector)
|
||||||
|
builder.setManifest(manifest)
|
||||||
|
case _ ⇒
|
||||||
|
builder.setManifest(
|
||||||
|
if (serializer.includeManifest) selector.getClass.getName
|
||||||
|
else ""
|
||||||
|
)
|
||||||
|
}
|
||||||
|
builder.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
private def mixMetricSelectorToBinary(mms: MixMetricsSelector): Array[Byte] = {
|
||||||
|
val builder = cm.MixMetricsSelector.newBuilder()
|
||||||
|
mms.selectors.foreach { selector ⇒
|
||||||
|
builder.addSelectors(metricsSelectorToProto(selector))
|
||||||
|
}
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
@volatile
|
@volatile
|
||||||
private var protocolCache: String = null
|
private var protocolCache: String = null
|
||||||
@volatile
|
@volatile
|
||||||
|
|
@ -200,4 +266,39 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
|
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def adaptiveLoadBalancingPoolFromBinary(bytes: Array[Byte]): AdaptiveLoadBalancingPool = {
|
||||||
|
val alb = cm.AdaptiveLoadBalancingPool.parseFrom(bytes)
|
||||||
|
|
||||||
|
val selector =
|
||||||
|
if (alb.hasMetricsSelector) {
|
||||||
|
val ms = alb.getMetricsSelector
|
||||||
|
serialization.deserialize(
|
||||||
|
ms.getData.toByteArray,
|
||||||
|
ms.getSerializerId,
|
||||||
|
ms.getManifest
|
||||||
|
).get.asInstanceOf[MetricsSelector]
|
||||||
|
} else MixMetricsSelector
|
||||||
|
|
||||||
|
AdaptiveLoadBalancingPool(
|
||||||
|
metricsSelector = selector,
|
||||||
|
nrOfInstances = alb.getNrOfInstances,
|
||||||
|
routerDispatcher = if (alb.hasRouterDispatcher) alb.getRouterDispatcher else Dispatchers.DefaultDispatcherId,
|
||||||
|
usePoolDispatcher = alb.getUsePoolDispatcher
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
def mixMetricSelectorFromBinary(bytes: Array[Byte]): MixMetricsSelector = {
|
||||||
|
val mm = cm.MixMetricsSelector.parseFrom(bytes)
|
||||||
|
MixMetricsSelector(mm.getSelectorsList.asScala
|
||||||
|
// should be safe because we serialized only the right subtypes of MetricsSelector
|
||||||
|
.map(s ⇒ metricSelectorFromProto(s).asInstanceOf[CapacityMetricsSelector]).toIndexedSeq)
|
||||||
|
}
|
||||||
|
|
||||||
|
def metricSelectorFromProto(selector: cm.MetricsSelector): MetricsSelector =
|
||||||
|
serialization.deserialize(
|
||||||
|
selector.getData.toByteArray,
|
||||||
|
selector.getSerializerId,
|
||||||
|
selector.getManifest
|
||||||
|
).get.asInstanceOf[MetricsSelector]
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,18 +4,19 @@
|
||||||
|
|
||||||
package akka.cluster.metrics.protobuf
|
package akka.cluster.metrics.protobuf
|
||||||
|
|
||||||
import akka.actor.{ ExtendedActorSystem, Address }
|
import akka.actor.{ Address, ExtendedActorSystem }
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.cluster.MemberStatus
|
import akka.cluster.MemberStatus
|
||||||
import akka.cluster.metrics.MetricsGossip
|
import akka.cluster.metrics._
|
||||||
import akka.cluster.metrics.NodeMetrics
|
|
||||||
import akka.cluster.metrics.Metric
|
|
||||||
import akka.cluster.metrics.EWMA
|
|
||||||
import akka.cluster.TestMember
|
import akka.cluster.TestMember
|
||||||
import akka.cluster.metrics.MetricsGossipEnvelope
|
|
||||||
|
|
||||||
class MessageSerializerSpec extends AkkaSpec(
|
class MessageSerializerSpec extends AkkaSpec(
|
||||||
"akka.actor.provider = cluster") {
|
"""
|
||||||
|
akka.actor.provider = cluster
|
||||||
|
akka.actor.serialize-messages = off
|
||||||
|
akka.actor.allow-java-serialization = off
|
||||||
|
akka.actor.enable-additional-serialization-bindings = on
|
||||||
|
""") {
|
||||||
|
|
||||||
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
val serializer = new MessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
||||||
|
|
||||||
|
|
@ -56,4 +57,18 @@ class MessageSerializerSpec extends AkkaSpec(
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"AdaptiveLoadBalancingPool" must {
|
||||||
|
"be serializable" in {
|
||||||
|
val simplePool = AdaptiveLoadBalancingPool()
|
||||||
|
checkSerialization(simplePool)
|
||||||
|
|
||||||
|
val complicatedPool = AdaptiveLoadBalancingPool(
|
||||||
|
metricsSelector = MixMetricsSelector(Vector(CpuMetricsSelector, HeapMetricsSelector, SystemLoadAverageMetricsSelector)),
|
||||||
|
nrOfInstances = 7,
|
||||||
|
routerDispatcher = "my-dispatcher",
|
||||||
|
usePoolDispatcher = true)
|
||||||
|
checkSerialization(complicatedPool)
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -202,3 +202,25 @@ message UniqueAddress {
|
||||||
// 64 bit uids but with backward wire compatibility
|
// 64 bit uids but with backward wire compatibility
|
||||||
optional uint32 uid2 = 3;
|
optional uint32 uid2 = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/****************************************
|
||||||
|
* Cluster routing
|
||||||
|
****************************************/
|
||||||
|
|
||||||
|
message ClusterRouterPool {
|
||||||
|
required Pool pool = 1;
|
||||||
|
required ClusterRouterPoolSettings settings = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
message Pool {
|
||||||
|
required uint32 serializerId = 1;
|
||||||
|
required string manifest = 2;
|
||||||
|
required bytes data = 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
message ClusterRouterPoolSettings {
|
||||||
|
required uint32 totalInstances = 1;
|
||||||
|
required uint32 maxInstancesPerNode = 2;
|
||||||
|
required bool allowLocalRoutees = 3;
|
||||||
|
optional string useRole = 4;
|
||||||
|
}
|
||||||
|
|
@ -247,6 +247,7 @@ akka {
|
||||||
|
|
||||||
serialization-bindings {
|
serialization-bindings {
|
||||||
"akka.cluster.ClusterMessage" = akka-cluster
|
"akka.cluster.ClusterMessage" = akka-cluster
|
||||||
|
"akka.cluster.routing.ClusterRouterPool" = akka-cluster
|
||||||
}
|
}
|
||||||
|
|
||||||
serialization-identifiers {
|
serialization-identifiers {
|
||||||
|
|
|
||||||
|
|
@ -3,15 +3,13 @@
|
||||||
*/
|
*/
|
||||||
package akka.cluster.protobuf
|
package akka.cluster.protobuf
|
||||||
|
|
||||||
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
|
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
|
||||||
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
|
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
|
||||||
import java.{ lang ⇒ jl }
|
|
||||||
|
|
||||||
import akka.actor.{ Address, ExtendedActorSystem }
|
import akka.actor.{ Address, ExtendedActorSystem }
|
||||||
import akka.cluster._
|
import akka.cluster._
|
||||||
import akka.cluster.protobuf.msg.{ ClusterMessages ⇒ cm }
|
import akka.cluster.protobuf.msg.{ ClusterMessages ⇒ cm }
|
||||||
import akka.serialization.BaseSerializer
|
import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest }
|
||||||
import akka.util.ClassLoaderObjectInputStream
|
|
||||||
import akka.protobuf.{ ByteString, MessageLite }
|
import akka.protobuf.{ ByteString, MessageLite }
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
@ -19,18 +17,23 @@ 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
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
import akka.cluster.InternalClusterAction.ExitingConfirmed
|
import akka.cluster.InternalClusterAction.ExitingConfirmed
|
||||||
|
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
||||||
|
import akka.routing.Pool
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Protobuf serializer of cluster messages.
|
* Protobuf serializer of cluster messages.
|
||||||
*/
|
*/
|
||||||
class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
|
private lazy val serialization = SerializationExtension(system)
|
||||||
|
|
||||||
private final val BufferSize = 1024 * 4
|
private final val BufferSize = 1024 * 4
|
||||||
// must be lazy because serializer is initialized from Cluster extension constructor
|
// must be lazy because serializer is initialized from Cluster extension constructor
|
||||||
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
|
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
|
||||||
|
|
||||||
private val fromBinaryMap = collection.immutable.HashMap[Class[_ <: ClusterMessage], Array[Byte] ⇒ AnyRef](
|
private val fromBinaryMap = collection.immutable.HashMap[Class[_], Array[Byte] ⇒ AnyRef](
|
||||||
classOf[InternalClusterAction.Join] → {
|
classOf[InternalClusterAction.Join] → {
|
||||||
case bytes ⇒
|
case bytes ⇒
|
||||||
val m = cm.Join.parseFrom(bytes)
|
val m = cm.Join.parseFrom(bytes)
|
||||||
|
|
@ -52,7 +55,9 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
classOf[ClusterHeartbeatSender.HeartbeatRsp] → (bytes ⇒ ClusterHeartbeatSender.HeartbeatRsp(uniqueAddressFromBinary(bytes))),
|
classOf[ClusterHeartbeatSender.HeartbeatRsp] → (bytes ⇒ ClusterHeartbeatSender.HeartbeatRsp(uniqueAddressFromBinary(bytes))),
|
||||||
classOf[ExitingConfirmed] → (bytes ⇒ InternalClusterAction.ExitingConfirmed(uniqueAddressFromBinary(bytes))),
|
classOf[ExitingConfirmed] → (bytes ⇒ InternalClusterAction.ExitingConfirmed(uniqueAddressFromBinary(bytes))),
|
||||||
classOf[GossipStatus] → gossipStatusFromBinary,
|
classOf[GossipStatus] → gossipStatusFromBinary,
|
||||||
classOf[GossipEnvelope] → gossipEnvelopeFromBinary)
|
classOf[GossipEnvelope] → gossipEnvelopeFromBinary,
|
||||||
|
classOf[ClusterRouterPool] → clusterRouterPoolFromBinary
|
||||||
|
)
|
||||||
|
|
||||||
def includeManifest: Boolean = true
|
def includeManifest: Boolean = true
|
||||||
|
|
||||||
|
|
@ -69,6 +74,7 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
case InternalClusterAction.InitJoinAck(address) ⇒ addressToProtoByteArray(address)
|
case InternalClusterAction.InitJoinAck(address) ⇒ addressToProtoByteArray(address)
|
||||||
case InternalClusterAction.InitJoinNack(address) ⇒ addressToProtoByteArray(address)
|
case InternalClusterAction.InitJoinNack(address) ⇒ addressToProtoByteArray(address)
|
||||||
case InternalClusterAction.ExitingConfirmed(node) ⇒ uniqueAddressToProtoByteArray(node)
|
case InternalClusterAction.ExitingConfirmed(node) ⇒ uniqueAddressToProtoByteArray(node)
|
||||||
|
case rp: ClusterRouterPool ⇒ clusterRouterPoolToProtoByteArray(rp)
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass}")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${obj.getClass}")
|
||||||
}
|
}
|
||||||
|
|
@ -99,7 +105,8 @@ 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 NotSerializableException(s"Unimplemented deserialization of message class $c in ClusterSerializer")
|
case None ⇒ throw new NotSerializableException(s"Unimplemented deserialization of message class $c in ClusterSerializer")
|
||||||
}
|
}
|
||||||
|
|
@ -130,6 +137,40 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
private def uniqueAddressToProtoByteArray(uniqueAddress: UniqueAddress): Array[Byte] =
|
private def uniqueAddressToProtoByteArray(uniqueAddress: UniqueAddress): Array[Byte] =
|
||||||
uniqueAddressToProto(uniqueAddress).build.toByteArray
|
uniqueAddressToProto(uniqueAddress).build.toByteArray
|
||||||
|
|
||||||
|
private def clusterRouterPoolToProtoByteArray(rp: ClusterRouterPool): Array[Byte] = {
|
||||||
|
val builder = cm.ClusterRouterPool.newBuilder()
|
||||||
|
builder.setPool(poolToProto(rp.local))
|
||||||
|
builder.setSettings(clusterRouterPoolSettingsToProto(rp.settings))
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def poolToProto(pool: Pool): cm.Pool = {
|
||||||
|
val builder = cm.Pool.newBuilder()
|
||||||
|
val serializer = serialization.findSerializerFor(pool)
|
||||||
|
builder.setSerializerId(serializer.identifier)
|
||||||
|
.setData(ByteString.copyFrom(serializer.toBinary(pool)))
|
||||||
|
serializer match {
|
||||||
|
case ser: SerializerWithStringManifest ⇒
|
||||||
|
builder.setManifest(ser.manifest(pool))
|
||||||
|
case _ ⇒
|
||||||
|
builder.setManifest(
|
||||||
|
if (serializer.includeManifest) pool.getClass.getName
|
||||||
|
else ""
|
||||||
|
)
|
||||||
|
}
|
||||||
|
builder.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
private def clusterRouterPoolSettingsToProto(settings: ClusterRouterPoolSettings): cm.ClusterRouterPoolSettings = {
|
||||||
|
val builder = cm.ClusterRouterPoolSettings.newBuilder()
|
||||||
|
builder.setAllowLocalRoutees(settings.allowLocalRoutees)
|
||||||
|
.setMaxInstancesPerNode(settings.maxInstancesPerNode)
|
||||||
|
.setTotalInstances(settings.totalInstances)
|
||||||
|
|
||||||
|
settings.useRole.foreach(builder.setUseRole)
|
||||||
|
builder.build()
|
||||||
|
}
|
||||||
|
|
||||||
// we don't care about races here since it's just a cache
|
// we don't care about races here since it's just a cache
|
||||||
@volatile
|
@volatile
|
||||||
private var protocolCache: String = _
|
private var protocolCache: String = _
|
||||||
|
|
@ -323,4 +364,26 @@ class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSeri
|
||||||
status.getVersion,
|
status.getVersion,
|
||||||
status.getAllHashesList.asScala.toVector))
|
status.getAllHashesList.asScala.toVector))
|
||||||
|
|
||||||
|
def clusterRouterPoolFromBinary(bytes: Array[Byte]): ClusterRouterPool = {
|
||||||
|
val crp = cm.ClusterRouterPool.parseFrom(bytes)
|
||||||
|
|
||||||
|
ClusterRouterPool(
|
||||||
|
poolFromProto(crp.getPool),
|
||||||
|
clusterRouterPoolSettingsFromProto(crp.getSettings)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def poolFromProto(pool: cm.Pool): Pool = {
|
||||||
|
serialization.deserialize(pool.getData.toByteArray, pool.getSerializerId, pool.getManifest).get.asInstanceOf[Pool]
|
||||||
|
}
|
||||||
|
|
||||||
|
private def clusterRouterPoolSettingsFromProto(crps: cm.ClusterRouterPoolSettings): ClusterRouterPoolSettings = {
|
||||||
|
ClusterRouterPoolSettings(
|
||||||
|
totalInstances = crps.getTotalInstances,
|
||||||
|
maxInstancesPerNode = crps.getMaxInstancesPerNode,
|
||||||
|
allowLocalRoutees = crps.getAllowLocalRoutees,
|
||||||
|
useRole = if (crps.hasUseRole) Some(crps.getUseRole) else None
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,15 @@ object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
|
||||||
|
|
||||||
commonConfig(debugConfig(on = false).
|
commonConfig(debugConfig(on = false).
|
||||||
withFallback(ConfigFactory.parseString("""
|
withFallback(ConfigFactory.parseString("""
|
||||||
akka.actor.deployment {
|
akka.actor {
|
||||||
|
allow-java-serialization = off
|
||||||
|
serialize-creators = off
|
||||||
|
serialize-messages = off
|
||||||
|
serialization-bindings {
|
||||||
|
"akka.cluster.routing.ClusterRoundRobinMultiJvmSpec$Reply" = test-message-serializer
|
||||||
|
}
|
||||||
|
|
||||||
|
deployment {
|
||||||
/router1 {
|
/router1 {
|
||||||
router = round-robin-pool
|
router = round-robin-pool
|
||||||
cluster {
|
cluster {
|
||||||
|
|
@ -82,6 +90,7 @@ object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
""")).
|
""")).
|
||||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,7 +1,7 @@
|
||||||
akka {
|
akka {
|
||||||
actor {
|
actor {
|
||||||
serialize-creators = on
|
serialize-creators = on
|
||||||
serialize-messages = on
|
serialize-messages = off
|
||||||
warn-about-java-serializer-usage = off
|
warn-about-java-serializer-usage = off
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,19 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
|
||||||
*/
|
|
||||||
package akka.cluster
|
|
||||||
|
|
||||||
import akka.testkit.AkkaSpec
|
|
||||||
|
|
||||||
class SerializationChecksSpec extends AkkaSpec {
|
|
||||||
|
|
||||||
"Settings serialize-messages and serialize-creators" must {
|
|
||||||
|
|
||||||
"be on for tests" in {
|
|
||||||
system.settings.SerializeAllCreators should ===(true)
|
|
||||||
system.settings.SerializeAllMessages should ===(true)
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
@ -4,7 +4,10 @@
|
||||||
package akka.cluster.protobuf
|
package akka.cluster.protobuf
|
||||||
|
|
||||||
import akka.cluster._
|
import akka.cluster._
|
||||||
import akka.actor.{ ExtendedActorSystem, Address }
|
import akka.actor.{ Address, ExtendedActorSystem }
|
||||||
|
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
||||||
|
import akka.routing.{ DefaultOptimalSizeExploringResizer, RoundRobinPool }
|
||||||
|
|
||||||
import collection.immutable.SortedSet
|
import collection.immutable.SortedSet
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
|
|
||||||
|
|
@ -73,4 +76,20 @@ class ClusterMessageSerializerSpec extends AkkaSpec(
|
||||||
checkSerialization(InternalClusterAction.Welcome(uniqueAddress, g2))
|
checkSerialization(InternalClusterAction.Welcome(uniqueAddress, g2))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
"Cluster router pool" must {
|
||||||
|
"be serializable" in {
|
||||||
|
checkSerialization(ClusterRouterPool(
|
||||||
|
RoundRobinPool(
|
||||||
|
nrOfInstances = 4
|
||||||
|
),
|
||||||
|
ClusterRouterPoolSettings(
|
||||||
|
totalInstances = 2,
|
||||||
|
maxInstancesPerNode = 5,
|
||||||
|
allowLocalRoutees = true,
|
||||||
|
useRole = Some("Richard, Duke of Gloucester")
|
||||||
|
)
|
||||||
|
))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
File diff suppressed because it is too large
Load diff
|
|
@ -5,6 +5,8 @@
|
||||||
option java_package = "akka.remote";
|
option java_package = "akka.remote";
|
||||||
option optimize_for = SPEED;
|
option optimize_for = SPEED;
|
||||||
|
|
||||||
|
import "ContainerFormats.proto";
|
||||||
|
|
||||||
/******************************************
|
/******************************************
|
||||||
* Remoting message formats
|
* Remoting message formats
|
||||||
******************************************/
|
******************************************/
|
||||||
|
|
@ -84,6 +86,14 @@ message DeployData {
|
||||||
optional bytes routerConfig = 3;
|
optional bytes routerConfig = 3;
|
||||||
optional bytes scope = 4;
|
optional bytes scope = 4;
|
||||||
optional string dispatcher = 5;
|
optional string dispatcher = 5;
|
||||||
|
// older wire protocol: hardcoded class used to look up serializer
|
||||||
|
// newer wire protocol: serializer id and manifest available for each
|
||||||
|
optional int32 scopeSerializerId = 6;
|
||||||
|
optional string scopeManifest = 7;
|
||||||
|
optional int32 configSerializerId = 8;
|
||||||
|
optional string configManifest = 9;
|
||||||
|
optional int32 routerConfigSerializerId = 10;
|
||||||
|
optional string routerConfigManifest = 11;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -135,3 +145,65 @@ message AddressData {
|
||||||
required uint32 port = 3;
|
required uint32 port = 3;
|
||||||
optional string protocol = 4;
|
optional string protocol = 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* java.util.concurrent.TimeUnit enum
|
||||||
|
*/
|
||||||
|
enum TimeUnit {
|
||||||
|
NANOSECONDS = 1;
|
||||||
|
MICROSECONDS = 2;
|
||||||
|
MILLISECONDS = 3;
|
||||||
|
SECONDS = 4;
|
||||||
|
MINUTES = 5;
|
||||||
|
HOURS = 6;
|
||||||
|
DAYS = 7;
|
||||||
|
}
|
||||||
|
|
||||||
|
message FiniteDuration {
|
||||||
|
required int64 value = 1;
|
||||||
|
required TimeUnit unit = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
message RemoteScope {
|
||||||
|
required AddressData node = 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// router configs
|
||||||
|
|
||||||
|
message DefaultResizer {
|
||||||
|
required uint32 lowerBound = 1;
|
||||||
|
required uint32 upperBound = 2;
|
||||||
|
required uint32 pressureThreshold = 3;
|
||||||
|
required double rampupRate = 4;
|
||||||
|
required double backoffThreshold = 5;
|
||||||
|
required double backoffRate = 6;
|
||||||
|
required uint32 messagesPerResize = 7;
|
||||||
|
}
|
||||||
|
|
||||||
|
message FromConfig {
|
||||||
|
optional Payload resizer = 1;
|
||||||
|
optional string routerDispatcher = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
message GenericRoutingPool {
|
||||||
|
required uint32 nrOfInstances = 1;
|
||||||
|
optional string routerDispatcher = 2;
|
||||||
|
required bool usePoolDispatcher = 3;
|
||||||
|
optional Payload resizer = 4;
|
||||||
|
}
|
||||||
|
|
||||||
|
message ScatterGatherPool {
|
||||||
|
required GenericRoutingPool generic = 1;
|
||||||
|
required FiniteDuration within = 2;
|
||||||
|
}
|
||||||
|
|
||||||
|
message TailChoppingPool {
|
||||||
|
required GenericRoutingPool generic = 1;
|
||||||
|
required FiniteDuration within = 2;
|
||||||
|
required FiniteDuration interval = 3;
|
||||||
|
}
|
||||||
|
|
||||||
|
message RemoteRouterConfig {
|
||||||
|
required Payload local = 1;
|
||||||
|
repeated AddressData nodes = 2;
|
||||||
|
}
|
||||||
|
|
@ -93,6 +93,28 @@ akka {
|
||||||
"akka.actor.ActorKilledException" = akka-misc
|
"akka.actor.ActorKilledException" = akka-misc
|
||||||
"akka.actor.InvalidActorNameException" = akka-misc
|
"akka.actor.InvalidActorNameException" = akka-misc
|
||||||
"akka.actor.InvalidMessageException" = akka-misc
|
"akka.actor.InvalidMessageException" = akka-misc
|
||||||
|
|
||||||
|
"akka.actor.LocalScope$" = akka-misc
|
||||||
|
"akka.remote.RemoteScope" = akka-misc
|
||||||
|
|
||||||
|
"com.typesafe.config.impl.SimpleConfig" = akka-misc
|
||||||
|
"com.typesafe.config.Config" = akka-misc
|
||||||
|
|
||||||
|
"akka.routing.FromConfig" = akka-misc
|
||||||
|
"akka.routing.DefaultResizer" = akka-misc
|
||||||
|
"akka.routing.BalancingPool" = akka-misc
|
||||||
|
"akka.routing.BroadcastGroup" = akka-misc
|
||||||
|
"akka.routing.BroadcastPool" = akka-misc
|
||||||
|
"akka.routing.RandomGroup" = akka-misc
|
||||||
|
"akka.routing.RandomPool" = akka-misc
|
||||||
|
"akka.routing.RoundRobinGroup" = akka-misc
|
||||||
|
"akka.routing.RoundRobinPool" = akka-misc
|
||||||
|
"akka.routing.ScatterGatherFirstCompletedGroup" = akka-misc
|
||||||
|
"akka.routing.ScatterGatherFirstCompletedPool" = akka-misc
|
||||||
|
"akka.routing.SmallestMailboxPool" = akka-misc
|
||||||
|
"akka.routing.TailChoppingGroup" = akka-misc
|
||||||
|
"akka.routing.TailChoppingPool" = akka-misc
|
||||||
|
"akka.remote.routing.RemoteRouterConfig" = akka-misc
|
||||||
}
|
}
|
||||||
|
|
||||||
serialization-identifiers {
|
serialization-identifiers {
|
||||||
|
|
|
||||||
|
|
@ -41,14 +41,31 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys
|
||||||
|
|
||||||
def deployProto(d: Deploy): DeployData = {
|
def deployProto(d: Deploy): DeployData = {
|
||||||
val builder = DeployData.newBuilder.setPath(d.path)
|
val builder = DeployData.newBuilder.setPath(d.path)
|
||||||
if (d.config != ConfigFactory.empty)
|
|
||||||
builder.setConfig(oldSerialize(d.config))
|
{
|
||||||
if (d.routerConfig != NoRouter)
|
val (serId, _, manifest, bytes) = serialize(d.config)
|
||||||
builder.setRouterConfig(oldSerialize(d.routerConfig))
|
builder.setConfigSerializerId(serId)
|
||||||
if (d.scope != NoScopeGiven)
|
builder.setConfigManifest(manifest)
|
||||||
builder.setScope(oldSerialize(d.scope))
|
builder.setConfig(ByteString.copyFrom(bytes))
|
||||||
if (d.dispatcher != NoDispatcherGiven)
|
}
|
||||||
|
|
||||||
|
if (d.routerConfig != NoRouter) {
|
||||||
|
val (serId, _, manifest, bytes) = serialize(d.routerConfig)
|
||||||
|
builder.setRouterConfigSerializerId(serId)
|
||||||
|
builder.setRouterConfigManifest(manifest)
|
||||||
|
builder.setRouterConfig(ByteString.copyFrom(bytes))
|
||||||
|
}
|
||||||
|
|
||||||
|
if (d.scope != NoScopeGiven) {
|
||||||
|
val (serId, _, manifest, bytes) = serialize(d.scope)
|
||||||
|
builder.setScopeSerializerId(serId)
|
||||||
|
builder.setScopeManifest(manifest)
|
||||||
|
builder.setScope(ByteString.copyFrom(bytes))
|
||||||
|
}
|
||||||
|
|
||||||
|
if (d.dispatcher != NoDispatcherGiven) {
|
||||||
builder.setDispatcher(d.dispatcher)
|
builder.setDispatcher(d.dispatcher)
|
||||||
|
}
|
||||||
builder.build
|
builder.build
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -82,15 +99,45 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys
|
||||||
val proto = DaemonMsgCreateData.parseFrom(bytes)
|
val proto = DaemonMsgCreateData.parseFrom(bytes)
|
||||||
|
|
||||||
def deploy(protoDeploy: DeployData): Deploy = {
|
def deploy(protoDeploy: DeployData): Deploy = {
|
||||||
|
|
||||||
val config =
|
val config =
|
||||||
if (protoDeploy.hasConfig) oldDeserialize(protoDeploy.getConfig, classOf[Config])
|
if (protoDeploy.hasConfig) {
|
||||||
else ConfigFactory.empty
|
if (protoDeploy.hasConfigSerializerId) {
|
||||||
|
serialization.deserialize(
|
||||||
|
protoDeploy.getConfig.toByteArray,
|
||||||
|
protoDeploy.getConfigSerializerId,
|
||||||
|
protoDeploy.getConfigManifest).get.asInstanceOf[Config]
|
||||||
|
} else {
|
||||||
|
// old wire format
|
||||||
|
oldDeserialize(protoDeploy.getConfig, classOf[Config])
|
||||||
|
}
|
||||||
|
} else ConfigFactory.empty
|
||||||
|
|
||||||
val routerConfig =
|
val routerConfig =
|
||||||
if (protoDeploy.hasRouterConfig) oldDeserialize(protoDeploy.getRouterConfig, classOf[RouterConfig])
|
if (protoDeploy.hasRouterConfig) {
|
||||||
else NoRouter
|
if (protoDeploy.hasRouterConfigSerializerId) {
|
||||||
|
serialization.deserialize(
|
||||||
|
protoDeploy.getRouterConfig.toByteArray,
|
||||||
|
protoDeploy.getRouterConfigSerializerId,
|
||||||
|
protoDeploy.getRouterConfigManifest).get.asInstanceOf[RouterConfig]
|
||||||
|
} else {
|
||||||
|
// old wire format
|
||||||
|
oldDeserialize(protoDeploy.getRouterConfig, classOf[RouterConfig])
|
||||||
|
}
|
||||||
|
} else NoRouter
|
||||||
|
|
||||||
val scope =
|
val scope =
|
||||||
if (protoDeploy.hasScope) oldDeserialize(protoDeploy.getScope, classOf[Scope])
|
if (protoDeploy.hasScope) {
|
||||||
else NoScopeGiven
|
if (protoDeploy.hasScopeSerializerId) {
|
||||||
|
serialization.deserialize(
|
||||||
|
protoDeploy.getScope.toByteArray,
|
||||||
|
protoDeploy.getScopeSerializerId,
|
||||||
|
protoDeploy.getScopeManifest).get.asInstanceOf[Scope]
|
||||||
|
} else {
|
||||||
|
// old wire format
|
||||||
|
oldDeserialize(protoDeploy.getScope, classOf[Scope])
|
||||||
|
}
|
||||||
|
} else NoScopeGiven
|
||||||
val dispatcher =
|
val dispatcher =
|
||||||
if (protoDeploy.hasDispatcher) protoDeploy.getDispatcher
|
if (protoDeploy.hasDispatcher) protoDeploy.getDispatcher
|
||||||
else NoDispatcherGiven
|
else NoDispatcherGiven
|
||||||
|
|
|
||||||
|
|
@ -3,12 +3,21 @@
|
||||||
*/
|
*/
|
||||||
package akka.remote.serialization
|
package akka.remote.serialization
|
||||||
|
|
||||||
import akka.actor._
|
|
||||||
import akka.protobuf.ByteString
|
|
||||||
import akka.remote.{ ContainerFormats, RemoteWatcher }
|
|
||||||
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
|
||||||
import java.util.Optional
|
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
import java.nio.charset.StandardCharsets
|
||||||
|
import java.util.Optional
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
|
import akka.actor._
|
||||||
|
import akka.dispatch.Dispatchers
|
||||||
|
import akka.remote.routing.RemoteRouterConfig
|
||||||
|
import akka.remote.{ ContainerFormats, RemoteScope, RemoteWatcher, WireFormats }
|
||||||
|
import akka.routing._
|
||||||
|
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
||||||
|
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
import scala.concurrent.duration.{ FiniteDuration, TimeUnit }
|
||||||
|
|
||||||
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
||||||
|
|
||||||
|
|
@ -18,6 +27,7 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
private val throwableSupport = new ThrowableSupport(system)
|
private val throwableSupport = new ThrowableSupport(system)
|
||||||
|
|
||||||
private val ParameterlessSerializedMessage = Array.empty[Byte]
|
private val ParameterlessSerializedMessage = Array.empty[Byte]
|
||||||
|
private val EmptyConfig = ConfigFactory.empty()
|
||||||
|
|
||||||
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||||
case identify: Identify ⇒ serializeIdentify(identify)
|
case identify: Identify ⇒ serializeIdentify(identify)
|
||||||
|
|
@ -34,6 +44,18 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
case Kill ⇒ ParameterlessSerializedMessage
|
case Kill ⇒ ParameterlessSerializedMessage
|
||||||
case RemoteWatcher.Heartbeat ⇒ ParameterlessSerializedMessage
|
case RemoteWatcher.Heartbeat ⇒ ParameterlessSerializedMessage
|
||||||
case hbrsp: RemoteWatcher.HeartbeatRsp ⇒ serializeHeartbeatRsp(hbrsp)
|
case hbrsp: RemoteWatcher.HeartbeatRsp ⇒ serializeHeartbeatRsp(hbrsp)
|
||||||
|
case rs: RemoteScope ⇒ serializeRemoteScope(rs)
|
||||||
|
case LocalScope ⇒ ParameterlessSerializedMessage
|
||||||
|
case c: Config ⇒ serializeConfig(c)
|
||||||
|
case dr: DefaultResizer ⇒ serializeDefaultResizer(dr)
|
||||||
|
case fc: FromConfig ⇒ serializeFromConfig(fc)
|
||||||
|
case bp: BalancingPool ⇒ serializeBalancingPool(bp)
|
||||||
|
case bp: BroadcastPool ⇒ serializeBroadcastPool(bp)
|
||||||
|
case rp: RandomPool ⇒ serializeRandomPool(rp)
|
||||||
|
case rrp: RoundRobinPool ⇒ serializeRoundRobinPool(rrp)
|
||||||
|
case sgp: ScatterGatherFirstCompletedPool ⇒ serializeScatterGatherFirstCompletedPool(sgp)
|
||||||
|
case tp: TailChoppingPool ⇒ serializeTailChoppingPool(tp)
|
||||||
|
case rrc: RemoteRouterConfig ⇒ serializeRemoteRouterConfig(rrc)
|
||||||
case _ ⇒ throw new IllegalArgumentException(s"Cannot serialize object of type [${obj.getClass.getName}]")
|
case _ ⇒ throw new IllegalArgumentException(s"Cannot serialize object of type [${obj.getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -80,6 +102,12 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
ContainerFormats.WatcherHeartbeatResponse.newBuilder().setUid(hbrsp.addressUid).build().toByteArray
|
ContainerFormats.WatcherHeartbeatResponse.newBuilder().setUid(hbrsp.addressUid).build().toByteArray
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def serializeRemoteScope(rs: RemoteScope): Array[Byte] = {
|
||||||
|
val builder = WireFormats.RemoteScope.newBuilder()
|
||||||
|
builder.setNode(buildAddressData(rs.node))
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
private def actorRefBuilder(actorRef: ActorRef): ContainerFormats.ActorRef.Builder =
|
private def actorRefBuilder(actorRef: ActorRef): ContainerFormats.ActorRef.Builder =
|
||||||
ContainerFormats.ActorRef.newBuilder()
|
ContainerFormats.ActorRef.newBuilder()
|
||||||
.setPath(Serialization.serializedActorPath(actorRef))
|
.setPath(Serialization.serializedActorPath(actorRef))
|
||||||
|
|
@ -101,6 +129,119 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
.build().toByteArray
|
.build().toByteArray
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def serializeConfig(c: Config): Array[Byte] = {
|
||||||
|
c.root.render(ConfigRenderOptions.concise()).getBytes(StandardCharsets.UTF_8)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeDefaultResizer(dr: DefaultResizer): Array[Byte] = {
|
||||||
|
val builder = WireFormats.DefaultResizer.newBuilder()
|
||||||
|
builder.setBackoffRate(dr.backoffRate)
|
||||||
|
builder.setBackoffThreshold(dr.backoffThreshold)
|
||||||
|
builder.setLowerBound(dr.lowerBound)
|
||||||
|
builder.setMessagesPerResize(dr.messagesPerResize)
|
||||||
|
builder.setPressureThreshold(dr.pressureThreshold)
|
||||||
|
builder.setRampupRate(dr.rampupRate)
|
||||||
|
builder.setUpperBound(dr.upperBound)
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeFromConfig(fc: FromConfig): Array[Byte] =
|
||||||
|
if (fc == FromConfig) ParameterlessSerializedMessage
|
||||||
|
else {
|
||||||
|
val builder = WireFormats.FromConfig.newBuilder()
|
||||||
|
if (fc.resizer.isDefined)
|
||||||
|
builder.setResizer(payloadSupport.payloadBuilder(fc.resizer.get))
|
||||||
|
if (fc.routerDispatcher != Dispatchers.DefaultDispatcherId)
|
||||||
|
builder.setRouterDispatcher(fc.routerDispatcher)
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeBalancingPool(bp: BalancingPool): Array[Byte] = {
|
||||||
|
buildGenericRoutingPool(bp.nrOfInstances, bp.routerDispatcher, bp.usePoolDispatcher, bp.resizer).toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeBroadcastPool(bp: BroadcastPool): Array[Byte] = {
|
||||||
|
buildGenericRoutingPool(bp.nrOfInstances, bp.routerDispatcher, bp.usePoolDispatcher, bp.resizer).toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeRandomPool(rp: RandomPool): Array[Byte] = {
|
||||||
|
buildGenericRoutingPool(rp.nrOfInstances, rp.routerDispatcher, rp.usePoolDispatcher, rp.resizer).toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeRoundRobinPool(rp: RoundRobinPool): Array[Byte] = {
|
||||||
|
buildGenericRoutingPool(rp.nrOfInstances, rp.routerDispatcher, rp.usePoolDispatcher, rp.resizer).toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeScatterGatherFirstCompletedPool(sgp: ScatterGatherFirstCompletedPool): Array[Byte] = {
|
||||||
|
val builder = WireFormats.ScatterGatherPool.newBuilder()
|
||||||
|
builder.setGeneric(buildGenericRoutingPool(sgp.nrOfInstances, sgp.routerDispatcher, sgp.usePoolDispatcher, sgp.resizer))
|
||||||
|
builder.setWithin(buildFiniteDuration(sgp.within))
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeTailChoppingPool(tp: TailChoppingPool): Array[Byte] = {
|
||||||
|
val builder = WireFormats.TailChoppingPool.newBuilder()
|
||||||
|
builder.setGeneric(buildGenericRoutingPool(tp.nrOfInstances, tp.routerDispatcher, tp.usePoolDispatcher, tp.resizer))
|
||||||
|
builder.setWithin(buildFiniteDuration(tp.within))
|
||||||
|
builder.setInterval(buildFiniteDuration(tp.interval))
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def serializeRemoteRouterConfig(rrc: RemoteRouterConfig): Array[Byte] = {
|
||||||
|
val builder = WireFormats.RemoteRouterConfig.newBuilder()
|
||||||
|
builder.setLocal(payloadSupport.payloadBuilder(rrc.local).build())
|
||||||
|
builder.addAllNodes(rrc.nodes.map(buildAddressData).asJava)
|
||||||
|
builder.build().toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
private def buildGenericRoutingPool(
|
||||||
|
nrOfInstances: Int,
|
||||||
|
routerDispatcher: String,
|
||||||
|
usePoolDispatcher: Boolean,
|
||||||
|
resizer: Option[Resizer]): WireFormats.GenericRoutingPool = {
|
||||||
|
val builder = WireFormats.GenericRoutingPool.newBuilder()
|
||||||
|
builder.setNrOfInstances(nrOfInstances)
|
||||||
|
if (routerDispatcher != Dispatchers.DefaultDispatcherId) {
|
||||||
|
builder.setRouterDispatcher(routerDispatcher)
|
||||||
|
}
|
||||||
|
if (resizer.isDefined) {
|
||||||
|
builder.setResizer(payloadSupport.payloadBuilder(resizer.get))
|
||||||
|
}
|
||||||
|
builder.setUsePoolDispatcher(usePoolDispatcher)
|
||||||
|
builder.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
private def timeUnitToWire(unit: TimeUnit): WireFormats.TimeUnit = unit match {
|
||||||
|
case TimeUnit.NANOSECONDS ⇒ WireFormats.TimeUnit.NANOSECONDS
|
||||||
|
case TimeUnit.MICROSECONDS ⇒ WireFormats.TimeUnit.MICROSECONDS
|
||||||
|
case TimeUnit.MILLISECONDS ⇒ WireFormats.TimeUnit.MILLISECONDS
|
||||||
|
case TimeUnit.SECONDS ⇒ WireFormats.TimeUnit.SECONDS
|
||||||
|
case TimeUnit.MINUTES ⇒ WireFormats.TimeUnit.MINUTES
|
||||||
|
case TimeUnit.HOURS ⇒ WireFormats.TimeUnit.HOURS
|
||||||
|
case TimeUnit.DAYS ⇒ WireFormats.TimeUnit.DAYS
|
||||||
|
}
|
||||||
|
|
||||||
|
private def buildFiniteDuration(duration: FiniteDuration): WireFormats.FiniteDuration = {
|
||||||
|
WireFormats.FiniteDuration.newBuilder()
|
||||||
|
.setValue(duration.length)
|
||||||
|
.setUnit(timeUnitToWire(duration.unit))
|
||||||
|
.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
private def buildAddressData(address: Address): WireFormats.AddressData = {
|
||||||
|
val builder = WireFormats.AddressData.newBuilder()
|
||||||
|
address match {
|
||||||
|
case Address(protocol, system, Some(host), Some(port)) ⇒
|
||||||
|
builder.setProtocol(protocol)
|
||||||
|
builder.setSystem(system)
|
||||||
|
builder.setHostname(host)
|
||||||
|
builder.setPort(port)
|
||||||
|
builder.build()
|
||||||
|
|
||||||
|
case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private val IdentifyManifest = "A"
|
private val IdentifyManifest = "A"
|
||||||
private val ActorIdentityManifest = "B"
|
private val ActorIdentityManifest = "B"
|
||||||
private val OptionManifest = "C"
|
private val OptionManifest = "C"
|
||||||
|
|
@ -114,6 +255,18 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
private val RemoteWatcherHBManifest = "RWHB"
|
private val RemoteWatcherHBManifest = "RWHB"
|
||||||
private val RemoteWatcherHBRespManifest = "RWHR"
|
private val RemoteWatcherHBRespManifest = "RWHR"
|
||||||
private val ActorInitializationExceptionManifest = "AIEX"
|
private val ActorInitializationExceptionManifest = "AIEX"
|
||||||
|
private val LocalScopeManifest = "LS"
|
||||||
|
private val RemoteScopeManifest = "RS"
|
||||||
|
private val ConfigManifest = "CF"
|
||||||
|
private val FromConfigManifest = "FC"
|
||||||
|
private val DefaultResizerManifest = "DR"
|
||||||
|
private val BalancingPoolManifest = "ROBAP"
|
||||||
|
private val BroadcastPoolManifest = "ROBP"
|
||||||
|
private val RandomPoolManifest = "RORP"
|
||||||
|
private val RoundRobinPoolManifest = "RORRP"
|
||||||
|
private val ScatterGatherPoolManifest = "ROSGP"
|
||||||
|
private val TailChoppingPoolManifest = "ROTCP"
|
||||||
|
private val RemoteRouterConfigManifest = "RORRC"
|
||||||
|
|
||||||
private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef](
|
private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef](
|
||||||
IdentifyManifest → deserializeIdentify,
|
IdentifyManifest → deserializeIdentify,
|
||||||
|
|
@ -129,7 +282,20 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
KillManifest → ((_) ⇒ Kill),
|
KillManifest → ((_) ⇒ Kill),
|
||||||
RemoteWatcherHBManifest → ((_) ⇒ RemoteWatcher.Heartbeat),
|
RemoteWatcherHBManifest → ((_) ⇒ RemoteWatcher.Heartbeat),
|
||||||
RemoteWatcherHBRespManifest → deserializeHeartbeatRsp,
|
RemoteWatcherHBRespManifest → deserializeHeartbeatRsp,
|
||||||
ActorInitializationExceptionManifest → deserializeActorInitializationException)
|
ActorInitializationExceptionManifest → deserializeActorInitializationException,
|
||||||
|
LocalScopeManifest → ((_) ⇒ LocalScope),
|
||||||
|
RemoteScopeManifest → deserializeRemoteScope,
|
||||||
|
ConfigManifest → deserializeConfig,
|
||||||
|
FromConfigManifest → deserializeFromConfig,
|
||||||
|
DefaultResizerManifest → deserializeDefaultResizer,
|
||||||
|
BalancingPoolManifest → deserializeBalancingPool,
|
||||||
|
BroadcastPoolManifest → deserializeBroadcastPool,
|
||||||
|
RandomPoolManifest → deserializeRandomPool,
|
||||||
|
RoundRobinPoolManifest → deserializeRoundRobinPool,
|
||||||
|
ScatterGatherPoolManifest → deserializeScatterGatherPool,
|
||||||
|
TailChoppingPoolManifest → deserializeTailChoppingPool,
|
||||||
|
RemoteRouterConfigManifest → deserializeRemoteRouterConfig
|
||||||
|
)
|
||||||
|
|
||||||
override def manifest(o: AnyRef): String =
|
override def manifest(o: AnyRef): String =
|
||||||
o match {
|
o match {
|
||||||
|
|
@ -146,6 +312,18 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
case Kill ⇒ KillManifest
|
case Kill ⇒ KillManifest
|
||||||
case RemoteWatcher.Heartbeat ⇒ RemoteWatcherHBManifest
|
case RemoteWatcher.Heartbeat ⇒ RemoteWatcherHBManifest
|
||||||
case _: RemoteWatcher.HeartbeatRsp ⇒ RemoteWatcherHBRespManifest
|
case _: RemoteWatcher.HeartbeatRsp ⇒ RemoteWatcherHBRespManifest
|
||||||
|
case LocalScope ⇒ LocalScopeManifest
|
||||||
|
case _: RemoteScope ⇒ RemoteScopeManifest
|
||||||
|
case _: Config ⇒ ConfigManifest
|
||||||
|
case _: FromConfig ⇒ FromConfigManifest
|
||||||
|
case _: DefaultResizer ⇒ DefaultResizerManifest
|
||||||
|
case _: BalancingPool ⇒ BalancingPoolManifest
|
||||||
|
case _: BroadcastPool ⇒ BroadcastPoolManifest
|
||||||
|
case _: RandomPool ⇒ RandomPoolManifest
|
||||||
|
case _: RoundRobinPool ⇒ RoundRobinPoolManifest
|
||||||
|
case _: ScatterGatherFirstCompletedPool ⇒ ScatterGatherPoolManifest
|
||||||
|
case _: TailChoppingPool ⇒ TailChoppingPoolManifest
|
||||||
|
case _: RemoteRouterConfig ⇒ RemoteRouterConfigManifest
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]")
|
throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]")
|
||||||
}
|
}
|
||||||
|
|
@ -224,4 +402,142 @@ class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerW
|
||||||
payloadSupport.deserializePayload(serializedEx.getCause).asInstanceOf[Throwable])
|
payloadSupport.deserializePayload(serializedEx.getCause).asInstanceOf[Throwable])
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def deserializeRemoteScope(bytes: Array[Byte]): RemoteScope = {
|
||||||
|
val rs = WireFormats.RemoteScope.parseFrom(bytes)
|
||||||
|
RemoteScope(
|
||||||
|
deserializeAddressData(rs.getNode)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeConfig(bytes: Array[Byte]): Config = {
|
||||||
|
if (bytes.isEmpty) EmptyConfig
|
||||||
|
else ConfigFactory.parseString(new String(bytes, StandardCharsets.UTF_8))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeFromConfig(bytes: Array[Byte]): FromConfig =
|
||||||
|
if (bytes.isEmpty) FromConfig
|
||||||
|
else {
|
||||||
|
val fc = WireFormats.FromConfig.parseFrom(bytes)
|
||||||
|
FromConfig(
|
||||||
|
resizer = if (fc.hasResizer) Some(payloadSupport.deserializePayload(fc.getResizer).asInstanceOf[Resizer]) else None,
|
||||||
|
routerDispatcher = if (fc.hasRouterDispatcher) fc.getRouterDispatcher else Dispatchers.DefaultDispatcherId
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeBalancingPool(bytes: Array[Byte]): BalancingPool = {
|
||||||
|
val bp = WireFormats.GenericRoutingPool.parseFrom(bytes)
|
||||||
|
BalancingPool(
|
||||||
|
nrOfInstances = bp.getNrOfInstances,
|
||||||
|
routerDispatcher = if (bp.hasRouterDispatcher) bp.getRouterDispatcher else Dispatchers.DefaultDispatcherId)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeBroadcastPool(bytes: Array[Byte]): BroadcastPool = {
|
||||||
|
val bp = WireFormats.GenericRoutingPool.parseFrom(bytes)
|
||||||
|
BroadcastPool(
|
||||||
|
nrOfInstances = bp.getNrOfInstances,
|
||||||
|
resizer =
|
||||||
|
if (bp.hasResizer) Some(payloadSupport.deserializePayload(bp.getResizer).asInstanceOf[Resizer])
|
||||||
|
else None,
|
||||||
|
routerDispatcher = if (bp.hasRouterDispatcher) bp.getRouterDispatcher else Dispatchers.DefaultDispatcherId,
|
||||||
|
usePoolDispatcher = bp.getUsePoolDispatcher
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeRandomPool(bytes: Array[Byte]): RandomPool = {
|
||||||
|
val rp = WireFormats.GenericRoutingPool.parseFrom(bytes)
|
||||||
|
RandomPool(
|
||||||
|
nrOfInstances = rp.getNrOfInstances,
|
||||||
|
resizer =
|
||||||
|
if (rp.hasResizer) Some(payloadSupport.deserializePayload(rp.getResizer).asInstanceOf[Resizer])
|
||||||
|
else None,
|
||||||
|
routerDispatcher = if (rp.hasRouterDispatcher) rp.getRouterDispatcher else Dispatchers.DefaultDispatcherId,
|
||||||
|
usePoolDispatcher = rp.getUsePoolDispatcher
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeRoundRobinPool(bytes: Array[Byte]): RoundRobinPool = {
|
||||||
|
val rp = WireFormats.GenericRoutingPool.parseFrom(bytes)
|
||||||
|
RoundRobinPool(
|
||||||
|
nrOfInstances = rp.getNrOfInstances,
|
||||||
|
resizer =
|
||||||
|
if (rp.hasResizer) Some(payloadSupport.deserializePayload(rp.getResizer).asInstanceOf[Resizer])
|
||||||
|
else None,
|
||||||
|
routerDispatcher = if (rp.hasRouterDispatcher) rp.getRouterDispatcher else Dispatchers.DefaultDispatcherId,
|
||||||
|
usePoolDispatcher = rp.getUsePoolDispatcher
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeScatterGatherPool(bytes: Array[Byte]): ScatterGatherFirstCompletedPool = {
|
||||||
|
val sgp = WireFormats.ScatterGatherPool.parseFrom(bytes)
|
||||||
|
ScatterGatherFirstCompletedPool(
|
||||||
|
nrOfInstances = sgp.getGeneric.getNrOfInstances,
|
||||||
|
resizer =
|
||||||
|
if (sgp.getGeneric.hasResizer) Some(payloadSupport.deserializePayload(sgp.getGeneric.getResizer).asInstanceOf[Resizer])
|
||||||
|
else None,
|
||||||
|
within = deserializeFiniteDuration(sgp.getWithin),
|
||||||
|
routerDispatcher =
|
||||||
|
if (sgp.getGeneric.hasRouterDispatcher) sgp.getGeneric.getRouterDispatcher
|
||||||
|
else Dispatchers.DefaultDispatcherId
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeTailChoppingPool(bytes: Array[Byte]): TailChoppingPool = {
|
||||||
|
val tcp = WireFormats.TailChoppingPool.parseFrom(bytes)
|
||||||
|
TailChoppingPool(
|
||||||
|
nrOfInstances = tcp.getGeneric.getNrOfInstances,
|
||||||
|
resizer =
|
||||||
|
if (tcp.getGeneric.hasResizer) Some(payloadSupport.deserializePayload(tcp.getGeneric.getResizer).asInstanceOf[Resizer])
|
||||||
|
else None,
|
||||||
|
routerDispatcher = if (tcp.getGeneric.hasRouterDispatcher) tcp.getGeneric.getRouterDispatcher else Dispatchers.DefaultDispatcherId,
|
||||||
|
usePoolDispatcher = tcp.getGeneric.getUsePoolDispatcher,
|
||||||
|
within = deserializeFiniteDuration(tcp.getWithin),
|
||||||
|
interval = deserializeFiniteDuration(tcp.getInterval)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeRemoteRouterConfig(bytes: Array[Byte]): RemoteRouterConfig = {
|
||||||
|
val rrc = WireFormats.RemoteRouterConfig.parseFrom(bytes)
|
||||||
|
RemoteRouterConfig(
|
||||||
|
local = payloadSupport.deserializePayload(rrc.getLocal).asInstanceOf[Pool],
|
||||||
|
nodes = rrc.getNodesList.asScala.map(deserializeAddressData)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeDefaultResizer(bytes: Array[Byte]): DefaultResizer = {
|
||||||
|
val dr = WireFormats.DefaultResizer.parseFrom(bytes)
|
||||||
|
DefaultResizer(
|
||||||
|
lowerBound = dr.getLowerBound,
|
||||||
|
upperBound = dr.getUpperBound,
|
||||||
|
pressureThreshold = dr.getPressureThreshold,
|
||||||
|
rampupRate = dr.getRampupRate,
|
||||||
|
backoffThreshold = dr.getBackoffThreshold,
|
||||||
|
backoffRate = dr.getBackoffRate,
|
||||||
|
messagesPerResize = dr.getMessagesPerResize
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeTimeUnit(unit: WireFormats.TimeUnit): TimeUnit = unit match {
|
||||||
|
case WireFormats.TimeUnit.NANOSECONDS ⇒ TimeUnit.NANOSECONDS
|
||||||
|
case WireFormats.TimeUnit.MICROSECONDS ⇒ TimeUnit.MICROSECONDS
|
||||||
|
case WireFormats.TimeUnit.MILLISECONDS ⇒ TimeUnit.MILLISECONDS
|
||||||
|
case WireFormats.TimeUnit.SECONDS ⇒ TimeUnit.SECONDS
|
||||||
|
case WireFormats.TimeUnit.MINUTES ⇒ TimeUnit.MINUTES
|
||||||
|
case WireFormats.TimeUnit.HOURS ⇒ TimeUnit.HOURS
|
||||||
|
case WireFormats.TimeUnit.DAYS ⇒ TimeUnit.DAYS
|
||||||
|
}
|
||||||
|
|
||||||
|
private def deserializeFiniteDuration(duration: WireFormats.FiniteDuration): FiniteDuration =
|
||||||
|
FiniteDuration(
|
||||||
|
duration.getValue,
|
||||||
|
deserializeTimeUnit(duration.getUnit)
|
||||||
|
)
|
||||||
|
|
||||||
|
private def deserializeAddressData(address: WireFormats.AddressData): Address = {
|
||||||
|
Address(
|
||||||
|
address.getProtocol,
|
||||||
|
address.getSystem,
|
||||||
|
address.getHostname,
|
||||||
|
address.getPort
|
||||||
|
)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,8 +20,10 @@ object RemoteRouterSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class RemoteRouterSpec extends AkkaSpec("""
|
class RemoteRouterSpec extends AkkaSpec(s"""
|
||||||
akka.actor.provider = remote
|
akka.actor.provider = remote
|
||||||
|
akka.actor.allow-java-serialization = off
|
||||||
|
akka.actor.serialize-messages = off
|
||||||
akka.remote.netty.tcp {
|
akka.remote.netty.tcp {
|
||||||
hostname = localhost
|
hostname = localhost
|
||||||
port = 0
|
port = 0
|
||||||
|
|
@ -45,6 +47,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
||||||
|
|
||||||
val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get
|
val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get
|
||||||
val sysName = system.name
|
val sysName = system.name
|
||||||
|
val masterSystemName = "Master" + sysName
|
||||||
val protocol =
|
val protocol =
|
||||||
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
||||||
else "akka.tcp"
|
else "akka.tcp"
|
||||||
|
|
@ -71,7 +74,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
||||||
nr-of-instances = 2
|
nr-of-instances = 2
|
||||||
}
|
}
|
||||||
/local-blub {
|
/local-blub {
|
||||||
remote = "akka://MasterRemoteRouterSpec"
|
remote = "akka://$masterSystemName"
|
||||||
router = round-robin-pool
|
router = round-robin-pool
|
||||||
nr-of-instances = 2
|
nr-of-instances = 2
|
||||||
target.nodes = ["$protocol://${sysName}@localhost:${port}"]
|
target.nodes = ["$protocol://${sysName}@localhost:${port}"]
|
||||||
|
|
@ -83,7 +86,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}""").withFallback(system.settings.config)
|
}""").withFallback(system.settings.config)
|
||||||
val masterSystem = ActorSystem("Master" + sysName, conf)
|
val masterSystem = ActorSystem(masterSystemName, conf)
|
||||||
|
|
||||||
override def afterTermination() {
|
override def afterTermination() {
|
||||||
shutdown(masterSystem)
|
shutdown(masterSystem)
|
||||||
|
|
@ -168,7 +171,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
||||||
val probe = TestProbe()(masterSystem)
|
val probe = TestProbe()(masterSystem)
|
||||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
|
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
|
||||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub")
|
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub")
|
||||||
router.path.address.toString should ===("akka://MasterRemoteRouterSpec")
|
router.path.address.toString should ===(s"akka://$masterSystemName")
|
||||||
val replies = collectRouteePaths(probe, router, 5)
|
val replies = collectRouteePaths(probe, router, 5)
|
||||||
val children = replies.toSet
|
val children = replies.toSet
|
||||||
children should have size 2
|
children should have size 2
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,14 @@
|
||||||
|
|
||||||
package akka.remote.serialization
|
package akka.remote.serialization
|
||||||
|
|
||||||
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, ExtendedActorSystem, OneForOneStrategy, Props, SupervisorStrategy }
|
import akka.actor.ActorSystem
|
||||||
|
import akka.testkit.TestKit
|
||||||
|
import akka.actor.{ Actor, ActorRef, Address, Deploy, ExtendedActorSystem, OneForOneStrategy, Props, SupervisorStrategy }
|
||||||
import akka.remote.{ DaemonMsgCreate, RemoteScope }
|
import akka.remote.{ DaemonMsgCreate, RemoteScope }
|
||||||
import akka.routing.{ FromConfig, RoundRobinPool }
|
import akka.routing.{ FromConfig, RoundRobinPool }
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.{ Serialization, SerializationExtension }
|
||||||
import akka.testkit.{ AkkaSpec, TestKit }
|
import akka.testkit.AkkaSpec
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.language.postfixOps
|
import scala.language.postfixOps
|
||||||
|
|
||||||
|
|
@ -27,7 +28,32 @@ object DaemonMsgCreateSerializerSpec {
|
||||||
|
|
||||||
case class DummyParameter(val inner: String) extends Serializable
|
case class DummyParameter(val inner: String) extends Serializable
|
||||||
|
|
||||||
class DaemonMsgCreateSerializerSpec extends AkkaSpec {
|
trait SerializationVerification { self: AkkaSpec ⇒
|
||||||
|
|
||||||
|
def ser: Serialization
|
||||||
|
|
||||||
|
def verifySerialization(msg: DaemonMsgCreate): Unit = {
|
||||||
|
assertDaemonMsgCreate(msg, ser.deserialize(ser.serialize(msg).get, classOf[DaemonMsgCreate]).get.asInstanceOf[DaemonMsgCreate])
|
||||||
|
}
|
||||||
|
|
||||||
|
def assertDaemonMsgCreate(expected: DaemonMsgCreate, got: DaemonMsgCreate): Unit = {
|
||||||
|
// can't compare props.creator when function
|
||||||
|
got.props.clazz should ===(expected.props.clazz)
|
||||||
|
got.props.args.length should ===(expected.props.args.length)
|
||||||
|
got.props.args zip expected.props.args foreach {
|
||||||
|
case (g, e) ⇒
|
||||||
|
if (e.isInstanceOf[Function0[_]]) ()
|
||||||
|
else if (e.isInstanceOf[Function1[_, _]]) ()
|
||||||
|
else g should ===(e)
|
||||||
|
}
|
||||||
|
got.props.deploy should ===(expected.props.deploy)
|
||||||
|
got.deploy should ===(expected.deploy)
|
||||||
|
got.path should ===(expected.path)
|
||||||
|
got.supervisor should ===(expected.supervisor)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class DaemonMsgCreateSerializerSpec extends AkkaSpec with SerializationVerification {
|
||||||
|
|
||||||
import DaemonMsgCreateSerializerSpec._
|
import DaemonMsgCreateSerializerSpec._
|
||||||
val ser = SerializationExtension(system)
|
val ser = SerializationExtension(system)
|
||||||
|
|
@ -126,9 +152,8 @@ class DaemonMsgCreateSerializerSpec extends AkkaSpec {
|
||||||
"serialize and de-serialize DaemonMsgCreate with Deploy and RouterConfig" in {
|
"serialize and de-serialize DaemonMsgCreate with Deploy and RouterConfig" in {
|
||||||
verifySerialization {
|
verifySerialization {
|
||||||
// Duration.Inf doesn't equal Duration.Inf, so we use another for test
|
// Duration.Inf doesn't equal Duration.Inf, so we use another for test
|
||||||
val supervisorStrategy = OneForOneStrategy(3, 10 seconds) {
|
// we don't serialize the supervisor strategy, but always fallback to default
|
||||||
case _ ⇒ SupervisorStrategy.Escalate
|
val supervisorStrategy = SupervisorStrategy.defaultStrategy
|
||||||
}
|
|
||||||
val deploy1 = Deploy(
|
val deploy1 = Deploy(
|
||||||
path = "path1",
|
path = "path1",
|
||||||
config = ConfigFactory.parseString("a=1"),
|
config = ConfigFactory.parseString("a=1"),
|
||||||
|
|
@ -160,26 +185,47 @@ class DaemonMsgCreateSerializerSpec extends AkkaSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def verifySerialization(msg: DaemonMsgCreate): Unit = {
|
|
||||||
assertDaemonMsgCreate(msg, ser.deserialize(ser.serialize(msg).get, classOf[DaemonMsgCreate]).get.asInstanceOf[DaemonMsgCreate])
|
|
||||||
}
|
|
||||||
|
|
||||||
def assertDaemonMsgCreate(expected: DaemonMsgCreate, got: DaemonMsgCreate): Unit = {
|
|
||||||
// can't compare props.creator when function
|
|
||||||
got.props.clazz should ===(expected.props.clazz)
|
|
||||||
got.props.args.length should ===(expected.props.args.length)
|
|
||||||
got.props.args zip expected.props.args foreach {
|
|
||||||
case (g, e) ⇒
|
|
||||||
if (e.isInstanceOf[Function0[_]]) ()
|
|
||||||
else if (e.isInstanceOf[Function1[_, _]]) ()
|
|
||||||
else g should ===(e)
|
|
||||||
}
|
|
||||||
got.props.deploy should ===(expected.props.deploy)
|
|
||||||
got.deploy should ===(expected.deploy)
|
|
||||||
got.path should ===(expected.path)
|
|
||||||
got.supervisor should ===(expected.supervisor)
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class DaemonMsgCreateSerializerNoJavaSerializationSpec extends AkkaSpec(
|
||||||
|
"""
|
||||||
|
akka.actor.allow-java-serialization=off
|
||||||
|
akka.actor.serialize-messages=off
|
||||||
|
akka.actor.serialize-creators=off
|
||||||
|
""") with SerializationVerification {
|
||||||
|
|
||||||
|
import DaemonMsgCreateSerializerSpec.MyActor
|
||||||
|
|
||||||
|
val supervisor = system.actorOf(Props[MyActor], "supervisor")
|
||||||
|
val ser = SerializationExtension(system)
|
||||||
|
|
||||||
|
"serialize and de-serialize DaemonMsgCreate with Deploy and RouterConfig" in {
|
||||||
|
verifySerialization {
|
||||||
|
// Duration.Inf doesn't equal Duration.Inf, so we use another for test
|
||||||
|
val supervisorStrategy = OneForOneStrategy(3, 10 seconds) {
|
||||||
|
case _ ⇒ SupervisorStrategy.Escalate
|
||||||
|
}
|
||||||
|
|
||||||
|
val deploy1 = Deploy(
|
||||||
|
path = "path1",
|
||||||
|
config = ConfigFactory.parseString("a=1"),
|
||||||
|
// a whole can of worms: routerConfig = RoundRobinPool(nrOfInstances = 5, supervisorStrategy = supervisorStrategy),
|
||||||
|
scope = RemoteScope(Address("akka", "Test", "host1", 1921)),
|
||||||
|
dispatcher = "mydispatcher")
|
||||||
|
val deploy2 = Deploy(
|
||||||
|
path = "path2",
|
||||||
|
config = ConfigFactory.parseString("a=2"),
|
||||||
|
routerConfig = FromConfig,
|
||||||
|
scope = RemoteScope(Address("akka", "Test", "host2", 1922)),
|
||||||
|
dispatcher = Deploy.NoDispatcherGiven)
|
||||||
|
DaemonMsgCreate(
|
||||||
|
props = Props[MyActor].withDispatcher("my-disp").withDeploy(deploy1),
|
||||||
|
deploy = deploy2,
|
||||||
|
path = "foo",
|
||||||
|
supervisor = supervisor)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,15 +5,19 @@
|
||||||
package akka.remote.serialization
|
package akka.remote.serialization
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.remote.{ MessageSerializer, RemoteWatcher }
|
import akka.remote.{ MessageSerializer, RemoteScope, RemoteWatcher }
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
import scala.concurrent.duration._
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
|
import akka.remote.routing.RemoteRouterConfig
|
||||||
|
import akka.routing._
|
||||||
|
|
||||||
object MiscMessageSerializerSpec {
|
object MiscMessageSerializerSpec {
|
||||||
val serializationTestOverrides =
|
val serializationTestOverrides =
|
||||||
"""
|
"""
|
||||||
|
|
@ -80,7 +84,25 @@ class MiscMessageSerializerSpec extends AkkaSpec(MiscMessageSerializerSpec.testC
|
||||||
"Kill" → Kill,
|
"Kill" → Kill,
|
||||||
"PoisonPill" → PoisonPill,
|
"PoisonPill" → PoisonPill,
|
||||||
"RemoteWatcher.Heartbeat" → RemoteWatcher.Heartbeat,
|
"RemoteWatcher.Heartbeat" → RemoteWatcher.Heartbeat,
|
||||||
"RemoteWatcher.HertbeatRsp" → RemoteWatcher.HeartbeatRsp(65537)).foreach {
|
"RemoteWatcher.HertbeatRsp" → RemoteWatcher.HeartbeatRsp(65537),
|
||||||
|
"LocalScope" → LocalScope,
|
||||||
|
"RemoteScope" → RemoteScope(Address("akka", "system", "localhost", 2525)),
|
||||||
|
"Config" → system.settings.config,
|
||||||
|
"Empty Config" → ConfigFactory.empty(),
|
||||||
|
"FromConfig" → FromConfig,
|
||||||
|
// routers
|
||||||
|
"DefaultResizer" → DefaultResizer(),
|
||||||
|
"BalancingPool" → BalancingPool(nrOfInstances = 25),
|
||||||
|
"BalancingPool with custom dispatcher" → BalancingPool(nrOfInstances = 25, routerDispatcher = "my-dispatcher"),
|
||||||
|
"BroadcastPool" → BroadcastPool(nrOfInstances = 25),
|
||||||
|
"BroadcastPool with custom dispatcher and resizer" → BroadcastPool(nrOfInstances = 25, routerDispatcher = "my-dispatcher", usePoolDispatcher = true, resizer = Some(DefaultResizer())),
|
||||||
|
"RandomPool" → RandomPool(nrOfInstances = 25),
|
||||||
|
"RandomPool with custom dispatcher" → RandomPool(nrOfInstances = 25, routerDispatcher = "my-dispatcher"),
|
||||||
|
"RoundRobinPool" → RoundRobinPool(25),
|
||||||
|
"ScatterGatherFirstCompletedPool" → ScatterGatherFirstCompletedPool(25, within = 3.seconds),
|
||||||
|
"TailChoppingPool" → TailChoppingPool(25, within = 3.seconds, interval = 1.second),
|
||||||
|
"RemoteRouterConfig" → RemoteRouterConfig(local = RandomPool(25), nodes = List(Address("akka", "system", "localhost", 2525)))
|
||||||
|
).foreach {
|
||||||
case (scenario, item) ⇒
|
case (scenario, item) ⇒
|
||||||
s"resolve serializer for $scenario" in {
|
s"resolve serializer for $scenario" in {
|
||||||
val serializer = SerializationExtension(system)
|
val serializer = SerializationExtension(system)
|
||||||
|
|
|
||||||
|
|
@ -27,7 +27,17 @@ akka {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
actor.serialization-bindings {
|
actor {
|
||||||
|
serializers {
|
||||||
|
test-message-serializer = "akka.testkit.TestMessageSerializer"
|
||||||
|
}
|
||||||
|
|
||||||
|
serialization-identifiers {
|
||||||
|
"akka.testkit.TestMessageSerializer" = 23
|
||||||
|
}
|
||||||
|
|
||||||
|
serialization-bindings {
|
||||||
"akka.testkit.JavaSerializable" = java
|
"akka.testkit.JavaSerializable" = java
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,45 @@
|
||||||
|
package akka.testkit
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
|
||||||
|
import java.util.concurrent.Callable
|
||||||
|
|
||||||
|
import akka.actor.ExtendedActorSystem
|
||||||
|
import akka.serialization.{ BaseSerializer, JavaSerializer }
|
||||||
|
import akka.util.ClassLoaderObjectInputStream
|
||||||
|
|
||||||
|
import scala.util.DynamicVariable
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This Serializer uses standard Java Serialization and is useful for tests where ad-hoc messages are created and sent
|
||||||
|
* between actor systems. It needs to be explicitly enabled in the config (or through `ActorSystemSetup`) like so:
|
||||||
|
*
|
||||||
|
* ```
|
||||||
|
* akka.actor.serialization-bindings {
|
||||||
|
* "my.test.AdHocMessage" = test-message-serializer
|
||||||
|
* }
|
||||||
|
* ```
|
||||||
|
*/
|
||||||
|
class TestMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
|
def includeManifest: Boolean = false
|
||||||
|
|
||||||
|
def toBinary(o: AnyRef): Array[Byte] = {
|
||||||
|
val bos = new ByteArrayOutputStream
|
||||||
|
val out = new ObjectOutputStream(bos)
|
||||||
|
JavaSerializer.currentSystem.withValue(system) { out.writeObject(o) }
|
||||||
|
out.close()
|
||||||
|
bos.toByteArray
|
||||||
|
}
|
||||||
|
|
||||||
|
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
|
||||||
|
val in = new ClassLoaderObjectInputStream(system.dynamicAccess.classLoader, new ByteArrayInputStream(bytes))
|
||||||
|
val obj = JavaSerializer.currentSystem.withValue(system) { in.readObject }
|
||||||
|
in.close()
|
||||||
|
obj
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
@ -464,41 +464,6 @@ object MiMa extends AutoPlugin {
|
||||||
// #22208 remove extension key
|
// #22208 remove extension key
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.event.Logging$Extension$"),
|
ProblemFilters.exclude[MissingClassProblem]("akka.event.Logging$Extension$"),
|
||||||
|
|
||||||
// #22224 DaemonMsgCreateSerializer using manifests
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData.getClassesBytes"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData.getClassesList"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData.getClassesCount"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData.getClasses"),
|
|
||||||
ProblemFilters.exclude[MissingFieldProblem]("akka.remote.WireFormats#PropsData.CLASSES_FIELD_NUMBER"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getHasManifest"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getHasManifestCount"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getSerializerIdsList"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getSerializerIds"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getHasManifestList"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getSerializerIdsCount"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getClassesBytes"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getClassesList"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getClassesCount"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getClasses"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getManifestsBytes"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getManifests"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getManifestsList"),
|
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#PropsDataOrBuilder.getManifestsCount"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.getClassesBytes"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.getClassesList"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.addClassesBytes"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.getClassesCount"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.clearClasses"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.addClasses"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.getClasses"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.addAllClasses"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.WireFormats#PropsData#Builder.setClasses"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.DaemonMsgCreateSerializer.serialize"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.DaemonMsgCreateSerializer.deserialize"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.DaemonMsgCreateSerializer.deserialize"),
|
|
||||||
ProblemFilters.exclude[FinalClassProblem]("akka.remote.serialization.DaemonMsgCreateSerializer"),
|
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.DaemonMsgCreateSerializer.serialization"),
|
|
||||||
|
|
||||||
// new materializer changes relating to old module structure
|
// new materializer changes relating to old module structure
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.BidiShape.copyFromPorts"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.BidiShape.copyFromPorts"),
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.BidiShape.reversed"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.stream.BidiShape.reversed"),
|
||||||
|
|
@ -1144,7 +1109,24 @@ object MiMa extends AutoPlugin {
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.Shard.messageBuffers_="),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.Shard.messageBuffers_="),
|
||||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.ShardRegion.totalBufferSize"),
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.sharding.ShardRegion.totalBufferSize"),
|
||||||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.sharding.ShardRegion.shardBuffers"),
|
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.sharding.ShardRegion.shardBuffers"),
|
||||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.ShardRegion.shardBuffers_=")
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.ShardRegion.shardBuffers_="),
|
||||||
|
|
||||||
|
// #22332 protobuf serializers for remote deployment
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getConfigManifest"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasScopeManifest"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getScopeManifestBytes"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getConfigSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasRouterConfigSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasRouterConfigManifest"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getRouterConfigSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getRouterConfigManifestBytes"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getConfigManifestBytes"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasConfigManifest"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasScopeSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getRouterConfigManifest"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.hasConfigSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getScopeSerializerId"),
|
||||||
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.remote.WireFormats#DeployDataOrBuilder.getScopeManifest")
|
||||||
)
|
)
|
||||||
// make sure that
|
// make sure that
|
||||||
// * this list ends with the latest released version number
|
// * this list ends with the latest released version number
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue