2013-04-04 17:56:29 +02:00
|
|
|
/**
|
2018-01-04 17:26:29 +00:00
|
|
|
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
2013-04-04 17:56:29 +02:00
|
|
|
*/
|
|
|
|
|
package akka.cluster.protobuf
|
|
|
|
|
|
|
|
|
|
import akka.cluster._
|
2017-08-09 16:06:18 +02:00
|
|
|
import akka.actor.{ ActorSystem, Address, ExtendedActorSystem }
|
2018-02-20 15:47:09 +01:00
|
|
|
import akka.cluster.InternalClusterAction.CompatibleConfig
|
2017-03-16 15:12:35 +01:00
|
|
|
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
2017-08-09 16:06:18 +02:00
|
|
|
import akka.routing.RoundRobinPool
|
2017-03-16 15:12:35 +01:00
|
|
|
|
2013-04-04 17:56:29 +02:00
|
|
|
import collection.immutable.SortedSet
|
2017-08-09 16:06:18 +02:00
|
|
|
import akka.testkit.{ AkkaSpec, TestKit }
|
2018-02-20 15:47:09 +01:00
|
|
|
import com.typesafe.config.ConfigFactory
|
2013-04-04 17:56:29 +02:00
|
|
|
|
2013-10-17 10:35:37 +02:00
|
|
|
class ClusterMessageSerializerSpec extends AkkaSpec(
|
2016-06-10 15:04:13 +02:00
|
|
|
"akka.actor.provider = cluster") {
|
2013-04-04 17:56:29 +02:00
|
|
|
|
|
|
|
|
val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
|
|
|
|
|
2017-07-04 05:52:03 -07:00
|
|
|
def roundtrip[T <: AnyRef](obj: T): T = {
|
2018-02-28 09:46:37 +01:00
|
|
|
val manifest = serializer.manifest(obj)
|
2013-04-04 17:56:29 +02:00
|
|
|
val blob = serializer.toBinary(obj)
|
2018-02-28 09:46:37 +01:00
|
|
|
serializer.fromBinary(blob, manifest).asInstanceOf[T]
|
2017-07-04 05:52:03 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def checkSerialization(obj: AnyRef): Unit = {
|
|
|
|
|
(obj, roundtrip(obj)) match {
|
|
|
|
|
case (env: GossipEnvelope, env2: GossipEnvelope) ⇒
|
2015-01-16 11:09:59 +01:00
|
|
|
env2.from should ===(env.from)
|
|
|
|
|
env2.to should ===(env.to)
|
|
|
|
|
env2.gossip should ===(env.gossip)
|
2017-07-04 05:52:03 -07:00
|
|
|
case (_, ref) ⇒
|
2015-01-16 11:09:59 +01:00
|
|
|
ref should ===(obj)
|
2013-10-17 10:35:37 +02:00
|
|
|
}
|
|
|
|
|
|
2013-04-04 17:56:29 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
import MemberStatus._
|
|
|
|
|
|
2017-11-02 08:17:24 +00:00
|
|
|
val a1 = TestMember(Address("akka.tcp", "sys", "a", 2552), Joining, Set.empty[String])
|
2013-04-11 09:18:12 +02:00
|
|
|
val b1 = TestMember(Address("akka.tcp", "sys", "b", 2552), Up, Set("r1"))
|
2017-11-02 08:17:24 +00:00
|
|
|
val c1 = TestMember(Address("akka.tcp", "sys", "c", 2552), Leaving, Set.empty[String], "foo")
|
2017-07-04 05:52:03 -07:00
|
|
|
val d1 = TestMember(Address("akka.tcp", "sys", "d", 2552), Exiting, Set("r1"), "foo")
|
2013-04-11 09:18:12 +02:00
|
|
|
val e1 = TestMember(Address("akka.tcp", "sys", "e", 2552), Down, Set("r3"))
|
2017-07-04 05:52:03 -07:00
|
|
|
val f1 = TestMember(Address("akka.tcp", "sys", "f", 2552), Removed, Set("r3"), "foo")
|
2013-04-04 17:56:29 +02:00
|
|
|
|
|
|
|
|
"ClusterMessages" must {
|
|
|
|
|
|
|
|
|
|
"be serializable" in {
|
|
|
|
|
val address = Address("akka.tcp", "system", "some.host.org", 4711)
|
2016-12-01 18:49:38 +01:00
|
|
|
val uniqueAddress = UniqueAddress(address, 17L)
|
2013-04-11 09:18:12 +02:00
|
|
|
val address2 = Address("akka.tcp", "system", "other.host.org", 4711)
|
2016-12-01 18:49:38 +01:00
|
|
|
val uniqueAddress2 = UniqueAddress(address2, 18L)
|
2017-10-23 11:49:51 +02:00
|
|
|
checkSerialization(InternalClusterAction.Join(uniqueAddress, Set("foo", "bar", "dc-A")))
|
2013-04-04 17:56:29 +02:00
|
|
|
checkSerialization(ClusterUserAction.Leave(address))
|
|
|
|
|
checkSerialization(ClusterUserAction.Down(address))
|
2018-02-20 15:47:09 +01:00
|
|
|
checkSerialization(InternalClusterAction.InitJoin(ConfigFactory.empty))
|
|
|
|
|
checkSerialization(InternalClusterAction.InitJoinAck(address, CompatibleConfig(ConfigFactory.empty)))
|
2013-04-04 17:56:29 +02:00
|
|
|
checkSerialization(InternalClusterAction.InitJoinNack(address))
|
2013-11-07 13:52:08 +01:00
|
|
|
checkSerialization(ClusterHeartbeatSender.Heartbeat(address))
|
|
|
|
|
checkSerialization(ClusterHeartbeatSender.HeartbeatRsp(uniqueAddress))
|
2016-12-01 18:49:38 +01:00
|
|
|
checkSerialization(InternalClusterAction.ExitingConfirmed(uniqueAddress))
|
2013-04-04 17:56:29 +02:00
|
|
|
|
|
|
|
|
val node1 = VectorClock.Node("node1")
|
|
|
|
|
val node2 = VectorClock.Node("node2")
|
2013-04-25 10:59:27 +02:00
|
|
|
val node3 = VectorClock.Node("node3")
|
|
|
|
|
val node4 = VectorClock.Node("node4")
|
|
|
|
|
val g1 = (Gossip(SortedSet(a1, b1, c1, d1)) :+ node1 :+ node2).seen(a1.uniqueAddress).seen(b1.uniqueAddress)
|
|
|
|
|
val g2 = (g1 :+ node3 :+ node4).seen(a1.uniqueAddress).seen(c1.uniqueAddress)
|
2013-08-27 15:14:53 +02:00
|
|
|
val reachability3 = Reachability.empty.unreachable(a1.uniqueAddress, e1.uniqueAddress).unreachable(b1.uniqueAddress, e1.uniqueAddress)
|
|
|
|
|
val g3 = g2.copy(members = SortedSet(a1, b1, c1, d1, e1), overview = g2.overview.copy(reachability = reachability3))
|
2017-07-04 09:09:40 +01:00
|
|
|
val g4 = g1.remove(d1.uniqueAddress, 352684800)
|
2013-04-25 10:59:27 +02:00
|
|
|
checkSerialization(GossipEnvelope(a1.uniqueAddress, uniqueAddress2, g1))
|
|
|
|
|
checkSerialization(GossipEnvelope(a1.uniqueAddress, uniqueAddress2, g2))
|
2013-04-28 22:28:20 +02:00
|
|
|
checkSerialization(GossipEnvelope(a1.uniqueAddress, uniqueAddress2, g3))
|
2017-07-04 09:09:40 +01:00
|
|
|
checkSerialization(GossipEnvelope(a1.uniqueAddress, uniqueAddress2, g4))
|
2013-04-28 22:28:20 +02:00
|
|
|
|
|
|
|
|
checkSerialization(GossipStatus(a1.uniqueAddress, g1.version))
|
|
|
|
|
checkSerialization(GossipStatus(a1.uniqueAddress, g2.version))
|
|
|
|
|
checkSerialization(GossipStatus(a1.uniqueAddress, g3.version))
|
2013-04-11 09:18:12 +02:00
|
|
|
|
|
|
|
|
checkSerialization(InternalClusterAction.Welcome(uniqueAddress, g2))
|
2013-04-04 17:56:29 +02:00
|
|
|
}
|
2017-07-04 05:52:03 -07:00
|
|
|
|
2018-02-28 09:46:37 +01:00
|
|
|
"be compatible with wire format of version 2.5.9 (using InitJoin singleton instead of class)" in {
|
|
|
|
|
// we must use the old singleton class name so that the other side will see an InitJoin
|
|
|
|
|
// but discard the config as it does not know about the config check
|
|
|
|
|
val oldClassName = "akka.cluster.InternalClusterAction$InitJoin$"
|
|
|
|
|
serializer.manifest(InternalClusterAction.InitJoin(ConfigFactory.empty())) should ===(oldClassName)
|
|
|
|
|
|
|
|
|
|
// in 2.5.9 and earlier, it was an object and serialized to empty byte array
|
|
|
|
|
// and we should accept that
|
|
|
|
|
val deserialized = serializer.fromBinary(Array.emptyByteArray, oldClassName)
|
|
|
|
|
deserialized shouldBe an[InternalClusterAction.InitJoin]
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be compatible with wire format of version 2.5.9 (using serialized address for InitJoinAck)" in {
|
|
|
|
|
// we must use the old singleton class name so that the other side will see an InitJoin
|
|
|
|
|
// but discard the config as it does not know about the config check
|
|
|
|
|
val initJoinAck = InternalClusterAction.InitJoinAck(
|
|
|
|
|
Address("akka.tcp", "cluster", "127.0.0.1", 2552),
|
|
|
|
|
InternalClusterAction.UncheckedConfig)
|
|
|
|
|
val serializedinInitJoinAckPre2510 = serializer.addressToProto(initJoinAck.address).build().toByteArray
|
|
|
|
|
|
|
|
|
|
val deserialized = serializer.fromBinary(serializedinInitJoinAckPre2510, ClusterMessageSerializer.InitJoinAckManifest)
|
|
|
|
|
deserialized shouldEqual initJoinAck
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-09 16:06:18 +02:00
|
|
|
"be compatible with wire format of version 2.5.3 (using use-role instead of use-roles)" in {
|
|
|
|
|
val system = ActorSystem("ClusterMessageSerializer-old-wire-format")
|
|
|
|
|
|
|
|
|
|
try {
|
|
|
|
|
val serializer = new ClusterMessageSerializer(system.asInstanceOf[ExtendedActorSystem])
|
|
|
|
|
|
|
|
|
|
// the oldSnapshot was created with the version of ClusterRouterPoolSettings in Akka 2.5.3. See issue #23257.
|
|
|
|
|
// It was created with:
|
|
|
|
|
/*
|
|
|
|
|
import org.apache.commons.codec.binary.Hex.encodeHex
|
|
|
|
|
val bytes = serializer.toBinary(
|
|
|
|
|
ClusterRouterPool(RoundRobinPool(nrOfInstances = 4), ClusterRouterPoolSettings(123, 345, true, Some("role ABC"))))
|
|
|
|
|
println(String.valueOf(encodeHex(bytes)))
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
val oldBytesHex = "0a0f08101205524f5252501a04080418001211087b10d90218012208726f6c6520414243"
|
|
|
|
|
|
|
|
|
|
import org.apache.commons.codec.binary.Hex.decodeHex
|
|
|
|
|
val oldBytes = decodeHex(oldBytesHex.toCharArray)
|
|
|
|
|
val result = serializer.fromBinary(oldBytes, classOf[ClusterRouterPool])
|
|
|
|
|
|
|
|
|
|
result match {
|
|
|
|
|
case pool: ClusterRouterPool ⇒
|
|
|
|
|
pool.settings.totalInstances should ===(123)
|
|
|
|
|
pool.settings.maxInstancesPerNode should ===(345)
|
|
|
|
|
pool.settings.allowLocalRoutees should ===(true)
|
|
|
|
|
pool.settings.useRole should ===(Some("role ABC"))
|
|
|
|
|
pool.settings.useRoles should ===(Set("role ABC"))
|
|
|
|
|
}
|
|
|
|
|
} finally {
|
|
|
|
|
TestKit.shutdownActorSystem(system)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
2017-08-31 10:51:12 +02:00
|
|
|
|
2017-10-23 11:49:51 +02:00
|
|
|
"add a default data center role to gossip if none is present" in {
|
2017-07-04 05:52:03 -07:00
|
|
|
val env = roundtrip(GossipEnvelope(a1.uniqueAddress, d1.uniqueAddress, Gossip(SortedSet(a1, d1))))
|
2017-07-04 17:11:21 +02:00
|
|
|
env.gossip.members.head.roles should be(Set(ClusterSettings.DcRolePrefix + "default"))
|
|
|
|
|
env.gossip.members.tail.head.roles should be(Set("r1", ClusterSettings.DcRolePrefix + "foo"))
|
2017-07-04 05:52:03 -07:00
|
|
|
}
|
2017-10-23 11:49:51 +02:00
|
|
|
|
|
|
|
|
"add a default data center role to internal join action if none is present" in {
|
|
|
|
|
val join = roundtrip(InternalClusterAction.Join(a1.uniqueAddress, Set()))
|
|
|
|
|
join.roles should be(Set(ClusterSettings.DcRolePrefix + "default"))
|
|
|
|
|
}
|
2013-04-04 17:56:29 +02:00
|
|
|
}
|
2017-03-16 15:12:35 +01:00
|
|
|
"Cluster router pool" must {
|
2017-08-09 16:06:18 +02:00
|
|
|
"be serializable with no role" in {
|
|
|
|
|
checkSerialization(ClusterRouterPool(
|
|
|
|
|
RoundRobinPool(
|
|
|
|
|
nrOfInstances = 4
|
|
|
|
|
),
|
|
|
|
|
ClusterRouterPoolSettings(
|
|
|
|
|
totalInstances = 2,
|
|
|
|
|
maxInstancesPerNode = 5,
|
|
|
|
|
allowLocalRoutees = true
|
|
|
|
|
)
|
|
|
|
|
))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be serializable with one role" in {
|
|
|
|
|
checkSerialization(ClusterRouterPool(
|
|
|
|
|
RoundRobinPool(
|
|
|
|
|
nrOfInstances = 4
|
|
|
|
|
),
|
|
|
|
|
ClusterRouterPoolSettings(
|
|
|
|
|
totalInstances = 2,
|
|
|
|
|
maxInstancesPerNode = 5,
|
|
|
|
|
allowLocalRoutees = true,
|
|
|
|
|
useRoles = Set("Richard, Duke of Gloucester")
|
|
|
|
|
)
|
|
|
|
|
))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"be serializable with many roles" in {
|
2017-03-16 15:12:35 +01:00
|
|
|
checkSerialization(ClusterRouterPool(
|
|
|
|
|
RoundRobinPool(
|
2017-07-04 17:11:21 +02:00
|
|
|
nrOfInstances = 4),
|
2017-03-16 15:12:35 +01:00
|
|
|
ClusterRouterPoolSettings(
|
|
|
|
|
totalInstances = 2,
|
|
|
|
|
maxInstancesPerNode = 5,
|
|
|
|
|
allowLocalRoutees = true,
|
2017-08-09 16:06:18 +02:00
|
|
|
useRoles = Set("Richard, Duke of Gloucester", "Hongzhi Emperor", "Red Rackham")
|
2017-03-16 15:12:35 +01:00
|
|
|
)
|
|
|
|
|
))
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-04 17:56:29 +02:00
|
|
|
}
|