Use delta in direct write also, #22188

* Follow up on the causal delivery of deltas.
* The first implementation used full state for the direct
  Write messages, i.e. updates with WriteConsistency != LocalWrite
* This is an optimization so that delatas are tried first and if
  they can't be applied it falls back to full state.
* For simultanious updates the messages may be reordered because we
  create separate WriteAggregator actor and such, but normally they
  will be sent in order so the deltas will typically be received in
  order, otherwise we fall back to retrying with full state in the
  second round in the WriteAggregator.
This commit is contained in:
Patrik Nordwall 2017-02-23 12:12:29 +01:00
parent b2759ab56a
commit 233e784154
9 changed files with 365 additions and 85 deletions

View file

@ -17,18 +17,24 @@ import akka.cluster.ddata.Replicator._
import akka.remote.RARP
import scala.concurrent.Future
import akka.cluster.Cluster
object WriteAggregatorSpec {
val key = GSetKey[String]("a")
val KeyA = GSetKey[String]("A")
val KeyB = ORSetKey[String]("B")
def writeAggregatorProps(data: GSet[String], consistency: Replicator.WriteConsistency,
probes: Map[Address, ActorRef], nodes: Set[Address], unreachable: Set[Address], replyTo: ActorRef, durable: Boolean): Props =
Props(new TestWriteAggregator(data, consistency, probes, nodes, unreachable, replyTo, durable))
Props(new TestWriteAggregator(KeyA, data, None, consistency, probes, nodes, unreachable, replyTo, durable))
class TestWriteAggregator(data: GSet[String], consistency: Replicator.WriteConsistency,
def writeAggregatorPropsWithDelta(data: ORSet[String], delta: Delta, consistency: Replicator.WriteConsistency,
probes: Map[Address, ActorRef], nodes: Set[Address], unreachable: Set[Address], replyTo: ActorRef, durable: Boolean): Props =
Props(new TestWriteAggregator(KeyB, data, Some(delta), consistency, probes, nodes, unreachable, replyTo, durable))
class TestWriteAggregator(key: Key.KeyR, data: ReplicatedData, delta: Option[Delta], consistency: Replicator.WriteConsistency,
probes: Map[Address, ActorRef], nodes: Set[Address], unreachable: Set[Address], replyTo: ActorRef, durable: Boolean)
extends WriteAggregator(key, DataEnvelope(data), consistency, None, nodes, unreachable, replyTo, durable) {
extends WriteAggregator(key, DataEnvelope(data), delta, consistency, None, nodes, unreachable, replyTo, durable) {
override def replica(address: Address): ActorSelection =
context.actorSelection(probes(address).path)
@ -48,6 +54,8 @@ object WriteAggregatorSpec {
replicator.foreach(_ ! WriteAck)
case WriteNack
replicator.foreach(_ ! WriteNack)
case DeltaNack
replicator.foreach(_ ! DeltaNack)
case msg
replicator = Some(sender())
replica ! msg
@ -89,6 +97,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
val timeout = 3.seconds.dilated
val writeThree = WriteTo(3, timeout)
val writeMajority = WriteMajority(timeout)
val writeAll = WriteAll(timeout)
def probes(probe: ActorRef): Map[Address, ActorRef] =
nodes.toSeq.map(_ system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
@ -111,7 +120,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
probe.lastSender ! WriteAck
probe.expectMsgType[Write]
probe.lastSender ! WriteAck
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -132,15 +141,14 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
Future.sequence {
Seq(
Future { testProbes(nodeC).expectNoMsg(t) },
Future { testProbes(nodeD).expectNoMsg(t) }
)
Future { testProbes(nodeD).expectNoMsg(t) })
}.futureValue
testProbes(nodeC).expectMsgType[Write]
testProbes(nodeC).lastSender ! WriteAck
testProbes(nodeD).expectMsgType[Write]
testProbes(nodeD).lastSender ! WriteAck
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -158,7 +166,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
// no reply
probe.expectMsgType[Write]
// no reply
expectMsg(UpdateTimeout(WriteAggregatorSpec.key, None))
expectMsg(UpdateTimeout(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -181,6 +189,82 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
}
}
"WriteAggregator with delta" must {
implicit val cluster = Cluster(system)
val fullState1 = ORSet.empty[String] + "a" + "b"
val fullState2 = fullState1.resetDelta + "c"
val delta = Delta(DataEnvelope(fullState2.delta.get), 2L, 2L)
"send deltas first" in {
val probe = TestProbe()
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorPropsWithDelta(
fullState2, delta, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
probe.expectMsgType[DeltaPropagation]
probe.lastSender ! WriteAck
probe.expectMsgType[DeltaPropagation]
probe.lastSender ! WriteAck
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyB, None))
watch(aggr)
expectTerminated(aggr)
}
"retry with full state when no immediate reply or nack" in {
val testProbes = probes()
val testProbeRefs = testProbes.map { case (a, tm) a tm.writeAckAdapter }
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorPropsWithDelta(
fullState2, delta, writeAll, testProbeRefs, nodes, Set.empty, testActor, durable = false))
testProbes(nodeA).expectMsgType[DeltaPropagation]
// no reply
testProbes(nodeB).expectMsgType[DeltaPropagation]
testProbes(nodeB).lastSender ! WriteAck
testProbes(nodeC).expectMsgType[DeltaPropagation]
testProbes(nodeC).lastSender ! WriteAck
testProbes(nodeD).expectMsgType[DeltaPropagation]
testProbes(nodeD).lastSender ! DeltaNack
// here is the second round
testProbes(nodeA).expectMsgType[Write]
testProbes(nodeA).lastSender ! WriteAck
testProbes(nodeD).expectMsgType[Write]
testProbes(nodeD).lastSender ! WriteAck
testProbes(nodeB).expectNoMsg(100.millis)
testProbes(nodeC).expectNoMsg(100.millis)
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyB, None))
watch(aggr)
expectTerminated(aggr)
}
"timeout when less than required acks" in {
val probe = TestProbe()
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorPropsWithDelta(
fullState2, delta, writeAll, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
probe.expectMsgType[DeltaPropagation]
// no reply
probe.expectMsgType[DeltaPropagation]
probe.lastSender ! WriteAck
probe.expectMsgType[DeltaPropagation]
// nack
probe.lastSender ! DeltaNack
probe.expectMsgType[DeltaPropagation]
// no reply
// only 1 ack so we expect 3 full state Write
probe.expectMsgType[Write]
probe.lastSender ! WriteAck
probe.expectMsgType[Write]
probe.expectMsgType[Write]
// still not enough acks
expectMsg(UpdateTimeout(WriteAggregatorSpec.KeyB, None))
watch(aggr)
expectTerminated(aggr)
}
}
"Durable WriteAggregator" must {
"not reply before local confirmation" in {
val probe = TestProbe()
@ -194,9 +278,9 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
expectNoMsg(200.millis)
// the local write
aggr ! UpdateSuccess(WriteAggregatorSpec.key, None)
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None)
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -206,7 +290,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
aggr ! UpdateSuccess(WriteAggregatorSpec.key, None) // the local write
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None) // the local write
probe.expectMsgType[Write]
probe.lastSender ! WriteAck
probe.expectMsgType[Write]
@ -214,7 +298,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
probe.expectMsgType[Write]
probe.lastSender ! WriteAck
expectMsg(UpdateSuccess(WriteAggregatorSpec.key, None))
expectMsg(UpdateSuccess(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -226,7 +310,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
probe.expectMsgType[Write]
probe.lastSender ! WriteNack
aggr ! UpdateSuccess(WriteAggregatorSpec.key, None) // the local write
aggr ! UpdateSuccess(WriteAggregatorSpec.KeyA, None) // the local write
probe.expectMsgType[Write]
probe.lastSender ! WriteAck
probe.expectMsgType[Write]
@ -234,7 +318,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
probe.expectMsgType[Write]
probe.lastSender ! WriteNack
expectMsg(StoreFailure(WriteAggregatorSpec.key, None))
expectMsg(StoreFailure(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}
@ -253,7 +337,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
probe.expectMsgType[Write]
probe.lastSender ! WriteNack
expectMsg(UpdateTimeout(WriteAggregatorSpec.key, None))
expectMsg(UpdateTimeout(WriteAggregatorSpec.KeyA, None))
watch(aggr)
expectTerminated(aggr)
}