=cdd #18768 Cache serialization of read and write msg
The WriteAggregator and ReadAggregator typically send the same message to several replicas and by caching the serialized bytes we avoid doing the same thing for each message and add a test for WriteAggregator
This commit is contained in:
parent
81cba2e580
commit
6b935e0c0b
5 changed files with 399 additions and 4 deletions
|
|
@ -0,0 +1,133 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2015 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.ddata
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Props
|
||||
import akka.cluster.Cluster
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
import org.scalatest.Matchers
|
||||
import org.scalatest.WordSpecLike
|
||||
import akka.actor.Address
|
||||
import akka.actor.ActorRef
|
||||
import akka.cluster.ddata.Replicator.Internal._
|
||||
import akka.cluster.ddata.Replicator._
|
||||
import akka.actor.ActorSelection
|
||||
import akka.cluster.UniqueAddress
|
||||
|
||||
object WriteAggregatorSpec {
|
||||
|
||||
val key = GSetKey[String]("a")
|
||||
|
||||
def writeAggregatorProps(data: GSet[String], consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef], nodes: Set[Address], replyTo: ActorRef): Props =
|
||||
Props(new TestWriteAggregator(data, consistency, probes, nodes, replyTo))
|
||||
|
||||
class TestWriteAggregator(data: GSet[String], consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef], nodes: Set[Address], replyTo: ActorRef)
|
||||
extends WriteAggregator(key, DataEnvelope(data), consistency, None, nodes, replyTo) {
|
||||
|
||||
override def replica(address: Address): ActorSelection =
|
||||
context.actorSelection(probes(address).path)
|
||||
|
||||
override def senderAddress(): Address =
|
||||
probes.find { case (a, r) ⇒ r == sender() }.get._1
|
||||
}
|
||||
|
||||
def writeAckAdapterProps(replica: ActorRef): Props =
|
||||
Props(new WriteAckAdapter(replica))
|
||||
|
||||
class WriteAckAdapter(replica: ActorRef) extends Actor {
|
||||
var replicator: Option[ActorRef] = None
|
||||
|
||||
def receive = {
|
||||
case WriteAck ⇒
|
||||
replicator.foreach(_ ! WriteAck)
|
||||
case msg ⇒
|
||||
replicator = Some(sender())
|
||||
replica ! msg
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class WriteAggregatorSpec extends AkkaSpec("""
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.netty.tcp.port=0
|
||||
""")
|
||||
with ImplicitSender {
|
||||
|
||||
val nodeA = Address("akka.tcp", "Sys", "a", 2552)
|
||||
val nodeB = nodeA.copy(host = Some("b"))
|
||||
val nodeC = nodeA.copy(host = Some("c"))
|
||||
val nodeD = nodeA.copy(host = Some("d"))
|
||||
// 4 replicas + the local => 5
|
||||
val nodes = Set(nodeA, nodeB, nodeC, nodeD)
|
||||
|
||||
val data = GSet.empty + "A" + "B"
|
||||
val timeout = 3.seconds.dilated
|
||||
val writeTwo = WriteTo(2, timeout)
|
||||
val writeMajority = WriteMajority(timeout)
|
||||
|
||||
def probes(probe: ActorRef): Map[Address, ActorRef] =
|
||||
nodes.toSeq.map(_ -> system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||
|
||||
"WriteAggregator" must {
|
||||
"send to at least N/2+1 replicas when WriteMajority" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
|
||||
watch(aggr)
|
||||
expectTerminated(aggr)
|
||||
}
|
||||
|
||||
"send to more when no immediate reply" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
|
||||
watch(aggr)
|
||||
expectTerminated(aggr)
|
||||
}
|
||||
|
||||
"timeout when less than required acks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
expectMsg(UpdateTimeout(WriteAggregatorSpec.key, None))
|
||||
watch(aggr)
|
||||
expectTerminated(aggr)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -78,4 +78,134 @@ class ReplicatorMessageSerializerSpec extends TestKit(ActorSystem("ReplicatorMes
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
"Cache" must {
|
||||
import ReplicatorMessageSerializer._
|
||||
"be power of 2" in {
|
||||
intercept[IllegalArgumentException] {
|
||||
new SmallCache[String, String](3, 5.seconds, _ ⇒ null)
|
||||
}
|
||||
}
|
||||
|
||||
"get added element" in {
|
||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ ⇒ null)
|
||||
val a = Read("a")
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
val b = Read("b")
|
||||
cache.add(b, "B")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be("B")
|
||||
}
|
||||
|
||||
"return null for non-existing elements" in {
|
||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ ⇒ null)
|
||||
val a = Read("a")
|
||||
cache.get(a) should be(null)
|
||||
cache.add(a, "A")
|
||||
val b = Read("b")
|
||||
cache.get(b) should be(null)
|
||||
}
|
||||
|
||||
"hold latest added elements" in {
|
||||
val cache = new SmallCache[Read, String](4, 5.seconds, _ ⇒ null)
|
||||
val a = Read("a")
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
val d = Read("d")
|
||||
val e = Read("e")
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
cache.add(b, "B")
|
||||
cache.get(a) should be("A")
|
||||
cache.add(c, "C")
|
||||
cache.get(a) should be("A")
|
||||
cache.add(d, "D")
|
||||
cache.get(a) should be("A")
|
||||
// now it is full and a will be pushed out
|
||||
cache.add(e, "E")
|
||||
cache.get(a) should be(null)
|
||||
cache.get(b) should be("B")
|
||||
cache.get(c) should be("C")
|
||||
cache.get(d) should be("D")
|
||||
cache.get(e) should be("E")
|
||||
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be(null)
|
||||
cache.get(c) should be("C")
|
||||
cache.get(d) should be("D")
|
||||
cache.get(e) should be("E")
|
||||
}
|
||||
|
||||
"handle Int wrap around" ignore { // ignored because it takes 20 seconds (but it works)
|
||||
val cache = new SmallCache[Read, String](2, 5.seconds, _ ⇒ null)
|
||||
val a = Read("a")
|
||||
val x = a -> "A"
|
||||
var n = 0
|
||||
while (n <= Int.MaxValue - 3) {
|
||||
cache.add(x)
|
||||
n += 1
|
||||
}
|
||||
|
||||
cache.get(a) should be("A")
|
||||
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
cache.add(b, "B")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be("B")
|
||||
|
||||
cache.add(c, "C")
|
||||
cache.get(a) should be(null)
|
||||
cache.get(b) should be("B")
|
||||
cache.get(c) should be("C")
|
||||
|
||||
cache.add(a, "A")
|
||||
cache.get(a) should be("A")
|
||||
cache.get(b) should be(null)
|
||||
cache.get(c) should be("C")
|
||||
}
|
||||
|
||||
"suppory getOrAdd" in {
|
||||
var n = 0
|
||||
def createValue(a: Read): AnyRef = {
|
||||
n += 1
|
||||
new AnyRef {
|
||||
override val toString = "v" + n
|
||||
}
|
||||
}
|
||||
|
||||
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, a ⇒ createValue(a))
|
||||
val a = Read("a")
|
||||
val v1 = cache.getOrAdd(a)
|
||||
v1.toString should be("v1")
|
||||
cache.getOrAdd(a) should be theSameInstanceAs v1
|
||||
}
|
||||
|
||||
"evict cache after time-to-live" in {
|
||||
val cache = new SmallCache[Read, AnyRef](4, 10.millis, _ ⇒ null)
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
cache.add(b, "B")
|
||||
cache.add(c, "C")
|
||||
|
||||
Thread.sleep(30)
|
||||
cache.evict()
|
||||
cache.get(b) should be(null)
|
||||
cache.get(c) should be(null)
|
||||
}
|
||||
|
||||
"not evict cache before time-to-live" in {
|
||||
val cache = new SmallCache[Read, AnyRef](4, 5.seconds, _ ⇒ null)
|
||||
val b = Read("b")
|
||||
val c = Read("c")
|
||||
cache.add(b, "B")
|
||||
cache.add(c, "C")
|
||||
cache.evict()
|
||||
cache.get(b) should be("B")
|
||||
cache.get(c) should be("C")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue