#21648 Prefer reachable nodes in consistency writes/reads
This commit is contained in:
parent
5c79b81e92
commit
81f081b33c
5 changed files with 99 additions and 49 deletions
|
|
@ -4,9 +4,7 @@
|
|||
package akka.cluster.ddata
|
||||
|
||||
import java.security.MessageDigest
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.Queue
|
||||
import scala.collection.mutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
|
@ -969,7 +967,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
sender() ! reply
|
||||
} else
|
||||
context.actorOf(ReadAggregator.props(key, consistency, req, nodes, localValue, sender())
|
||||
context.actorOf(ReadAggregator.props(key, consistency, req, nodes, unreachable, localValue, sender())
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
}
|
||||
|
||||
|
|
@ -1010,7 +1008,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
sender() ! UpdateSuccess(key, req)
|
||||
} else {
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, sender(), durable)
|
||||
context.actorOf(WriteAggregator.props(key, envelope, writeConsistency, req, nodes, unreachable, sender(), durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, envelope.data,
|
||||
|
|
@ -1100,7 +1098,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
sender() ! DeleteSuccess(key)
|
||||
} else {
|
||||
val writeAggregator =
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, None, nodes, sender(), durable)
|
||||
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, None, nodes, unreachable, sender(), durable)
|
||||
.withDispatcher(context.props.dispatcher))
|
||||
if (durable) {
|
||||
durableStore ! Store(key.id, DeletedData,
|
||||
|
|
@ -1473,6 +1471,8 @@ private[akka] abstract class ReadWriteAggregator extends Actor {
|
|||
|
||||
def timeout: FiniteDuration
|
||||
def nodes: Set[Address]
|
||||
def unreachable: Set[Address]
|
||||
def reachableNodes: Set[Address] = nodes diff unreachable
|
||||
|
||||
import context.dispatcher
|
||||
var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
|
||||
|
|
@ -1487,7 +1487,9 @@ private[akka] abstract class ReadWriteAggregator extends Actor {
|
|||
if (primarySize >= nodes.size)
|
||||
(nodes, Set.empty[Address])
|
||||
else {
|
||||
val (p, s) = scala.util.Random.shuffle(nodes.toVector).splitAt(primarySize)
|
||||
// Prefer to use reachable nodes over the unreachable nodes first
|
||||
val orderedNodes = scala.util.Random.shuffle(reachableNodes.toVector) ++ scala.util.Random.shuffle(unreachable.toVector)
|
||||
val (p, s) = orderedNodes.splitAt(primarySize)
|
||||
(p, s.take(MaxSecondaryNodes))
|
||||
}
|
||||
}
|
||||
|
|
@ -1512,9 +1514,10 @@ private[akka] object WriteAggregator {
|
|||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean): Props =
|
||||
Props(new WriteAggregator(key, envelope, consistency, req, nodes, replyTo, durable))
|
||||
Props(new WriteAggregator(key, envelope, consistency, req, nodes, unreachable, replyTo, durable))
|
||||
.withDeploy(Deploy.local)
|
||||
}
|
||||
|
||||
|
|
@ -1522,13 +1525,14 @@ private[akka] object WriteAggregator {
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class WriteAggregator(
|
||||
key: KeyR,
|
||||
envelope: Replicator.Internal.DataEnvelope,
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean) extends ReadWriteAggregator {
|
||||
key: KeyR,
|
||||
envelope: Replicator.Internal.DataEnvelope,
|
||||
consistency: Replicator.WriteConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
override val unreachable: Set[Address],
|
||||
replyTo: ActorRef,
|
||||
durable: Boolean) extends ReadWriteAggregator {
|
||||
|
||||
import Replicator._
|
||||
import Replicator.Internal._
|
||||
|
|
@ -1616,9 +1620,10 @@ private[akka] object ReadAggregator {
|
|||
consistency: Replicator.ReadConsistency,
|
||||
req: Option[Any],
|
||||
nodes: Set[Address],
|
||||
unreachable: Set[Address],
|
||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||
replyTo: ActorRef): Props =
|
||||
Props(new ReadAggregator(key, consistency, req, nodes, localValue, replyTo))
|
||||
Props(new ReadAggregator(key, consistency, req, nodes, unreachable, localValue, replyTo))
|
||||
.withDeploy(Deploy.local)
|
||||
|
||||
}
|
||||
|
|
@ -1627,12 +1632,13 @@ private[akka] object ReadAggregator {
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ReadAggregator(
|
||||
key: KeyR,
|
||||
consistency: Replicator.ReadConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||
replyTo: ActorRef) extends ReadWriteAggregator {
|
||||
key: KeyR,
|
||||
consistency: Replicator.ReadConsistency,
|
||||
req: Option[Any],
|
||||
override val nodes: Set[Address],
|
||||
override val unreachable: Set[Address],
|
||||
localValue: Option[Replicator.Internal.DataEnvelope],
|
||||
replyTo: ActorRef) extends ReadWriteAggregator {
|
||||
|
||||
import Replicator._
|
||||
import Replicator.Internal._
|
||||
|
|
|
|||
|
|
@ -6,26 +6,29 @@ package akka.cluster.ddata
|
|||
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSelection
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Address
|
||||
import akka.actor.Props
|
||||
import akka.testkit._
|
||||
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.remote.RARP
|
||||
|
||||
import scala.concurrent.Future
|
||||
|
||||
object WriteAggregatorSpec {
|
||||
|
||||
val key = GSetKey[String]("a")
|
||||
|
||||
def writeAggregatorProps(data: GSet[String], consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef], nodes: Set[Address], replyTo: ActorRef, durable: Boolean): Props =
|
||||
Props(new TestWriteAggregator(data, consistency, probes, nodes, replyTo, durable))
|
||||
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))
|
||||
|
||||
class TestWriteAggregator(data: GSet[String], consistency: Replicator.WriteConsistency,
|
||||
probes: Map[Address, ActorRef], nodes: Set[Address], replyTo: ActorRef, durable: Boolean)
|
||||
extends WriteAggregator(key, DataEnvelope(data), consistency, None, nodes, replyTo, durable) {
|
||||
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) {
|
||||
|
||||
override def replica(address: Address): ActorSelection =
|
||||
context.actorSelection(probes(address).path)
|
||||
|
|
@ -50,6 +53,13 @@ object WriteAggregatorSpec {
|
|||
replica ! msg
|
||||
}
|
||||
}
|
||||
|
||||
object TestMock {
|
||||
def apply()(implicit system: ActorSystem) = new TestMock(system)
|
||||
}
|
||||
class TestMock(_application: ActorSystem) extends TestProbe(_application) {
|
||||
val writeAckAdapter = system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(this.ref))
|
||||
}
|
||||
}
|
||||
|
||||
class WriteAggregatorSpec extends AkkaSpec(s"""
|
||||
|
|
@ -62,6 +72,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
}
|
||||
""")
|
||||
with ImplicitSender {
|
||||
import WriteAggregatorSpec._
|
||||
|
||||
val protocol =
|
||||
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
|
||||
|
|
@ -82,11 +93,19 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
def probes(probe: ActorRef): Map[Address, ActorRef] =
|
||||
nodes.toSeq.map(_ → system.actorOf(WriteAggregatorSpec.writeAckAdapterProps(probe))).toMap
|
||||
|
||||
/**
|
||||
* Create a tuple for each node with the WriteAckAdapter and the TestProbe
|
||||
*/
|
||||
def probes(): Map[Address, TestMock] = {
|
||||
val probe = TestProbe()
|
||||
nodes.toSeq.map(_ → TestMock()).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, durable = false))
|
||||
data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
|
|
@ -98,19 +117,29 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
}
|
||||
|
||||
"send to more when no immediate reply" in {
|
||||
val probe = TestProbe()
|
||||
val testProbes = probes()
|
||||
val testProbeRefs = testProbes.map { case (a, tm) ⇒ a → tm.writeAckAdapter }
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor, durable = false))
|
||||
data, writeMajority, testProbeRefs, nodes, Set(nodeC, nodeD), testActor, durable = false))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
testProbes(nodeA).expectMsgType[Write]
|
||||
// no reply
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
testProbes(nodeB).expectMsgType[Write]
|
||||
testProbes(nodeB).lastSender ! WriteAck
|
||||
// Make sure that unreachable nodes do not get a message until 1/5 of the time the reachable nodes did not answer
|
||||
val t = timeout / 5 - 50.milliseconds.dilated
|
||||
import system.dispatcher
|
||||
Future.sequence {
|
||||
Seq(
|
||||
Future { testProbes(nodeC).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))
|
||||
watch(aggr)
|
||||
expectTerminated(aggr)
|
||||
|
|
@ -119,7 +148,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"timeout when less than required acks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor, durable = false))
|
||||
data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = false))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
|
|
@ -139,7 +168,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"not reply before local confirmation" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeThree, probes(probe.ref), nodes, testActor, durable = true))
|
||||
data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteAck
|
||||
|
|
@ -158,7 +187,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"tolerate WriteNack if enough WriteAck" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeThree, probes(probe.ref), nodes, testActor, durable = true))
|
||||
data, writeThree, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
|
||||
aggr ! UpdateSuccess(WriteAggregatorSpec.key, None) // the local write
|
||||
probe.expectMsgType[Write]
|
||||
|
|
@ -176,7 +205,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"reply with StoreFailure when too many nacks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor, durable = true))
|
||||
data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
probe.lastSender ! WriteNack
|
||||
|
|
@ -196,7 +225,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
|
|||
"timeout when less than required acks" in {
|
||||
val probe = TestProbe()
|
||||
val aggr = system.actorOf(WriteAggregatorSpec.writeAggregatorProps(
|
||||
data, writeMajority, probes(probe.ref), nodes, testActor, durable = true))
|
||||
data, writeMajority, probes(probe.ref), nodes, Set.empty, testActor, durable = true))
|
||||
|
||||
probe.expectMsgType[Write]
|
||||
// no reply
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue