2016-04-28 23:33:59 +10:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
2016-04-28 23:33:59 +10:00
|
|
|
*/
|
2018-03-13 23:45:55 +09:00
|
|
|
|
2016-04-28 23:33:59 +10:00
|
|
|
package akka.remote.serialization
|
|
|
|
|
|
2017-03-16 15:12:35 +01:00
|
|
|
import java.io.NotSerializableException
|
|
|
|
|
import java.nio.charset.StandardCharsets
|
|
|
|
|
import java.util.Optional
|
|
|
|
|
import java.util.concurrent.TimeUnit
|
|
|
|
|
|
2018-03-05 17:38:17 +01:00
|
|
|
import akka.{ Done, NotUsed }
|
2016-04-28 23:33:59 +10:00
|
|
|
import akka.actor._
|
2017-03-16 15:12:35 +01:00
|
|
|
import akka.dispatch.Dispatchers
|
2018-01-10 19:28:51 +09:00
|
|
|
import akka.remote.WireFormats.AddressData
|
2017-03-16 15:12:35 +01:00
|
|
|
import akka.remote.routing.RemoteRouterConfig
|
2018-01-10 19:28:51 +09:00
|
|
|
import akka.remote._
|
2017-03-16 15:12:35 +01:00
|
|
|
import akka.routing._
|
2016-09-28 16:00:50 +02:00
|
|
|
import akka.serialization.{ BaseSerializer, Serialization, SerializationExtension, SerializerWithStringManifest }
|
2017-03-16 15:12:35 +01:00
|
|
|
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
|
|
|
|
|
|
|
|
|
|
import scala.collection.JavaConverters._
|
|
|
|
|
import scala.concurrent.duration.{ FiniteDuration, TimeUnit }
|
2016-04-28 23:33:59 +10:00
|
|
|
|
|
|
|
|
class MiscMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer {
|
|
|
|
|
|
2016-09-28 16:00:50 +02:00
|
|
|
// WARNING! This must lazy otherwise it will deadlock the ActorSystem creation
|
2016-04-28 23:33:59 +10:00
|
|
|
private lazy val serialization = SerializationExtension(system)
|
2016-09-28 11:14:33 +02:00
|
|
|
private val payloadSupport = new WrappedPayloadSupport(system)
|
|
|
|
|
private val throwableSupport = new ThrowableSupport(system)
|
2016-04-28 23:33:59 +10:00
|
|
|
|
2016-09-28 16:00:50 +02:00
|
|
|
private val ParameterlessSerializedMessage = Array.empty[Byte]
|
2017-03-16 15:12:35 +01:00
|
|
|
private val EmptyConfig = ConfigFactory.empty()
|
2016-06-19 17:55:10 +02:00
|
|
|
|
2016-04-28 23:33:59 +10:00
|
|
|
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
2017-03-16 15:12:35 +01:00
|
|
|
case identify: Identify ⇒ serializeIdentify(identify)
|
|
|
|
|
case identity: ActorIdentity ⇒ serializeActorIdentity(identity)
|
|
|
|
|
case Some(value) ⇒ serializeSome(value)
|
|
|
|
|
case None ⇒ ParameterlessSerializedMessage
|
|
|
|
|
case o: Optional[_] ⇒ serializeOptional(o)
|
|
|
|
|
case r: ActorRef ⇒ serializeActorRef(r)
|
|
|
|
|
case s: Status.Success ⇒ serializeStatusSuccess(s)
|
|
|
|
|
case f: Status.Failure ⇒ serializeStatusFailure(f)
|
|
|
|
|
case ex: ActorInitializationException ⇒ serializeActorInitializationException(ex)
|
|
|
|
|
case t: Throwable ⇒ throwableSupport.serializeThrowable(t)
|
|
|
|
|
case PoisonPill ⇒ ParameterlessSerializedMessage
|
|
|
|
|
case Kill ⇒ ParameterlessSerializedMessage
|
|
|
|
|
case RemoteWatcher.Heartbeat ⇒ ParameterlessSerializedMessage
|
2017-11-01 19:22:09 -04:00
|
|
|
case Done ⇒ ParameterlessSerializedMessage
|
2018-03-05 17:38:17 +01:00
|
|
|
case NotUsed ⇒ ParameterlessSerializedMessage
|
2017-03-16 15:12:35 +01:00
|
|
|
case hbrsp: RemoteWatcher.HeartbeatRsp ⇒ serializeHeartbeatRsp(hbrsp)
|
|
|
|
|
case rs: RemoteScope ⇒ serializeRemoteScope(rs)
|
|
|
|
|
case LocalScope ⇒ ParameterlessSerializedMessage
|
2018-01-10 19:28:51 +09:00
|
|
|
case a: Address ⇒ serializeAddressData(a)
|
|
|
|
|
case u: UniqueAddress ⇒ serializeClassicUniqueAddress(u)
|
2017-03-16 15:12:35 +01:00
|
|
|
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}]")
|
2016-04-28 23:33:59 +10:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def serializeIdentify(identify: Identify): Array[Byte] =
|
|
|
|
|
ContainerFormats.Identify.newBuilder()
|
2016-09-28 11:14:33 +02:00
|
|
|
.setMessageId(payloadSupport.payloadBuilder(identify.messageId))
|
2016-04-28 23:33:59 +10:00
|
|
|
.build()
|
|
|
|
|
.toByteArray
|
|
|
|
|
|
|
|
|
|
private def serializeActorIdentity(actorIdentity: ActorIdentity): Array[Byte] = {
|
|
|
|
|
val builder =
|
|
|
|
|
ContainerFormats.ActorIdentity.newBuilder()
|
2016-09-28 11:14:33 +02:00
|
|
|
.setCorrelationId(payloadSupport.payloadBuilder(actorIdentity.correlationId))
|
2016-04-28 23:33:59 +10:00
|
|
|
|
|
|
|
|
actorIdentity.ref.foreach { actorRef ⇒
|
|
|
|
|
builder.setRef(actorRefBuilder(actorRef))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
builder
|
|
|
|
|
.build()
|
|
|
|
|
.toByteArray
|
|
|
|
|
}
|
|
|
|
|
|
2016-06-19 17:55:10 +02:00
|
|
|
private def serializeSome(someValue: Any): Array[Byte] =
|
|
|
|
|
ContainerFormats.Option.newBuilder()
|
2016-09-28 11:14:33 +02:00
|
|
|
.setValue(payloadSupport.payloadBuilder(someValue))
|
2016-06-19 17:55:10 +02:00
|
|
|
.build()
|
|
|
|
|
.toByteArray
|
|
|
|
|
|
2016-12-01 16:51:39 +01:00
|
|
|
private def serializeOptional(opt: Optional[_]): Array[Byte] = {
|
|
|
|
|
if (opt.isPresent)
|
|
|
|
|
ContainerFormats.Option.newBuilder()
|
|
|
|
|
.setValue(payloadSupport.payloadBuilder(opt.get))
|
|
|
|
|
.build()
|
|
|
|
|
.toByteArray
|
|
|
|
|
else
|
|
|
|
|
ParameterlessSerializedMessage
|
|
|
|
|
}
|
|
|
|
|
|
2016-09-27 15:46:22 +02:00
|
|
|
private def serializeActorRef(ref: ActorRef): Array[Byte] =
|
|
|
|
|
actorRefBuilder(ref).build().toByteArray
|
|
|
|
|
|
2016-09-28 16:00:50 +02:00
|
|
|
private def serializeHeartbeatRsp(hbrsp: RemoteWatcher.HeartbeatRsp): Array[Byte] = {
|
|
|
|
|
ContainerFormats.WatcherHeartbeatResponse.newBuilder().setUid(hbrsp.addressUid).build().toByteArray
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-16 15:12:35 +01:00
|
|
|
private def serializeRemoteScope(rs: RemoteScope): Array[Byte] = {
|
|
|
|
|
val builder = WireFormats.RemoteScope.newBuilder()
|
|
|
|
|
builder.setNode(buildAddressData(rs.node))
|
|
|
|
|
builder.build().toByteArray
|
|
|
|
|
}
|
|
|
|
|
|
2016-04-28 23:33:59 +10:00
|
|
|
private def actorRefBuilder(actorRef: ActorRef): ContainerFormats.ActorRef.Builder =
|
|
|
|
|
ContainerFormats.ActorRef.newBuilder()
|
|
|
|
|
.setPath(Serialization.serializedActorPath(actorRef))
|
|
|
|
|
|
2016-09-28 11:14:33 +02:00
|
|
|
private def serializeStatusSuccess(success: Status.Success): Array[Byte] =
|
|
|
|
|
payloadSupport.payloadBuilder(success.status).build().toByteArray
|
2016-04-28 23:33:59 +10:00
|
|
|
|
2016-09-28 11:14:33 +02:00
|
|
|
private def serializeStatusFailure(failure: Status.Failure): Array[Byte] =
|
|
|
|
|
payloadSupport.payloadBuilder(failure.cause).build().toByteArray
|
2016-04-28 23:33:59 +10:00
|
|
|
|
2016-09-28 16:00:50 +02:00
|
|
|
private def serializeActorInitializationException(ex: ActorInitializationException): Array[Byte] = {
|
|
|
|
|
val builder = ContainerFormats.ActorInitializationException.newBuilder()
|
|
|
|
|
if (ex.getActor ne null)
|
|
|
|
|
builder.setActor(actorRefBuilder(ex.getActor))
|
|
|
|
|
|
|
|
|
|
builder
|
|
|
|
|
.setMessage(ex.getMessage)
|
|
|
|
|
.setCause(payloadSupport.payloadBuilder(ex.getCause))
|
|
|
|
|
.build().toByteArray
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-16 15:12:35 +01:00
|
|
|
private def serializeConfig(c: Config): Array[Byte] = {
|
|
|
|
|
c.root.render(ConfigRenderOptions.concise()).getBytes(StandardCharsets.UTF_8)
|
|
|
|
|
}
|
|
|
|
|
|
2018-01-10 19:28:51 +09:00
|
|
|
private def protoForAddressData(address: Address): AddressData.Builder =
|
|
|
|
|
address match {
|
|
|
|
|
case Address(protocol, actorSystem, Some(host), Some(port)) ⇒
|
|
|
|
|
WireFormats.AddressData.newBuilder()
|
|
|
|
|
.setSystem(actorSystem)
|
|
|
|
|
.setHostname(host)
|
|
|
|
|
.setPort(port)
|
|
|
|
|
.setProtocol(protocol)
|
|
|
|
|
case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
|
|
|
|
|
}
|
|
|
|
|
private def protoForAddress(address: Address): ArteryControlFormats.Address.Builder =
|
|
|
|
|
address match {
|
|
|
|
|
case Address(protocol, actorSystem, Some(host), Some(port)) ⇒
|
|
|
|
|
ArteryControlFormats.Address.newBuilder()
|
|
|
|
|
.setSystem(actorSystem)
|
|
|
|
|
.setHostname(host)
|
|
|
|
|
.setPort(port)
|
|
|
|
|
.setProtocol(protocol)
|
|
|
|
|
case _ ⇒ throw new IllegalArgumentException(s"Address [$address] could not be serialized: host or port missing.")
|
|
|
|
|
}
|
|
|
|
|
private def serializeAddressData(address: Address): Array[Byte] =
|
|
|
|
|
protoForAddressData(address).build().toByteArray
|
|
|
|
|
|
|
|
|
|
private def serializeClassicUniqueAddress(uniqueAddress: UniqueAddress): Array[Byte] =
|
|
|
|
|
ArteryControlFormats.UniqueAddress.newBuilder()
|
|
|
|
|
.setUid(uniqueAddress.uid)
|
|
|
|
|
.setAddress(protoForAddress(uniqueAddress.address))
|
|
|
|
|
.build().toByteArray
|
|
|
|
|
|
2017-03-16 15:12:35 +01:00
|
|
|
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.")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-04-28 23:33:59 +10:00
|
|
|
private val IdentifyManifest = "A"
|
2016-09-26 15:04:53 +02:00
|
|
|
private val ActorIdentityManifest = "B"
|
2016-06-19 17:55:10 +02:00
|
|
|
private val OptionManifest = "C"
|
2016-09-28 11:14:33 +02:00
|
|
|
private val StatusSuccessManifest = "D"
|
|
|
|
|
private val StatusFailureManifest = "E"
|
|
|
|
|
private val ThrowableManifest = "F"
|
2016-09-27 15:46:22 +02:00
|
|
|
private val ActorRefManifest = "G"
|
2016-12-01 16:51:39 +01:00
|
|
|
private val OptionalManifest = "H"
|
2016-09-26 15:04:53 +02:00
|
|
|
private val PoisonPillManifest = "P"
|
|
|
|
|
private val KillManifest = "K"
|
2016-09-28 16:00:50 +02:00
|
|
|
private val RemoteWatcherHBManifest = "RWHB"
|
2017-11-01 19:22:09 -04:00
|
|
|
private val DoneManifest = "DONE"
|
2018-03-05 17:38:17 +01:00
|
|
|
private val NotUsedManifest = "NU"
|
2018-01-10 19:28:51 +09:00
|
|
|
private val AddressManifest = "AD"
|
|
|
|
|
private val UniqueAddressManifest = "UD"
|
2016-09-28 16:00:50 +02:00
|
|
|
private val RemoteWatcherHBRespManifest = "RWHR"
|
|
|
|
|
private val ActorInitializationExceptionManifest = "AIEX"
|
2017-03-16 15:12:35 +01:00
|
|
|
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"
|
2016-04-28 23:33:59 +10:00
|
|
|
|
|
|
|
|
private val fromBinaryMap = Map[String, Array[Byte] ⇒ AnyRef](
|
2016-06-02 14:06:57 +02:00
|
|
|
IdentifyManifest → deserializeIdentify,
|
2016-09-28 16:00:50 +02:00
|
|
|
ActorIdentityManifest → deserializeActorIdentity,
|
2016-09-28 11:14:33 +02:00
|
|
|
StatusSuccessManifest → deserializeStatusSuccess,
|
|
|
|
|
StatusFailureManifest → deserializeStatusFailure,
|
2016-09-27 15:46:22 +02:00
|
|
|
ThrowableManifest → throwableSupport.deserializeThrowable,
|
2016-09-28 16:00:50 +02:00
|
|
|
ActorRefManifest → deserializeActorRefBytes,
|
2016-09-26 15:04:53 +02:00
|
|
|
OptionManifest → deserializeOption,
|
2016-12-01 16:51:39 +01:00
|
|
|
OptionalManifest → deserializeOptional,
|
2016-09-26 15:04:53 +02:00
|
|
|
PoisonPillManifest → ((_) ⇒ PoisonPill),
|
2016-09-28 16:00:50 +02:00
|
|
|
KillManifest → ((_) ⇒ Kill),
|
|
|
|
|
RemoteWatcherHBManifest → ((_) ⇒ RemoteWatcher.Heartbeat),
|
2017-11-01 19:22:09 -04:00
|
|
|
DoneManifest → ((_) ⇒ Done),
|
2018-03-05 17:38:17 +01:00
|
|
|
NotUsedManifest → ((_) ⇒ NotUsed),
|
2018-01-10 19:28:51 +09:00
|
|
|
AddressManifest → deserializeAddressData,
|
|
|
|
|
UniqueAddressManifest → deserializeUniqueAddress,
|
2016-09-28 16:00:50 +02:00
|
|
|
RemoteWatcherHBRespManifest → deserializeHeartbeatRsp,
|
2017-03-16 15:12:35 +01:00
|
|
|
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
|
|
|
|
|
)
|
2016-04-28 23:33:59 +10:00
|
|
|
|
|
|
|
|
override def manifest(o: AnyRef): String =
|
|
|
|
|
o match {
|
2017-03-16 15:12:35 +01:00
|
|
|
case _: Identify ⇒ IdentifyManifest
|
|
|
|
|
case _: ActorIdentity ⇒ ActorIdentityManifest
|
|
|
|
|
case _: Option[Any] ⇒ OptionManifest
|
|
|
|
|
case _: Optional[_] ⇒ OptionalManifest
|
|
|
|
|
case _: ActorRef ⇒ ActorRefManifest
|
|
|
|
|
case _: Status.Success ⇒ StatusSuccessManifest
|
|
|
|
|
case _: Status.Failure ⇒ StatusFailureManifest
|
|
|
|
|
case _: ActorInitializationException ⇒ ActorInitializationExceptionManifest
|
|
|
|
|
case _: Throwable ⇒ ThrowableManifest
|
|
|
|
|
case PoisonPill ⇒ PoisonPillManifest
|
|
|
|
|
case Kill ⇒ KillManifest
|
|
|
|
|
case RemoteWatcher.Heartbeat ⇒ RemoteWatcherHBManifest
|
2017-11-01 19:22:09 -04:00
|
|
|
case Done ⇒ DoneManifest
|
2018-03-05 17:38:17 +01:00
|
|
|
case NotUsed ⇒ NotUsedManifest
|
2018-01-10 19:28:51 +09:00
|
|
|
case _: Address ⇒ AddressManifest
|
|
|
|
|
case _: UniqueAddress ⇒ UniqueAddressManifest
|
2017-03-16 15:12:35 +01:00
|
|
|
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
|
2016-04-28 23:33:59 +10:00
|
|
|
case _ ⇒
|
|
|
|
|
throw new IllegalArgumentException(s"Can't serialize object of type ${o.getClass} in [${getClass.getName}]")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef =
|
|
|
|
|
fromBinaryMap.get(manifest) match {
|
|
|
|
|
case Some(deserializer) ⇒ deserializer(bytes)
|
2016-12-16 11:36:04 +01:00
|
|
|
case None ⇒ throw new NotSerializableException(
|
2016-04-28 23:33:59 +10:00
|
|
|
s"Unimplemented deserialization of message with manifest [$manifest] in [${getClass.getName}]")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def deserializeIdentify(bytes: Array[Byte]): Identify = {
|
|
|
|
|
val identifyProto = ContainerFormats.Identify.parseFrom(bytes)
|
2016-09-28 11:14:33 +02:00
|
|
|
val messageId = payloadSupport.deserializePayload(identifyProto.getMessageId)
|
2016-04-28 23:33:59 +10:00
|
|
|
Identify(messageId)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def deserializeActorIdentity(bytes: Array[Byte]): ActorIdentity = {
|
|
|
|
|
val actorIdentityProto = ContainerFormats.ActorIdentity.parseFrom(bytes)
|
2016-09-28 11:14:33 +02:00
|
|
|
val correlationId = payloadSupport.deserializePayload(actorIdentityProto.getCorrelationId)
|
2016-04-28 23:33:59 +10:00
|
|
|
val actorRef =
|
|
|
|
|
if (actorIdentityProto.hasRef)
|
|
|
|
|
Some(deserializeActorRef(actorIdentityProto.getRef))
|
|
|
|
|
else
|
|
|
|
|
None
|
|
|
|
|
ActorIdentity(correlationId, actorRef)
|
|
|
|
|
}
|
|
|
|
|
|
2016-09-27 15:46:22 +02:00
|
|
|
private def deserializeActorRefBytes(bytes: Array[Byte]): ActorRef =
|
|
|
|
|
deserializeActorRef(ContainerFormats.ActorRef.parseFrom(bytes))
|
|
|
|
|
|
2016-04-28 23:33:59 +10:00
|
|
|
private def deserializeActorRef(actorRef: ContainerFormats.ActorRef): ActorRef =
|
|
|
|
|
serialization.system.provider.resolveActorRef(actorRef.getPath)
|
|
|
|
|
|
2016-06-19 17:55:10 +02:00
|
|
|
private def deserializeOption(bytes: Array[Byte]): Option[Any] = {
|
|
|
|
|
if (bytes.length == 0)
|
|
|
|
|
None
|
|
|
|
|
else {
|
|
|
|
|
val optionProto = ContainerFormats.Option.parseFrom(bytes)
|
2016-09-28 11:14:33 +02:00
|
|
|
Some(payloadSupport.deserializePayload(optionProto.getValue))
|
2016-06-19 17:55:10 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-01 16:51:39 +01:00
|
|
|
private def deserializeOptional(bytes: Array[Byte]): Optional[Any] = {
|
|
|
|
|
if (bytes.length == 0)
|
|
|
|
|
Optional.empty()
|
|
|
|
|
else {
|
|
|
|
|
val optionProto = ContainerFormats.Option.parseFrom(bytes)
|
|
|
|
|
Optional.of(payloadSupport.deserializePayload(optionProto.getValue))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-09-28 11:14:33 +02:00
|
|
|
private def deserializeStatusSuccess(bytes: Array[Byte]): Status.Success =
|
|
|
|
|
Status.Success(payloadSupport.deserializePayload(ContainerFormats.Payload.parseFrom(bytes)))
|
|
|
|
|
|
|
|
|
|
private def deserializeStatusFailure(bytes: Array[Byte]): Status.Failure =
|
|
|
|
|
Status.Failure(payloadSupport.deserializePayload(ContainerFormats.Payload.parseFrom(bytes)).asInstanceOf[Throwable])
|
2016-04-28 23:33:59 +10:00
|
|
|
|
2018-01-10 19:28:51 +09:00
|
|
|
private def deserializeAddressData(bytes: Array[Byte]): Address =
|
|
|
|
|
addressFromDataProto(WireFormats.AddressData.parseFrom(bytes))
|
|
|
|
|
|
|
|
|
|
private def addressFromDataProto(a: WireFormats.AddressData): Address = {
|
|
|
|
|
Address(
|
|
|
|
|
a.getProtocol,
|
|
|
|
|
a.getSystem,
|
|
|
|
|
// technicaly the presence of hostname and port are guaranteed, see our serializeAddressData
|
|
|
|
|
if (a.hasHostname) Some(a.getHostname) else None,
|
|
|
|
|
if (a.hasPort) Some(a.getPort) else None
|
|
|
|
|
)
|
|
|
|
|
}
|
|
|
|
|
private def addressFromProto(a: ArteryControlFormats.Address): Address = {
|
|
|
|
|
Address(
|
|
|
|
|
a.getProtocol,
|
|
|
|
|
a.getSystem,
|
|
|
|
|
// technicaly the presence of hostname and port are guaranteed, see our serializeAddressData
|
|
|
|
|
if (a.hasHostname) Some(a.getHostname) else None,
|
|
|
|
|
if (a.hasPort) Some(a.getPort) else None
|
|
|
|
|
)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def deserializeUniqueAddress(bytes: Array[Byte]): UniqueAddress = {
|
|
|
|
|
val u = ArteryControlFormats.UniqueAddress.parseFrom(bytes)
|
|
|
|
|
UniqueAddress(
|
|
|
|
|
addressFromProto(u.getAddress),
|
|
|
|
|
u.getUid
|
|
|
|
|
)
|
|
|
|
|
}
|
|
|
|
|
|
2016-09-28 16:00:50 +02:00
|
|
|
private def deserializeHeartbeatRsp(bytes: Array[Byte]): RemoteWatcher.HeartbeatRsp = {
|
|
|
|
|
RemoteWatcher.HeartbeatRsp(ContainerFormats.WatcherHeartbeatResponse.parseFrom(bytes).getUid.toInt)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def deserializeActorInitializationException(bytes: Array[Byte]): ActorInitializationException = {
|
|
|
|
|
val serializedEx = ContainerFormats.ActorInitializationException.parseFrom(bytes)
|
|
|
|
|
val ref = deserializeActorRef(serializedEx.getActor)
|
|
|
|
|
val refString = ref.path.toString
|
|
|
|
|
val message = serializedEx.getMessage
|
|
|
|
|
|
|
|
|
|
val reconstructedMessage =
|
|
|
|
|
if (message.startsWith(refString)) message.drop(refString.length + 2)
|
|
|
|
|
else message
|
|
|
|
|
|
|
|
|
|
ActorInitializationException(
|
|
|
|
|
if (serializedEx.hasActor) ref else null,
|
|
|
|
|
reconstructedMessage,
|
2016-12-01 16:51:39 +01:00
|
|
|
payloadSupport.deserializePayload(serializedEx.getCause).asInstanceOf[Throwable])
|
2016-09-28 16:00:50 +02:00
|
|
|
}
|
|
|
|
|
|
2017-03-16 15:12:35 +01:00
|
|
|
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 =
|
2017-10-06 10:30:28 +02:00
|
|
|
if (bp.hasResizer) Some(payloadSupport.deserializePayload(bp.getResizer).asInstanceOf[Resizer])
|
|
|
|
|
else None,
|
2017-03-16 15:12:35 +01:00
|
|
|
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 =
|
2017-10-06 10:30:28 +02:00
|
|
|
if (rp.hasResizer) Some(payloadSupport.deserializePayload(rp.getResizer).asInstanceOf[Resizer])
|
|
|
|
|
else None,
|
2017-03-16 15:12:35 +01:00
|
|
|
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 =
|
2017-10-06 10:30:28 +02:00
|
|
|
if (rp.hasResizer) Some(payloadSupport.deserializePayload(rp.getResizer).asInstanceOf[Resizer])
|
|
|
|
|
else None,
|
2017-03-16 15:12:35 +01:00
|
|
|
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 =
|
2017-10-06 10:30:28 +02:00
|
|
|
if (sgp.getGeneric.hasResizer) Some(payloadSupport.deserializePayload(sgp.getGeneric.getResizer).asInstanceOf[Resizer])
|
|
|
|
|
else None,
|
2017-03-16 15:12:35 +01:00
|
|
|
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 =
|
2017-10-06 10:30:28 +02:00
|
|
|
if (tcp.getGeneric.hasResizer) Some(payloadSupport.deserializePayload(tcp.getGeneric.getResizer).asInstanceOf[Resizer])
|
|
|
|
|
else None,
|
2017-03-16 15:12:35 +01:00
|
|
|
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
|
|
|
|
|
)
|
|
|
|
|
}
|
2016-04-28 23:33:59 +10:00
|
|
|
}
|