Harden ShardCoordinator state replication, #28856 (#28895)

* Possibility to prefer oldest in ddata writes and reads
  * enabled for Cluster Sharding
* New ReadMajorityPlus and WriteMajorityPlus
  * used by Cluster Sharding, with configuration
  * also possible to define ReadAll in config
This commit is contained in:
Patrik Nordwall 2020-04-24 14:19:53 +02:00 committed by GitHub
parent d415211bf8
commit 4ba835d328
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
17 changed files with 767 additions and 196 deletions

View file

@ -1,5 +1,5 @@
/*
* Copyright (C) 2019-2020 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!

View file

@ -1,5 +1,5 @@
/*
* Copyright (C) 2019-2020 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/
// Generated by the protocol buffer compiler. DO NOT EDIT!
@ -73,6 +73,28 @@ public final class ReplicatorMessages {
* <code>optional .akka.cluster.ddata.OtherMessage request = 4;</code>
*/
akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessageOrBuilder getRequestOrBuilder();
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return Whether the consistencyMinCap field is set.
*/
boolean hasConsistencyMinCap();
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return The consistencyMinCap.
*/
int getConsistencyMinCap();
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return Whether the consistencyAdditional field is set.
*/
boolean hasConsistencyAdditional();
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return The consistencyAdditional.
*/
int getConsistencyAdditional();
}
/**
* Protobuf type {@code akka.cluster.ddata.Get}
@ -156,6 +178,16 @@ public final class ReplicatorMessages {
bitField0_ |= 0x00000008;
break;
}
case 40: {
bitField0_ |= 0x00000010;
consistencyMinCap_ = input.readInt32();
break;
}
case 48: {
bitField0_ |= 0x00000020;
consistencyAdditional_ = input.readInt32();
break;
}
default: {
if (!parseUnknownField(
input, unknownFields, extensionRegistry, tag)) {
@ -269,6 +301,40 @@ public final class ReplicatorMessages {
return request_ == null ? akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage.getDefaultInstance() : request_;
}
public static final int CONSISTENCYMINCAP_FIELD_NUMBER = 5;
private int consistencyMinCap_;
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return Whether the consistencyMinCap field is set.
*/
public boolean hasConsistencyMinCap() {
return ((bitField0_ & 0x00000010) != 0);
}
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return The consistencyMinCap.
*/
public int getConsistencyMinCap() {
return consistencyMinCap_;
}
public static final int CONSISTENCYADDITIONAL_FIELD_NUMBER = 6;
private int consistencyAdditional_;
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return Whether the consistencyAdditional field is set.
*/
public boolean hasConsistencyAdditional() {
return ((bitField0_ & 0x00000020) != 0);
}
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return The consistencyAdditional.
*/
public int getConsistencyAdditional() {
return consistencyAdditional_;
}
private byte memoizedIsInitialized = -1;
@java.lang.Override
public final boolean isInitialized() {
@ -317,6 +383,12 @@ public final class ReplicatorMessages {
if (((bitField0_ & 0x00000008) != 0)) {
output.writeMessage(4, getRequest());
}
if (((bitField0_ & 0x00000010) != 0)) {
output.writeInt32(5, consistencyMinCap_);
}
if (((bitField0_ & 0x00000020) != 0)) {
output.writeInt32(6, consistencyAdditional_);
}
unknownFields.writeTo(output);
}
@ -342,6 +414,14 @@ public final class ReplicatorMessages {
size += akka.protobufv3.internal.CodedOutputStream
.computeMessageSize(4, getRequest());
}
if (((bitField0_ & 0x00000010) != 0)) {
size += akka.protobufv3.internal.CodedOutputStream
.computeInt32Size(5, consistencyMinCap_);
}
if (((bitField0_ & 0x00000020) != 0)) {
size += akka.protobufv3.internal.CodedOutputStream
.computeInt32Size(6, consistencyAdditional_);
}
size += unknownFields.getSerializedSize();
memoizedSize = size;
return size;
@ -377,6 +457,16 @@ public final class ReplicatorMessages {
if (!getRequest()
.equals(other.getRequest())) return false;
}
if (hasConsistencyMinCap() != other.hasConsistencyMinCap()) return false;
if (hasConsistencyMinCap()) {
if (getConsistencyMinCap()
!= other.getConsistencyMinCap()) return false;
}
if (hasConsistencyAdditional() != other.hasConsistencyAdditional()) return false;
if (hasConsistencyAdditional()) {
if (getConsistencyAdditional()
!= other.getConsistencyAdditional()) return false;
}
if (!unknownFields.equals(other.unknownFields)) return false;
return true;
}
@ -404,6 +494,14 @@ public final class ReplicatorMessages {
hash = (37 * hash) + REQUEST_FIELD_NUMBER;
hash = (53 * hash) + getRequest().hashCode();
}
if (hasConsistencyMinCap()) {
hash = (37 * hash) + CONSISTENCYMINCAP_FIELD_NUMBER;
hash = (53 * hash) + getConsistencyMinCap();
}
if (hasConsistencyAdditional()) {
hash = (37 * hash) + CONSISTENCYADDITIONAL_FIELD_NUMBER;
hash = (53 * hash) + getConsistencyAdditional();
}
hash = (29 * hash) + unknownFields.hashCode();
memoizedHashCode = hash;
return hash;
@ -555,6 +653,10 @@ public final class ReplicatorMessages {
requestBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000008);
consistencyMinCap_ = 0;
bitField0_ = (bitField0_ & ~0x00000010);
consistencyAdditional_ = 0;
bitField0_ = (bitField0_ & ~0x00000020);
return this;
}
@ -607,6 +709,14 @@ public final class ReplicatorMessages {
}
to_bitField0_ |= 0x00000008;
}
if (((from_bitField0_ & 0x00000010) != 0)) {
result.consistencyMinCap_ = consistencyMinCap_;
to_bitField0_ |= 0x00000010;
}
if (((from_bitField0_ & 0x00000020) != 0)) {
result.consistencyAdditional_ = consistencyAdditional_;
to_bitField0_ |= 0x00000020;
}
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@ -668,6 +778,12 @@ public final class ReplicatorMessages {
if (other.hasRequest()) {
mergeRequest(other.getRequest());
}
if (other.hasConsistencyMinCap()) {
setConsistencyMinCap(other.getConsistencyMinCap());
}
if (other.hasConsistencyAdditional()) {
setConsistencyAdditional(other.getConsistencyAdditional());
}
this.mergeUnknownFields(other.unknownFields);
onChanged();
return this;
@ -1028,6 +1144,80 @@ public final class ReplicatorMessages {
}
return requestBuilder_;
}
private int consistencyMinCap_ ;
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return Whether the consistencyMinCap field is set.
*/
public boolean hasConsistencyMinCap() {
return ((bitField0_ & 0x00000010) != 0);
}
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return The consistencyMinCap.
*/
public int getConsistencyMinCap() {
return consistencyMinCap_;
}
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @param value The consistencyMinCap to set.
* @return This builder for chaining.
*/
public Builder setConsistencyMinCap(int value) {
bitField0_ |= 0x00000010;
consistencyMinCap_ = value;
onChanged();
return this;
}
/**
* <code>optional int32 consistencyMinCap = 5;</code>
* @return This builder for chaining.
*/
public Builder clearConsistencyMinCap() {
bitField0_ = (bitField0_ & ~0x00000010);
consistencyMinCap_ = 0;
onChanged();
return this;
}
private int consistencyAdditional_ ;
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return Whether the consistencyAdditional field is set.
*/
public boolean hasConsistencyAdditional() {
return ((bitField0_ & 0x00000020) != 0);
}
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return The consistencyAdditional.
*/
public int getConsistencyAdditional() {
return consistencyAdditional_;
}
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @param value The consistencyAdditional to set.
* @return This builder for chaining.
*/
public Builder setConsistencyAdditional(int value) {
bitField0_ |= 0x00000020;
consistencyAdditional_ = value;
onChanged();
return this;
}
/**
* <code>optional int32 consistencyAdditional = 6;</code>
* @return This builder for chaining.
*/
public Builder clearConsistencyAdditional() {
bitField0_ = (bitField0_ & ~0x00000020);
consistencyAdditional_ = 0;
onChanged();
return this;
}
@java.lang.Override
public final Builder setUnknownFields(
final akka.protobufv3.internal.UnknownFieldSet unknownFields) {
@ -23709,73 +23899,75 @@ public final class ReplicatorMessages {
static {
java.lang.String[] descriptorData = {
"\n\030ReplicatorMessages.proto\022\022akka.cluster" +
".ddata\"\215\001\n\003Get\022-\n\003key\030\001 \002(\0132 .akka.clust" +
".ddata\"\307\001\n\003Get\022-\n\003key\030\001 \002(\0132 .akka.clust" +
"er.ddata.OtherMessage\022\023\n\013consistency\030\002 \002" +
"(\021\022\017\n\007timeout\030\003 \002(\r\0221\n\007request\030\004 \001(\0132 .a" +
"kka.cluster.ddata.OtherMessage\"\236\001\n\nGetSu" +
"ccess\022-\n\003key\030\001 \002(\0132 .akka.cluster.ddata." +
"OtherMessage\022.\n\004data\030\002 \002(\0132 .akka.cluste" +
"r.ddata.OtherMessage\0221\n\007request\030\004 \001(\0132 ." +
"akka.cluster.ddata.OtherMessage\"l\n\010NotFo" +
"und\022-\n\003key\030\001 \002(\0132 .akka.cluster.ddata.Ot" +
"herMessage\0221\n\007request\030\002 \001(\0132 .akka.clust" +
"er.ddata.OtherMessage\"n\n\nGetFailure\022-\n\003k" +
"ey\030\001 \002(\0132 .akka.cluster.ddata.OtherMessa" +
"ge\0221\n\007request\030\002 \001(\0132 .akka.cluster.ddata" +
".OtherMessage\"G\n\tSubscribe\022-\n\003key\030\001 \002(\0132" +
" .akka.cluster.ddata.OtherMessage\022\013\n\003ref" +
"\030\002 \002(\t\"I\n\013Unsubscribe\022-\n\003key\030\001 \002(\0132 .akk" +
"a.cluster.ddata.OtherMessage\022\013\n\003ref\030\002 \002(" +
"\t\"h\n\007Changed\022-\n\003key\030\001 \002(\0132 .akka.cluster" +
".ddata.OtherMessage\022.\n\004data\030\002 \002(\0132 .akka" +
".cluster.ddata.OtherMessage\"}\n\005Write\022\013\n\003" +
"kka.cluster.ddata.OtherMessage\022\031\n\021consis" +
"tencyMinCap\030\005 \001(\005\022\035\n\025consistencyAddition" +
"al\030\006 \001(\005\"\236\001\n\nGetSuccess\022-\n\003key\030\001 \002(\0132 .a" +
"kka.cluster.ddata.OtherMessage\022.\n\004data\030\002" +
" \002(\0132 .akka.cluster.ddata.OtherMessage\0221" +
"\n\007request\030\004 \001(\0132 .akka.cluster.ddata.Oth" +
"erMessage\"l\n\010NotFound\022-\n\003key\030\001 \002(\0132 .akk" +
"a.cluster.ddata.OtherMessage\0221\n\007request\030" +
"\002 \001(\0132 .akka.cluster.ddata.OtherMessage\"" +
"n\n\nGetFailure\022-\n\003key\030\001 \002(\0132 .akka.cluste" +
"r.ddata.OtherMessage\0221\n\007request\030\002 \001(\0132 ." +
"akka.cluster.ddata.OtherMessage\"G\n\tSubsc" +
"ribe\022-\n\003key\030\001 \002(\0132 .akka.cluster.ddata.O" +
"therMessage\022\013\n\003ref\030\002 \002(\t\"I\n\013Unsubscribe\022" +
"-\n\003key\030\001 \002(\0132 .akka.cluster.ddata.OtherM" +
"essage\022\013\n\003ref\030\002 \002(\t\"h\n\007Changed\022-\n\003key\030\001 " +
"\002(\0132 .akka.cluster.ddata.OtherMessage\022.\n" +
"\004data\030\002 \002(\0132 .akka.cluster.ddata.OtherMe" +
"ssage\"}\n\005Write\022\013\n\003key\030\001 \002(\t\0222\n\010envelope\030" +
"\002 \002(\0132 .akka.cluster.ddata.DataEnvelope\022" +
"3\n\010fromNode\030\003 \001(\0132!.akka.cluster.ddata.U" +
"niqueAddress\"\007\n\005Empty\"H\n\004Read\022\013\n\003key\030\001 \002" +
"(\t\0223\n\010fromNode\030\002 \001(\0132!.akka.cluster.ddat" +
"a.UniqueAddress\"@\n\nReadResult\0222\n\010envelop" +
"e\030\001 \001(\0132 .akka.cluster.ddata.DataEnvelop" +
"e\"\221\003\n\014DataEnvelope\022.\n\004data\030\001 \002(\0132 .akka." +
"cluster.ddata.OtherMessage\022>\n\007pruning\030\002 " +
"\003(\0132-.akka.cluster.ddata.DataEnvelope.Pr" +
"uningEntry\0228\n\rdeltaVersions\030\003 \001(\0132!.akka" +
".cluster.ddata.VersionVector\032\326\001\n\014Pruning" +
"Entry\0229\n\016removedAddress\030\001 \002(\0132!.akka.clu" +
"ster.ddata.UniqueAddress\0227\n\014ownerAddress" +
"\030\002 \002(\0132!.akka.cluster.ddata.UniqueAddres" +
"s\022\021\n\tperformed\030\003 \002(\010\022)\n\004seen\030\004 \003(\0132\033.akk" +
"a.cluster.ddata.Address\022\024\n\014obsoleteTime\030" +
"\005 \001(\022\"\257\001\n\006Status\022\r\n\005chunk\030\001 \002(\r\022\021\n\ttotCh" +
"unks\030\002 \002(\r\0221\n\007entries\030\003 \003(\0132 .akka.clust" +
"er.ddata.Status.Entry\022\023\n\013toSystemUid\030\004 \001" +
"(\020\022\025\n\rfromSystemUid\030\005 \001(\020\032$\n\005Entry\022\013\n\003ke" +
"y\030\001 \002(\t\022\016\n\006digest\030\002 \002(\014\"\303\001\n\006Gossip\022\020\n\010se" +
"ndBack\030\001 \002(\010\0221\n\007entries\030\002 \003(\0132 .akka.clu" +
"ster.ddata.Gossip.Entry\022\023\n\013toSystemUid\030\003" +
" \001(\020\022\025\n\rfromSystemUid\030\004 \001(\020\032H\n\005Entry\022\013\n\003" +
"key\030\001 \002(\t\0222\n\010envelope\030\002 \002(\0132 .akka.clust" +
"er.ddata.DataEnvelope\0223\n\010fromNode\030\003 \001(\0132" +
"!.akka.cluster.ddata.UniqueAddress\"\007\n\005Em" +
"pty\"H\n\004Read\022\013\n\003key\030\001 \002(\t\0223\n\010fromNode\030\002 \001" +
"(\0132!.akka.cluster.ddata.UniqueAddress\"@\n" +
"\nReadResult\0222\n\010envelope\030\001 \001(\0132 .akka.clu" +
"ster.ddata.DataEnvelope\"\221\003\n\014DataEnvelope" +
"\022.\n\004data\030\001 \002(\0132 .akka.cluster.ddata.Othe" +
"rMessage\022>\n\007pruning\030\002 \003(\0132-.akka.cluster" +
".ddata.DataEnvelope.PruningEntry\0228\n\rdelt" +
"aVersions\030\003 \001(\0132!.akka.cluster.ddata.Ver" +
"sionVector\032\326\001\n\014PruningEntry\0229\n\016removedAd" +
"dress\030\001 \002(\0132!.akka.cluster.ddata.UniqueA" +
"ddress\0227\n\014ownerAddress\030\002 \002(\0132!.akka.clus" +
"ter.ddata.UniqueAddress\022\021\n\tperformed\030\003 \002" +
"(\010\022)\n\004seen\030\004 \003(\0132\033.akka.cluster.ddata.Ad" +
"dress\022\024\n\014obsoleteTime\030\005 \001(\022\"\257\001\n\006Status\022\r" +
"\n\005chunk\030\001 \002(\r\022\021\n\ttotChunks\030\002 \002(\r\0221\n\007entr" +
"ies\030\003 \003(\0132 .akka.cluster.ddata.Status.En" +
"try\022\023\n\013toSystemUid\030\004 \001(\020\022\025\n\rfromSystemUi" +
"d\030\005 \001(\020\032$\n\005Entry\022\013\n\003key\030\001 \002(\t\022\016\n\006digest\030" +
"\002 \002(\014\"\303\001\n\006Gossip\022\020\n\010sendBack\030\001 \002(\010\0221\n\007en" +
"tries\030\002 \003(\0132 .akka.cluster.ddata.Gossip." +
"Entry\022\023\n\013toSystemUid\030\003 \001(\020\022\025\n\rfromSystem" +
"Uid\030\004 \001(\020\032H\n\005Entry\022\013\n\003key\030\001 \002(\t\0222\n\010envel" +
"ope\030\002 \002(\0132 .akka.cluster.ddata.DataEnvel" +
"ope\"\201\002\n\020DeltaPropagation\0223\n\010fromNode\030\001 \002" +
"(\0132!.akka.cluster.ddata.UniqueAddress\022;\n" +
"\007entries\030\002 \003(\0132*.akka.cluster.ddata.Delt" +
"aPropagation.Entry\022\r\n\005reply\030\003 \001(\010\032l\n\005Ent" +
"ry\022\013\n\003key\030\001 \002(\t\0222\n\010envelope\030\002 \002(\0132 .akka" +
".cluster.ddata.DataEnvelope\022\021\n\tfromSeqNr" +
"\030\003 \002(\003\022\017\n\007toSeqNr\030\004 \001(\003\"X\n\rUniqueAddress" +
"\022,\n\007address\030\001 \002(\0132\033.akka.cluster.ddata.A" +
"ddress\022\013\n\003uid\030\002 \002(\017\022\014\n\004uid2\030\003 \001(\017\")\n\007Add" +
"ress\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"\224\001\n" +
"\rVersionVector\0228\n\007entries\030\001 \003(\0132\'.akka.c" +
"luster.ddata.VersionVector.Entry\032I\n\005Entr" +
"y\022/\n\004node\030\001 \002(\0132!.akka.cluster.ddata.Uni" +
"queAddress\022\017\n\007version\030\002 \002(\003\"V\n\014OtherMess" +
"age\022\027\n\017enclosedMessage\030\001 \002(\014\022\024\n\014serializ" +
"erId\030\002 \002(\005\022\027\n\017messageManifest\030\004 \001(\014\"\036\n\nS" +
"tringGSet\022\020\n\010elements\030\001 \003(\t\"\205\001\n\023DurableD" +
"ataEnvelope\022.\n\004data\030\001 \002(\0132 .akka.cluster" +
".ddata.OtherMessage\022>\n\007pruning\030\002 \003(\0132-.a" +
"kka.cluster.ddata.DataEnvelope.PruningEn" +
"tryB#\n\037akka.cluster.ddata.protobuf.msgH\001"
"er.ddata.DataEnvelope\"\201\002\n\020DeltaPropagati" +
"on\0223\n\010fromNode\030\001 \002(\0132!.akka.cluster.ddat" +
"a.UniqueAddress\022;\n\007entries\030\002 \003(\0132*.akka." +
"cluster.ddata.DeltaPropagation.Entry\022\r\n\005" +
"reply\030\003 \001(\010\032l\n\005Entry\022\013\n\003key\030\001 \002(\t\0222\n\010env" +
"elope\030\002 \002(\0132 .akka.cluster.ddata.DataEnv" +
"elope\022\021\n\tfromSeqNr\030\003 \002(\003\022\017\n\007toSeqNr\030\004 \001(" +
"\003\"X\n\rUniqueAddress\022,\n\007address\030\001 \002(\0132\033.ak" +
"ka.cluster.ddata.Address\022\013\n\003uid\030\002 \002(\017\022\014\n" +
"\004uid2\030\003 \001(\017\")\n\007Address\022\020\n\010hostname\030\001 \002(\t" +
"\022\014\n\004port\030\002 \002(\r\"\224\001\n\rVersionVector\0228\n\007entr" +
"ies\030\001 \003(\0132\'.akka.cluster.ddata.VersionVe" +
"ctor.Entry\032I\n\005Entry\022/\n\004node\030\001 \002(\0132!.akka" +
".cluster.ddata.UniqueAddress\022\017\n\007version\030" +
"\002 \002(\003\"V\n\014OtherMessage\022\027\n\017enclosedMessage" +
"\030\001 \002(\014\022\024\n\014serializerId\030\002 \002(\005\022\027\n\017messageM" +
"anifest\030\004 \001(\014\"\036\n\nStringGSet\022\020\n\010elements\030" +
"\001 \003(\t\"\205\001\n\023DurableDataEnvelope\022.\n\004data\030\001 " +
"\002(\0132 .akka.cluster.ddata.OtherMessage\022>\n" +
"\007pruning\030\002 \003(\0132-.akka.cluster.ddata.Data" +
"Envelope.PruningEntryB#\n\037akka.cluster.dd" +
"ata.protobuf.msgH\001"
};
descriptor = akka.protobufv3.internal.Descriptors.FileDescriptor
.internalBuildGeneratedFileFrom(descriptorData,
@ -23786,7 +23978,7 @@ public final class ReplicatorMessages {
internal_static_akka_cluster_ddata_Get_fieldAccessorTable = new
akka.protobufv3.internal.GeneratedMessageV3.FieldAccessorTable(
internal_static_akka_cluster_ddata_Get_descriptor,
new java.lang.String[] { "Key", "Consistency", "Timeout", "Request", });
new java.lang.String[] { "Key", "Consistency", "Timeout", "Request", "ConsistencyMinCap", "ConsistencyAdditional", });
internal_static_akka_cluster_ddata_GetSuccess_descriptor =
getDescriptor().getMessageTypes().get(1);
internal_static_akka_cluster_ddata_GetSuccess_fieldAccessorTable = new

View file

@ -0,0 +1,11 @@
# #28856 internal changes to Replicator
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.ddata.Replicator.nodes")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.Replicator.nodes_=")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.ddata.Replicator.weaklyUpNodes")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.Replicator.weaklyUpNodes_=")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.cluster.ddata.Replicator.joiningNodes")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.ddata.Replicator.joiningNodes_=")
ProblemFilters.exclude[Problem]("akka.cluster.ddata.ReadWriteAggregator.*")
ProblemFilters.exclude[Problem]("akka.cluster.ddata.ReadAggregator*")
ProblemFilters.exclude[Problem]("akka.cluster.ddata.WriteAggregator.*")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.protobuf.msg.ReplicatorMessages*")

View file

@ -14,6 +14,8 @@ message Get {
required sint32 consistency = 2;
required uint32 timeout = 3;
optional OtherMessage request = 4;
optional int32 consistencyMinCap = 5;
optional int32 consistencyAdditional = 6;
}
message GetSuccess {

View file

@ -56,6 +56,10 @@ akka.cluster.distributed-data {
# several nodes. If no further activity they are removed from the cache
# after this duration.
serializer-cache-time-to-live = 10s
# Update and Get operations are sent to oldest nodes first.
# This is useful together with Cluster Singleton, which is running on oldest nodes.
prefer-oldest = off
# Settings for delta-CRDT
delta-crdt {

View file

@ -64,7 +64,6 @@ import scala.annotation.varargs
import akka.event.Logging
import akka.util.JavaDurationConverters._
import akka.util.ccompat._
import com.github.ghik.silencer.silent
@ccompatUsedUntil213
@ -91,7 +90,7 @@ object ReplicatorSettings {
import akka.util.ccompat.JavaConverters._
new ReplicatorSettings(
role = roleOption(config.getString("role")),
roles = roleOption(config.getString("role")).toSet,
gossipInterval = config.getDuration("gossip-interval", MILLISECONDS).millis,
notifySubscribersInterval = config.getDuration("notify-subscribers-interval", MILLISECONDS).millis,
maxDeltaElements = config.getInt("max-delta-elements"),
@ -103,7 +102,8 @@ object ReplicatorSettings {
pruningMarkerTimeToLive = config.getDuration("pruning-marker-time-to-live", MILLISECONDS).millis,
durablePruningMarkerTimeToLive = config.getDuration("durable.pruning-marker-time-to-live", MILLISECONDS).millis,
deltaCrdtEnabled = config.getBoolean("delta-crdt.enabled"),
maxDeltaSize = config.getInt("delta-crdt.max-delta-size"))
maxDeltaSize = config.getInt("delta-crdt.max-delta-size"),
preferOldest = config.getBoolean("prefer-oldest"))
}
/**
@ -147,6 +147,7 @@ object ReplicatorSettings {
* @param durableKeys Keys that are durable. Prefix matching is supported by using
* `*` at the end of a key. All entries can be made durable by including "*"
* in the `Set`.
* @param preferOldest Update and Get operations are sent to oldest nodes first.
*/
final class ReplicatorSettings(
val roles: Set[String],
@ -161,7 +162,39 @@ final class ReplicatorSettings(
val pruningMarkerTimeToLive: FiniteDuration,
val durablePruningMarkerTimeToLive: FiniteDuration,
val deltaCrdtEnabled: Boolean,
val maxDeltaSize: Int) {
val maxDeltaSize: Int,
val preferOldest: Boolean) {
// for backwards compatibility
def this(
roles: Set[String],
gossipInterval: FiniteDuration,
notifySubscribersInterval: FiniteDuration,
maxDeltaElements: Int,
dispatcher: String,
pruningInterval: FiniteDuration,
maxPruningDissemination: FiniteDuration,
durableStoreProps: Either[(String, Config), Props],
durableKeys: Set[KeyId],
pruningMarkerTimeToLive: FiniteDuration,
durablePruningMarkerTimeToLive: FiniteDuration,
deltaCrdtEnabled: Boolean,
maxDeltaSize: Int) =
this(
roles,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
dispatcher,
pruningInterval,
maxPruningDissemination,
durableStoreProps,
durableKeys,
pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive,
deltaCrdtEnabled,
maxDeltaSize,
preferOldest = false)
// for backwards compatibility
def this(
@ -179,7 +212,7 @@ final class ReplicatorSettings(
deltaCrdtEnabled: Boolean,
maxDeltaSize: Int) =
this(
role.iterator.toSet,
role.toSet,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
@ -203,7 +236,7 @@ final class ReplicatorSettings(
pruningInterval: FiniteDuration,
maxPruningDissemination: FiniteDuration) =
this(
roles = role.iterator.toSet,
roles = role.toSet,
gossipInterval,
notifySubscribersInterval,
maxDeltaElements,
@ -272,9 +305,9 @@ final class ReplicatorSettings(
deltaCrdtEnabled,
200)
def withRole(role: String): ReplicatorSettings = copy(roles = ReplicatorSettings.roleOption(role).iterator.toSet)
def withRole(role: String): ReplicatorSettings = copy(roles = ReplicatorSettings.roleOption(role).toSet)
def withRole(role: Option[String]): ReplicatorSettings = copy(roles = role.iterator.toSet)
def withRole(role: Option[String]): ReplicatorSettings = copy(roles = role.toSet)
@varargs
def withRoles(roles: String*): ReplicatorSettings = copy(roles = roles.toSet)
@ -337,6 +370,9 @@ final class ReplicatorSettings(
def withMaxDeltaSize(maxDeltaSize: Int): ReplicatorSettings =
copy(maxDeltaSize = maxDeltaSize)
def withPreferOldest(preferOldest: Boolean): ReplicatorSettings =
copy(preferOldest = preferOldest)
private def copy(
roles: Set[String] = roles,
gossipInterval: FiniteDuration = gossipInterval,
@ -350,7 +386,8 @@ final class ReplicatorSettings(
pruningMarkerTimeToLive: FiniteDuration = pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive: FiniteDuration = durablePruningMarkerTimeToLive,
deltaCrdtEnabled: Boolean = deltaCrdtEnabled,
maxDeltaSize: Int = maxDeltaSize): ReplicatorSettings =
maxDeltaSize: Int = maxDeltaSize,
preferOldest: Boolean = preferOldest): ReplicatorSettings =
new ReplicatorSettings(
roles,
gossipInterval,
@ -364,7 +401,8 @@ final class ReplicatorSettings(
pruningMarkerTimeToLive,
durablePruningMarkerTimeToLive,
deltaCrdtEnabled,
maxDeltaSize)
maxDeltaSize,
preferOldest)
}
object Replicator {
@ -403,6 +441,19 @@ object Replicator {
*/
def this(timeout: java.time.Duration) = this(timeout.asScala, DefaultMajorityMinCap)
}
/**
* `ReadMajority` but with the given number of `additional` nodes added to the majority count. At most
* all nodes.
*/
final case class ReadMajorityPlus(timeout: FiniteDuration, additional: Int, minCap: Int = DefaultMajorityMinCap)
extends ReadConsistency {
/**
* Java API
*/
def this(timeout: java.time.Duration, additional: Int) = this(timeout.asScala, additional, DefaultMajorityMinCap)
}
final case class ReadAll(timeout: FiniteDuration) extends ReadConsistency {
/**
@ -434,6 +485,19 @@ object Replicator {
*/
def this(timeout: java.time.Duration) = this(timeout.asScala, DefaultMajorityMinCap)
}
/**
* `WriteMajority` but with the given number of `additional` nodes added to the majority count. At most
* all nodes.
*/
final case class WriteMajorityPlus(timeout: FiniteDuration, additional: Int, minCap: Int = DefaultMajorityMinCap)
extends WriteConsistency {
/**
* Java API
*/
def this(timeout: java.time.Duration, additional: Int) = this(timeout.asScala, additional, DefaultMajorityMinCap)
}
final case class WriteAll(timeout: FiniteDuration) extends WriteConsistency {
/**
@ -1008,7 +1072,9 @@ object Replicator {
extends ReplicatorMessage
with DestinationSystemUid
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long)
final case class Delta(dataEnvelope: DataEnvelope, fromSeqNr: Long, toSeqNr: Long) {
def requiresCausalDeliveryOfDeltas: Boolean = dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
}
final case class DeltaPropagation(_fromNode: UniqueAddress, reply: Boolean, deltas: Map[KeyId, Delta])
extends ReplicatorMessage
with SendingSystemUid {
@ -1288,8 +1354,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val deltaPropagationSelector = new DeltaPropagationSelector {
override val gossipIntervalDivisor = 5
override def allNodes: Vector[UniqueAddress] = {
// TODO optimize, by maintaining a sorted instance variable instead
Replicator.this.allNodes.diff(unreachable).toVector.sorted
// Replicator.allNodes is sorted
Replicator.this.allNodes.diff(unreachable).toVector
}
override def maxDeltaSize: Int = settings.maxDeltaSize
@ -1321,16 +1387,20 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
} else None
// cluster nodes, doesn't contain selfAddress, doesn't contain joining and weaklyUp
var nodes: Set[UniqueAddress] = Set.empty
var nodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// cluster members sorted by age, oldest first,, doesn't contain selfAddress, doesn't contain joining and weaklyUp
// only used when prefer-oldest is enabled
var membersByAge: immutable.SortedSet[Member] = immutable.SortedSet.empty(Member.ageOrdering)
// cluster weaklyUp nodes, doesn't contain selfAddress
var weaklyUpNodes: Set[UniqueAddress] = Set.empty
var weaklyUpNodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// cluster joining nodes, doesn't contain selfAddress
var joiningNodes: Set[UniqueAddress] = Set.empty
var joiningNodes: immutable.SortedSet[UniqueAddress] = immutable.SortedSet.empty
// up and weaklyUp nodes, doesn't contain joining and not selfAddress
private def allNodes: Set[UniqueAddress] = nodes.union(weaklyUpNodes)
private def allNodes: immutable.SortedSet[UniqueAddress] = nodes.union(weaklyUpNodes)
private def isKnownNode(node: UniqueAddress): Boolean =
nodes(node) || weaklyUpNodes(node) || joiningNodes(node) || selfUniqueAddress == node
@ -1370,6 +1440,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
// messages after loading durable data.
var replyTo: ActorRef = null
private def nodesForReadWrite(): Vector[UniqueAddress] = {
if (settings.preferOldest)
membersByAge.iterator.map(_.uniqueAddress).toVector
else
nodes.toVector
}
override protected[akka] def aroundReceive(rcv: Actor.Receive, msg: Any): Unit = {
replyTo = sender()
try {
@ -1547,7 +1624,16 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
} else {
context.actorOf(
ReadAggregator
.props(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo)
.props(
key,
consistency,
req,
selfUniqueAddress,
nodesForReadWrite(),
unreachable,
!settings.preferOldest,
localValue,
replyTo)
.withDispatcher(context.props.dispatcher))
}
}
@ -1632,6 +1718,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case Some(d) => (newEnvelope.copy(data = d), None)
case None => (newEnvelope, None)
}
// When RequiresCausalDeliveryOfDeltas use deterministic order to so that sequence numbers
// of subsequent updates are in sync on the destination nodes.
// The order is also kept when prefer-oldest is enabled.
val shuffle = !(settings.preferOldest || writeDelta.exists(_.requiresCausalDeliveryOfDeltas))
val writeAggregator =
context.actorOf(
WriteAggregator
@ -1642,8 +1732,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
writeConsistency,
req,
selfUniqueAddress,
nodes,
nodesForReadWrite(),
unreachable,
shuffle,
replyTo,
durable)
.withDispatcher(context.props.dispatcher))
@ -1758,8 +1849,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency,
req,
selfUniqueAddress,
nodes,
nodesForReadWrite(),
unreachable,
!settings.preferOldest,
replyTo,
durable)
.withDispatcher(context.props.dispatcher))
@ -2106,6 +2198,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
nodes += m.uniqueAddress
weaklyUpNodes -= m.uniqueAddress
joiningNodes -= m.uniqueAddress
if (settings.preferOldest)
membersByAge += m
}
}
@ -2121,6 +2215,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
joiningNodes -= m.uniqueAddress
removedNodes = removedNodes.updated(m.uniqueAddress, allReachableClockTime)
unreachable -= m.uniqueAddress
if (settings.preferOldest)
membersByAge -= m
deltaPropagationSelector.cleanupRemovedNode(m.uniqueAddress)
}
}
@ -2260,14 +2356,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case object SendToSecondary
val MaxSecondaryNodes = 10
def calculateMajorityWithMinCap(minCap: Int, numberOfNodes: Int): Int = {
if (numberOfNodes <= minCap) {
numberOfNodes
} else {
val majority = numberOfNodes / 2 + 1
if (majority <= minCap) minCap
else majority
}
def calculateMajority(minCap: Int, numberOfNodes: Int, additional: Int): Int = {
val majority = numberOfNodes / 2 + 1
math.min(numberOfNodes, math.max(majority + additional, minCap))
}
}
@ -2278,30 +2369,33 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
import ReadWriteAggregator._
def timeout: FiniteDuration
def nodes: Set[UniqueAddress]
def nodes: Vector[UniqueAddress]
def unreachable: Set[UniqueAddress]
def reachableNodes: Set[UniqueAddress] = nodes.diff(unreachable)
def reachableNodes: Vector[UniqueAddress] = nodes.filterNot(unreachable)
def shuffle: Boolean
import context.dispatcher
var sendToSecondarySchedule = context.system.scheduler.scheduleOnce(timeout / 5, self, SendToSecondary)
var timeoutSchedule = context.system.scheduler.scheduleOnce(timeout, self, ReceiveTimeout)
var remaining = nodes.map(_.address)
var remaining = nodes.iterator.map(_.address).toSet
def doneWhenRemainingSize: Int
def primaryAndSecondaryNodes(
requiresCausalDeliveryOfDeltas: Boolean): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
def primaryAndSecondaryNodes(): (Vector[UniqueAddress], Vector[UniqueAddress]) = {
val primarySize = nodes.size - doneWhenRemainingSize
if (primarySize >= nodes.size)
(nodes.toVector, Vector.empty[UniqueAddress])
(nodes, Vector.empty[UniqueAddress])
else {
// Prefer to use reachable nodes over the unreachable nodes first.
// When RequiresCausalDeliveryOfDeltas use deterministic order to so that sequence numbers of subsequent
// updates are in sync on the destination nodes.
// When RequiresCausalDeliveryOfDeltas (shuffle=false) use deterministic order to so that sequence numbers
// of subsequent updates are in sync on the destination nodes.
// The order is also kept when prefer-oldest is enabled.
val orderedNodes =
if (requiresCausalDeliveryOfDeltas) reachableNodes.toVector.sorted ++ unreachable.toVector.sorted
else scala.util.Random.shuffle(reachableNodes.toVector) ++ scala.util.Random.shuffle(unreachable.toVector)
if (shuffle)
scala.util.Random.shuffle(reachableNodes) ++ scala.util.Random.shuffle(unreachable.toVector)
else
reachableNodes ++ unreachable
val (p, s) = orderedNodes.splitAt(primarySize)
(p, s.take(MaxSecondaryNodes))
}
@ -2328,8 +2422,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.WriteConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
shuffle: Boolean,
replyTo: ActorRef,
durable: Boolean): Props =
Props(
@ -2342,6 +2437,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
selfUniqueAddress,
nodes,
unreachable,
shuffle,
replyTo,
durable)).withDeploy(Deploy.local)
}
@ -2356,8 +2452,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.WriteConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
override val nodes: Set[UniqueAddress],
override val nodes: Vector[UniqueAddress],
override val unreachable: Set[UniqueAddress],
override val shuffle: Boolean,
replyTo: ActorRef,
durable: Boolean)
extends ReadWriteAggregator
@ -2370,11 +2467,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
override def timeout: FiniteDuration = consistency.timeout
override val doneWhenRemainingSize = consistency match {
case WriteTo(n, _) => nodes.size - (n - 1)
case _: WriteAll => 0
case WriteTo(n, _) => nodes.size - (n - 1)
case _: WriteAll => 0
case WriteMajority(_, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val w = calculateMajorityWithMinCap(minCap, N)
val w = calculateMajority(minCap, N, 0)
log.debug("WriteMajority [{}] [{}] of [{}].", key, w, N)
N - w
case WriteMajorityPlus(_, additional, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val w = calculateMajority(minCap, N, additional)
log.debug("WriteMajorityPlus [{}] [{}] of [{}].", key, w, N)
N - w
case WriteLocal =>
throw new IllegalArgumentException("WriteLocal not supported by WriteAggregator")
@ -2389,13 +2494,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var gotLocalStoreReply = !durable
var gotWriteNackFrom = Set.empty[Address]
private val (primaryNodes, secondaryNodes) = {
val requiresCausalDeliveryOfDeltas = delta match {
case None => false
case Some(d) => d.dataEnvelope.data.isInstanceOf[RequiresCausalDeliveryOfDeltas]
}
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas)
}
private val (primaryNodes, secondaryNodes) = primaryAndSecondaryNodes()
override def preStart(): Unit = {
val msg = deltaMsg match {
@ -2479,11 +2578,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.ReadConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
shuffle: Boolean,
localValue: Option[Replicator.Internal.DataEnvelope],
replyTo: ActorRef): Props =
Props(new ReadAggregator(key, consistency, req, selfUniqueAddress, nodes, unreachable, localValue, replyTo))
Props(
new ReadAggregator(key, consistency, req, selfUniqueAddress, nodes, unreachable, shuffle, localValue, replyTo))
.withDeploy(Deploy.local)
}
@ -2496,11 +2597,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
consistency: Replicator.ReadConsistency,
req: Option[Any],
selfUniqueAddress: UniqueAddress,
override val nodes: Set[UniqueAddress],
override val nodes: Vector[UniqueAddress],
override val unreachable: Set[UniqueAddress],
override val shuffle: Boolean,
localValue: Option[Replicator.Internal.DataEnvelope],
replyTo: ActorRef)
extends ReadWriteAggregator {
extends ReadWriteAggregator
with ActorLogging {
import Replicator._
import Replicator.Internal._
@ -2510,11 +2613,19 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
var result = localValue
override val doneWhenRemainingSize = consistency match {
case ReadFrom(n, _) => nodes.size - (n - 1)
case _: ReadAll => 0
case ReadFrom(n, _) => nodes.size - (n - 1)
case _: ReadAll => 0
case ReadMajority(_, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val r = calculateMajorityWithMinCap(minCap, N)
val r = calculateMajority(minCap, N, 0)
log.debug("ReadMajority [{}] [{}] of [{}].", key, r, N)
N - r
case ReadMajorityPlus(_, additional, minCap) =>
// +1 because local node is not included in `nodes`
val N = nodes.size + 1
val r = calculateMajority(minCap, N, additional)
log.debug("ReadMajorityPlus [{}] [{}] of [{}].", key, r, N)
N - r
case ReadLocal =>
throw new IllegalArgumentException("ReadLocal not supported by ReadAggregator")
@ -2522,9 +2633,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val readMsg = Read(key.id, Some(selfUniqueAddress))
private val (primaryNodes, secondaryNodes) = {
primaryAndSecondaryNodes(requiresCausalDeliveryOfDeltas = false)
}
private val (primaryNodes, secondaryNodes) = primaryAndSecondaryNodes()
override def preStart(): Unit = {
primaryNodes.foreach { replica(_) ! readMsg }

View file

@ -338,21 +338,27 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
}
private def getToProto(get: Get[_]): dm.Get = {
val consistencyValue = get.consistency match {
case ReadLocal => 1
case ReadFrom(n, _) => n
case _: ReadMajority => 0
case _: ReadAll => -1
}
val timoutInMillis = get.consistency.timeout.toMillis
require(timoutInMillis <= 0XFFFFFFFFL, "Timeouts must fit in a 32-bit unsigned int")
val b = dm.Get
.newBuilder()
.setKey(otherMessageToProto(get.key))
.setConsistency(consistencyValue)
.setTimeout(timoutInMillis.toInt)
val b = dm.Get.newBuilder().setKey(otherMessageToProto(get.key)).setTimeout(timoutInMillis.toInt)
get.consistency match {
case ReadLocal => b.setConsistency(1)
case ReadFrom(n, _) => b.setConsistency(n)
case ReadMajority(_, minCap) =>
b.setConsistency(0)
if (minCap != 0)
b.setConsistencyMinCap(minCap)
case ReadMajorityPlus(_, additional, minCap) =>
b.setConsistency(0)
if (minCap != 0)
b.setConsistencyMinCap(minCap)
if (additional != 0)
b.setConsistencyAdditional(additional)
case _: ReadAll =>
b.setConsistency(-1)
}
get.request.foreach(o => b.setRequest(otherMessageToProto(o)))
b.build()
@ -367,8 +373,13 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
val timeout =
if (get.getTimeout < 0) Duration(Int.MaxValue.toLong + (get.getTimeout - Int.MaxValue), TimeUnit.MILLISECONDS)
else Duration(get.getTimeout.toLong, TimeUnit.MILLISECONDS)
def minCap = if (get.hasConsistencyMinCap) get.getConsistencyMinCap else 0
val consistency = get.getConsistency match {
case 0 => ReadMajority(timeout)
case 0 =>
if (get.hasConsistencyAdditional)
ReadMajorityPlus(timeout, get.getConsistencyAdditional, minCap)
else
ReadMajority(timeout, minCap)
case -1 => ReadAll(timeout)
case 1 => ReadLocal
case n => ReadFrom(n, timeout)

View file

@ -67,6 +67,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
val KeyH = ORMapKey[String, Flag]("H")
val KeyI = GSetKey[String]("I")
val KeyJ = GSetKey[String]("J")
val KeyK = LWWRegisterKey[String]("K")
val KeyX = GCounterKey("X")
val KeyY = GCounterKey("Y")
val KeyZ = GCounterKey("Z")
@ -591,4 +592,48 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
enterBarrierAfterTestStep()
}
"support prefer oldest members" in {
// disable gossip and delta replication to only verify the write and read operations
val oldestReplicator = system.actorOf(
Replicator.props(
ReplicatorSettings(system).withPreferOldest(true).withGossipInterval(1.minute).withDeltaCrdtEnabled(false)),
"oldestReplicator")
within(5.seconds) {
val countProbe = TestProbe()
awaitAssert {
oldestReplicator.tell(GetReplicaCount, countProbe.ref)
countProbe.expectMsg(ReplicaCount(3))
}
}
enterBarrier("oldest-replicator-started")
val probe = TestProbe()
runOn(second) {
oldestReplicator.tell(
Update(KeyK, LWWRegister(selfUniqueAddress, "0"), writeTwo)(_.withValue(selfUniqueAddress, "1")),
probe.ref)
probe.expectMsg(UpdateSuccess(KeyK, None))
}
enterBarrier("updated-1")
runOn(first) {
// replicated to oldest
oldestReplicator.tell(Get(KeyK, ReadLocal), probe.ref)
probe.expectMsgType[GetSuccess[LWWRegister[String]]].dataValue.value should ===("1")
}
runOn(third) {
// not replicated to third (not among the two oldest)
oldestReplicator.tell(Get(KeyK, ReadLocal), probe.ref)
probe.expectMsg(NotFound(KeyK, None))
// read from oldest
oldestReplicator.tell(Get(KeyK, readTwo), probe.ref)
probe.expectMsgType[GetSuccess[LWWRegister[String]]].dataValue.value should ===("1")
}
enterBarrierAfterTestStep()
}
}

View file

@ -31,7 +31,7 @@ object WriteAggregatorSpec {
consistency: Replicator.WriteConsistency,
probes: Map[UniqueAddress, ActorRef],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
replyTo: ActorRef,
durable: Boolean): Props =
@ -54,7 +54,7 @@ object WriteAggregatorSpec {
consistency: Replicator.WriteConsistency,
probes: Map[UniqueAddress, ActorRef],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
replyTo: ActorRef,
durable: Boolean): Props =
@ -78,7 +78,7 @@ object WriteAggregatorSpec {
consistency: Replicator.WriteConsistency,
probes: Map[UniqueAddress, ActorRef],
selfUniqueAddress: UniqueAddress,
nodes: Set[UniqueAddress],
nodes: Vector[UniqueAddress],
unreachable: Set[UniqueAddress],
replyTo: ActorRef,
durable: Boolean)
@ -91,6 +91,7 @@ object WriteAggregatorSpec {
selfUniqueAddress,
nodes,
unreachable,
shuffle = false,
replyTo,
durable) {
@ -148,7 +149,7 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
val nodeC = UniqueAddress(Address(protocol, "Sys", "c", 2552), 17L)
val nodeD = UniqueAddress(Address(protocol, "Sys", "d", 2552), 17L)
// 4 replicas + the local => 5
val nodes = Set(nodeA, nodeB, nodeC, nodeD)
val nodes = Vector(nodeA, nodeB, nodeC, nodeD)
val data = GSet.empty + "A" + "B"
val timeout = 3.seconds.dilated
@ -256,16 +257,42 @@ class WriteAggregatorSpec extends AkkaSpec(s"""
import ReadWriteAggregator._
calculateMajorityWithMinCap(minCap, 3) should be(3)
calculateMajorityWithMinCap(minCap, 4) should be(4)
calculateMajorityWithMinCap(minCap, 5) should be(5)
calculateMajorityWithMinCap(minCap, 6) should be(5)
calculateMajorityWithMinCap(minCap, 7) should be(5)
calculateMajorityWithMinCap(minCap, 8) should be(5)
calculateMajorityWithMinCap(minCap, 9) should be(5)
calculateMajorityWithMinCap(minCap, 10) should be(6)
calculateMajorityWithMinCap(minCap, 11) should be(6)
calculateMajorityWithMinCap(minCap, 12) should be(7)
calculateMajority(minCap, 3, 0) should be(3)
calculateMajority(minCap, 4, 0) should be(4)
calculateMajority(minCap, 5, 0) should be(5)
calculateMajority(minCap, 6, 0) should be(5)
calculateMajority(minCap, 7, 0) should be(5)
calculateMajority(minCap, 8, 0) should be(5)
calculateMajority(minCap, 9, 0) should be(5)
calculateMajority(minCap, 10, 0) should be(6)
calculateMajority(minCap, 11, 0) should be(6)
calculateMajority(minCap, 12, 0) should be(7)
}
"calculate majority with additional" in {
import ReadWriteAggregator._
calculateMajority(0, 3, 1) should be(3)
calculateMajority(0, 3, 2) should be(3)
calculateMajority(0, 4, 1) should be(4)
calculateMajority(0, 5, 1) should be(4)
calculateMajority(0, 5, 2) should be(5)
calculateMajority(0, 6, 1) should be(5)
calculateMajority(0, 7, 1) should be(5)
calculateMajority(0, 8, 1) should be(6)
calculateMajority(0, 8, 2) should be(7)
calculateMajority(0, 9, 1) should be(6)
calculateMajority(0, 10, 1) should be(7)
calculateMajority(0, 11, 1) should be(7)
calculateMajority(0, 11, 3) should be(9)
}
"calculate majority with additional and minCap" in {
import ReadWriteAggregator._
calculateMajority(5, 9, 1) should be(6)
calculateMajority(7, 9, 1) should be(7)
calculateMajority(10, 9, 1) should be(9)
}
}

View file

@ -76,6 +76,7 @@ class ReplicatorMessageSerializerSpec
checkSerialization(Get(keyA, ReadLocal))
checkSerialization(Get(keyA, ReadMajority(2.seconds), Some("x")))
checkSerialization(Get(keyA, ReadMajority((Int.MaxValue.toLong + 50).milliseconds), Some("x")))
checkSerialization(Get(keyA, ReadMajority(2.seconds, minCap = 3), Some("x")))
try {
serializer.toBinary(Get(keyA, ReadMajority((Int.MaxValue.toLong * 3).milliseconds), Some("x")))
fail("Our protobuf protocol does not support timeouts larger than unsigned ints")
@ -83,6 +84,8 @@ class ReplicatorMessageSerializerSpec
case e: IllegalArgumentException =>
e.getMessage should include("unsigned int")
}
checkSerialization(Get(keyA, ReadMajorityPlus(2.seconds, 3), Some("x")))
checkSerialization(Get(keyA, ReadMajorityPlus(2.seconds, 3, 5), Some("x")))
checkSerialization(GetSuccess(keyA, None)(data1))
checkSerialization(GetSuccess(keyA, Some("x"))(data1))
checkSerialization(NotFound(keyA, Some("x")))