Skip to content

Commit a3a37f3

Browse files
committed
Refactor RedundantStatus to encode vector of states that can be merged independently
Also fix: - Truncate command on first access, without participants - Use Ballot.ZERO when invoking CFK.insertOutOfRange where appropriate - Don't supply a command's own route to ProgressLog.waiting to ensure new keys are incorporated - Ensure progress in CommandsForKey by setting vestigial commands to ERASED - Add any missing owned keys to StoreParticipants.route to ensure fetch can make progress - Recovery must wait for earlier not-accepted transactions if either has the privileged coordinator optimisation - Inclusive SyncPoint used incorrect topologies for propose phase - Barrier must not register local listener without up-to-date topology information - Stop home shard truncating a TxnId to vestigial rather than Invalidated so other shards can make progress Also improve: - Validate commands are constructed with non-empty participants - Remove some unnecessary synchronized keywords - Clear ok messages on PreAccept and Accept to free up memory - Introduce TxnId.Cardinality flag so we can optimise single key queries - Update CommandsForKey serialization to better handle larger flag space - Configurable which Txn.Kind can result in a CommandStore being marked stale - Process DefaultProgressLog queue synchronously when relevant state is resident in memory - Remove defunct CollectMaxApplied version of ListStore bootstrap - Standardise linearizability violation reporting - Improve CommandStore.execute method naming to reduce chance of misuse - Prune and address some comments patch by Benedict; reviewed by Alex Petrov for CASSANDRA-20282
1 parent 2936dd7 commit a3a37f3

15 files changed

+109
-71
lines changed

modules/accord

Submodule accord updated 110 files

src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java

+4-4
Original file line numberDiff line numberDiff line change
@@ -472,11 +472,11 @@ public DataSet data()
472472
ds.row(storeId, decompose(start), decompose(end))
473473
.column("start_ownership_epoch", entry.startOwnershipEpoch)
474474
.column("end_ownership_epoch", entry.endOwnershipEpoch)
475-
.column("locally_applied_or_invalidated_before", entry.locallyAppliedOrInvalidatedBefore.toString())
476-
.column("locally_decided_and_applied_or_invalidated_before", entry.locallyDecidedAndAppliedOrInvalidatedBefore.toString())
477-
.column("shard_applied_or_invalidated_before", entry.shardAppliedOrInvalidatedBefore.toString())
475+
.column("locally_applied_before", entry.locallyAppliedBefore.toString())
476+
.column("locally_decided_and_applied_before", entry.locallyDecidedAndAppliedBefore.toString())
477+
.column("shard_applied_before", entry.shardAppliedBefore.toString())
478478
.column("gc_before", entry.gcBefore.toString())
479-
.column("shard_only_applied_or_invalidated_before", entry.shardOnlyAppliedOrInvalidatedBefore.toString())
479+
.column("shard_only_applied_before", entry.shardOnlyAppliedBefore.toString())
480480
.column("bootstrapped_at", entry.bootstrappedAt.toString())
481481
.column("stale_until_at_least", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null);
482482
return ds;

src/java/org/apache/cassandra/service/accord/AccordCommandStore.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -309,7 +309,7 @@ public long nextSystemTimestampMicros()
309309
return lastSystemTimestampMicros;
310310
}
311311
@Override
312-
public <T> AsyncChain<T> submit(PreLoadContext loadCtx, Function<? super SafeCommandStore, T> function)
312+
public <T> AsyncChain<T> build(PreLoadContext loadCtx, Function<? super SafeCommandStore, T> function)
313313
{
314314
return AccordTask.create(this, loadCtx, function).chain();
315315
}
@@ -336,7 +336,7 @@ ProgressLog progressLog()
336336
}
337337

338338
@Override
339-
public AsyncChain<Void> execute(PreLoadContext preLoadContext, Consumer<? super SafeCommandStore> consumer)
339+
public AsyncChain<Void> build(PreLoadContext preLoadContext, Consumer<? super SafeCommandStore> consumer)
340340
{
341341
return AccordTask.create(this, preLoadContext, consumer).chain();
342342
}

src/java/org/apache/cassandra/service/accord/AccordMessageSink.java

+11-11
Original file line numberDiff line numberDiff line change
@@ -71,13 +71,13 @@ public class AccordMessageSink implements MessageSink
7171

7272
public static final class AccordMessageType extends MessageType
7373
{
74-
public static final AccordMessageType INTEROP_READ_REQ = remote("INTEROP_READ_REQ", false);
75-
public static final AccordMessageType INTEROP_READ_RSP = remote("INTEROP_READ_RSP", false);
76-
public static final AccordMessageType INTEROP_STABLE_THEN_READ_REQ = remote("INTEROP_STABLE_THEN_READ_REQ", false);
77-
public static final AccordMessageType INTEROP_READ_REPAIR_REQ = remote("INTEROP_READ_REPAIR_REQ", false);
78-
public static final AccordMessageType INTEROP_READ_REPAIR_RSP = remote("INTEROP_READ_REPAIR_RSP", false);
79-
public static final AccordMessageType INTEROP_APPLY_MINIMAL_REQ = remote("INTEROP_APPLY_MINIMAL_REQ", true );
80-
public static final AccordMessageType INTEROP_APPLY_MAXIMAL_REQ = remote("INTEROP_APPLY_MAXIMAL_REQ", true );
74+
public static final AccordMessageType INTEROP_READ_REQ = remote("INTEROP_READ_REQ");
75+
public static final AccordMessageType INTEROP_READ_RSP = remote("INTEROP_READ_RSP");
76+
public static final AccordMessageType INTEROP_STABLE_THEN_READ_REQ = remote("INTEROP_STABLE_THEN_READ_REQ");
77+
public static final AccordMessageType INTEROP_READ_REPAIR_REQ = remote("INTEROP_READ_REPAIR_REQ");
78+
public static final AccordMessageType INTEROP_READ_REPAIR_RSP = remote("INTEROP_READ_REPAIR_RSP");
79+
public static final AccordMessageType INTEROP_APPLY_MINIMAL_REQ = remote("INTEROP_APPLY_MINIMAL_REQ");
80+
public static final AccordMessageType INTEROP_APPLY_MAXIMAL_REQ = remote("INTEROP_APPLY_MAXIMAL_REQ");
8181

8282
public static final List<MessageType> values;
8383

@@ -101,14 +101,14 @@ public static final class AccordMessageType extends MessageType
101101
values = builder.build();
102102
}
103103

104-
protected static AccordMessageType remote(String name, boolean hasSideEffects)
104+
protected static AccordMessageType remote(String name)
105105
{
106-
return new AccordMessageType(name, REMOTE, hasSideEffects);
106+
return new AccordMessageType(name, REMOTE);
107107
}
108108

109-
private AccordMessageType(String name, MessageType.Kind kind, boolean hasSideEffects)
109+
private AccordMessageType(String name, MessageType.Kind kind)
110110
{
111-
super(name, kind, hasSideEffects);
111+
super(name, kind);
112112
}
113113
}
114114

src/java/org/apache/cassandra/service/accord/AccordObjectSizes.java

+6-4
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import accord.primitives.PartialKeyRoute;
4545
import accord.primitives.PartialRangeRoute;
4646
import accord.primitives.PartialTxn;
47+
import accord.primitives.Participants;
4748
import accord.primitives.Range;
4849
import accord.primitives.RangeDeps;
4950
import accord.primitives.Ranges;
@@ -290,8 +291,9 @@ private static class CommandEmptySizes
290291
private static ICommand attrs(boolean hasDeps, boolean hasTxn, boolean executes)
291292
{
292293
FullKeyRoute route = new FullKeyRoute(EMPTY_KEY, new RoutingKey[]{ EMPTY_KEY });
294+
Participants<?> empty = route.slice(0, 0);
293295
ICommand.Builder builder = new ICommand.Builder(EMPTY_TXNID)
294-
.setParticipants(StoreParticipants.empty(EMPTY_TXNID, route, !executes))
296+
.setParticipants(StoreParticipants.create(route, empty, executes ? empty : null, empty, route))
295297
.durability(Status.Durability.NotDurable)
296298
.executeAt(EMPTY_TXNID)
297299
.promised(Ballot.ZERO);
@@ -316,8 +318,8 @@ private static ICommand attrs(boolean hasDeps, boolean hasTxn, boolean executes)
316318
final static long ACCEPTED = measure(Command.Accepted.accepted(attrs(true, false, false), SaveStatus.AcceptedMedium));
317319
final static long COMMITTED = measure(Command.Committed.committed(attrs(true, true, false), SaveStatus.Committed));
318320
final static long EXECUTED = measure(Command.Executed.executed(attrs(true, true, true), SaveStatus.Applied));
319-
final static long TRUNCATED = measure(Command.Truncated.truncatedApply(attrs(false, false, false), SaveStatus.TruncatedApply, EMPTY_TXNID, null, null));
320-
final static long INVALIDATED = measure(Command.Truncated.invalidated(EMPTY_TXNID, StoreParticipants.empty(EMPTY_TXNID)));
321+
final static long TRUNCATED = measure(Command.Truncated.truncated(attrs(false, false, false), SaveStatus.TruncatedApply, EMPTY_TXNID, null, null));
322+
final static long INVALIDATED = measure(Command.Truncated.invalidated(EMPTY_TXNID, attrs(false, false, false).participants()));
321323

322324
private static long emptySize(Command command)
323325
{
@@ -353,7 +355,7 @@ private static long emptySize(Command command)
353355
case Applied:
354356
return EXECUTED;
355357
case TruncatedApply:
356-
case TruncatedApplyWithDeps:
358+
case TruncatedUnapplied:
357359
case TruncatedApplyWithOutcome:
358360
case Vestigial:
359361
case Erased:

src/java/org/apache/cassandra/service/accord/AccordService.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -173,6 +173,7 @@
173173
import static accord.messages.SimpleReply.Ok;
174174
import static accord.primitives.Routable.Domain.Key;
175175
import static accord.primitives.Routable.Domain.Range;
176+
import static accord.primitives.TxnId.Cardinality.cardinality;
176177
import static accord.utils.Invariants.require;
177178
import static java.util.concurrent.TimeUnit.MILLISECONDS;
178179
import static java.util.concurrent.TimeUnit.NANOSECONDS;
@@ -818,7 +819,7 @@ private static Set<TableId> txnDroppedTables(Seekables<?,?> keys)
818819
@Override
819820
public @Nonnull AsyncTxnResult coordinateAsync(long minEpoch, @Nonnull Txn txn, @Nonnull ConsistencyLevel consistencyLevel, @Nonnull Dispatcher.RequestTime requestTime)
820821
{
821-
TxnId txnId = node.nextTxnId(txn.kind(), txn.keys().domain());
822+
TxnId txnId = node.nextTxnId(txn.kind(), txn.keys().domain(), cardinality(txn.keys()));
822823
ClientRequestMetrics sharedMetrics;
823824
AccordClientRequestMetrics metrics;
824825
if (txn.isWrite())

src/java/org/apache/cassandra/service/accord/api/AccordAgent.java

+5-4
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,8 @@
2020

2121
import java.util.concurrent.TimeUnit;
2222

23+
import javax.annotation.Nullable;
24+
2325
import com.google.common.annotations.VisibleForTesting;
2426
import org.slf4j.Logger;
2527
import org.slf4j.LoggerFactory;
@@ -35,6 +37,7 @@
3537
import accord.local.SafeCommandStore;
3638
import accord.messages.ReplyContext;
3739
import accord.primitives.Keys;
40+
import accord.primitives.Participants;
3841
import accord.primitives.Ranges;
3942
import accord.primitives.Routable;
4043
import accord.primitives.Seekables;
@@ -59,7 +62,6 @@
5962

6063
import static accord.primitives.Routable.Domain.Key;
6164
import static accord.primitives.Txn.Kind.Write;
62-
import static accord.utils.Invariants.illegalState;
6365
import static java.util.concurrent.TimeUnit.MICROSECONDS;
6466
import static java.util.concurrent.TimeUnit.MILLISECONDS;
6567
import static java.util.concurrent.TimeUnit.NANOSECONDS;
@@ -259,9 +261,8 @@ public long expiresAt(ReplyContext replyContext, TimeUnit unit)
259261
}
260262

261263
@Override
262-
public void onViolation(String message)
264+
public void onViolation(String message, Participants<?> participants, @Nullable TxnId notWitnessed, @Nullable Timestamp notWitnessedExecuteAt, @Nullable TxnId by, @Nullable Timestamp byEexecuteAt)
263265
{
264-
try { throw illegalState(message); }
265-
catch (Throwable t) { logger.error("Consistency violation", t); }
266+
logger.error(message);
266267
}
267268
}

src/java/org/apache/cassandra/service/accord/serializers/CommandSerializers.java

+46-17
Original file line numberDiff line numberDiff line change
@@ -94,7 +94,12 @@ public static Timestamp deserialize(TxnId txnId, DataInputPlus in) throws IOExce
9494

9595
long epoch = txnId.epoch();
9696
if((flags & HAS_EPOCH) != 0)
97-
epoch += in.readUnsignedVInt();
97+
{
98+
long delta = in.readUnsignedVInt();
99+
if (delta == 0)
100+
return Timestamp.NONE;
101+
epoch += delta - 1;
102+
}
98103

99104
long hlc = txnId.hlc() + in.readUnsignedVInt();
100105
Node.Id node = new Node.Id(in.readUnsignedVInt32());
@@ -108,8 +113,8 @@ public static void skip(TxnId txnId, DataInputPlus in) throws IOException
108113
int flags = in.readUnsignedVInt32();
109114
if ((flags & 1) != 0)
110115
{
111-
if ((flags & HAS_EPOCH) != 0)
112-
in.readUnsignedVInt();
116+
if ((flags & HAS_EPOCH) != 0 && in.readUnsignedVInt() == 0)
117+
return;
113118
in.readUnsignedVInt();
114119
in.readUnsignedVInt32();
115120
if ((flags & HAS_UNIQUE_HLC) != 0)
@@ -124,7 +129,14 @@ public static void serialize(TxnId txnId, Timestamp executeAt, DataOutputPlus ou
124129
if ((flags & 1) != 0)
125130
{
126131
if ((flags & HAS_EPOCH) != 0)
127-
out.writeUnsignedVInt(executeAt.epoch() - txnId.epoch());
132+
{
133+
if (executeAt.equals(Timestamp.NONE))
134+
{
135+
out.writeUnsignedVInt(0L);
136+
return;
137+
}
138+
out.writeUnsignedVInt(1 + executeAt.epoch() - txnId.epoch());
139+
}
128140
out.writeUnsignedVInt(executeAt.hlc() - txnId.hlc());
129141
out.writeUnsignedVInt32(executeAt.node.id);
130142
if ((flags & HAS_UNIQUE_HLC) != 0)
@@ -152,7 +164,12 @@ public static long serializedSize(TxnId txnId, Timestamp executeAt)
152164
if ((flags & 1) != 0)
153165
{
154166
if ((flags & HAS_EPOCH) != 0)
167+
{
168+
if (executeAt.equals(Timestamp.NONE))
169+
return size + TypeSizes.sizeofUnsignedVInt(0L);
170+
155171
size += TypeSizes.sizeofUnsignedVInt(executeAt.epoch() - txnId.epoch());
172+
}
156173
size += TypeSizes.sizeofUnsignedVInt(executeAt.hlc() - txnId.hlc());
157174
size += TypeSizes.sizeofUnsignedVInt(executeAt.node.id);
158175
if ((flags & HAS_UNIQUE_HLC) != 0)
@@ -176,10 +193,13 @@ private static Timestamp deserialize(DataInputPlus in, boolean nullable) throws
176193
int flags = in.readUnsignedVInt32();
177194
if (nullable)
178195
{
179-
if ((flags & 1) != 0) return null;
180-
flags >>>= 1;
196+
if (flags == 0) return null;
197+
flags--;
181198
}
182199
long epoch = in.readUnsignedVInt();
200+
if (epoch-- == 0)
201+
return Timestamp.NONE;
202+
183203
long hlc = in.readUnsignedVInt();
184204
Node.Id node = new Node.Id(in.readUnsignedVInt32());
185205
if ((flags & HAS_UNIQUE_HLC) == 0)
@@ -206,11 +226,12 @@ private static void skip(DataInputPlus in, boolean nullable) throws IOException
206226
int flags = in.readUnsignedVInt32();
207227
if (nullable)
208228
{
209-
if ((flags & 1) != 0)
229+
if (flags == 0)
210230
return;
211-
flags >>>= 1;
231+
flags--;
212232
}
213-
in.readUnsignedVInt();
233+
if (0 == in.readUnsignedVInt())
234+
return;
214235
in.readUnsignedVInt();
215236
in.readUnsignedVInt32();
216237
if ((flags & HAS_UNIQUE_HLC) != 0)
@@ -235,9 +256,13 @@ private static void serialize(Timestamp executeAt, DataOutputPlus out, boolean n
235256
{
236257
Invariants.require(nullable);
237258
}
259+
else if (executeAt.equals(Timestamp.NONE))
260+
{
261+
out.writeUnsignedVInt(0L);
262+
}
238263
else
239264
{
240-
out.writeUnsignedVInt(executeAt.epoch());
265+
out.writeUnsignedVInt(1 + executeAt.epoch());
241266
out.writeUnsignedVInt(executeAt.hlc());
242267
out.writeUnsignedVInt32(executeAt.node.id);
243268
if (executeAt.hasDistinctHlcAndUniqueHlc())
@@ -264,11 +289,15 @@ private static long serializedSize(Timestamp executeAt, boolean nullable)
264289
Invariants.require(nullable);
265290
return size;
266291
}
267-
size += TypeSizes.sizeofUnsignedVInt(executeAt.epoch());
268-
size += TypeSizes.sizeofUnsignedVInt(executeAt.hlc());
269-
size += TypeSizes.sizeofUnsignedVInt(executeAt.node.id);
270-
if (executeAt.hasDistinctHlcAndUniqueHlc())
271-
size += TypeSizes.sizeofUnsignedVInt(executeAt.uniqueHlc() - executeAt.hlc());
292+
if (executeAt.equals(Timestamp.NONE)) size += TypeSizes.sizeofUnsignedVInt(0);
293+
else
294+
{
295+
size += TypeSizes.sizeofUnsignedVInt(1 + executeAt.epoch());
296+
size += TypeSizes.sizeofUnsignedVInt(executeAt.hlc());
297+
size += TypeSizes.sizeofUnsignedVInt(executeAt.node.id);
298+
if (executeAt.hasDistinctHlcAndUniqueHlc())
299+
size += TypeSizes.sizeofUnsignedVInt(executeAt.uniqueHlc() - executeAt.hlc());
300+
}
272301
return size;
273302
}
274303

@@ -277,7 +306,7 @@ private static int flags(Timestamp executeAt, boolean nullable)
277306
if (executeAt == null)
278307
{
279308
Invariants.require(nullable);
280-
return 1;
309+
return 0;
281310
}
282311

283312
int flags = executeAt.flags() << 2;
@@ -286,7 +315,7 @@ private static int flags(Timestamp executeAt, boolean nullable)
286315
if (executeAt.hasDistinctHlcAndUniqueHlc())
287316
flags |= HAS_UNIQUE_HLC;
288317
if (nullable)
289-
flags <<= 1;
318+
flags++;
290319
return flags;
291320
}
292321
}

src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java

+10-10
Original file line numberDiff line numberDiff line change
@@ -142,11 +142,11 @@ public void serialize(RedundantBefore.Entry t, DataOutputPlus out, int version)
142142
out.writeUnsignedVInt(t.startOwnershipEpoch);
143143
if (t.endOwnershipEpoch == Long.MAX_VALUE) out.writeUnsignedVInt(0L);
144144
else out.writeUnsignedVInt(1 + t.endOwnershipEpoch - t.startOwnershipEpoch);
145-
CommandSerializers.txnId.serialize(t.locallyWitnessedOrInvalidatedBefore, out, version);
146-
CommandSerializers.txnId.serialize(t.locallyAppliedOrInvalidatedBefore, out, version);
147-
CommandSerializers.txnId.serialize(t.locallyDecidedAndAppliedOrInvalidatedBefore, out, version);
148-
CommandSerializers.txnId.serialize(t.shardOnlyAppliedOrInvalidatedBefore, out, version);
149-
CommandSerializers.txnId.serialize(t.shardAppliedOrInvalidatedBefore, out, version);
145+
CommandSerializers.txnId.serialize(t.locallyWitnessedBefore, out, version);
146+
CommandSerializers.txnId.serialize(t.locallyAppliedBefore, out, version);
147+
CommandSerializers.txnId.serialize(t.locallyDecidedAndAppliedBefore, out, version);
148+
CommandSerializers.txnId.serialize(t.shardOnlyAppliedBefore, out, version);
149+
CommandSerializers.txnId.serialize(t.shardAppliedBefore, out, version);
150150
CommandSerializers.txnId.serialize(t.gcBefore, out, version);
151151
CommandSerializers.txnId.serialize(t.bootstrappedAt, out, version);
152152
CommandSerializers.nullableTimestamp.serialize(t.staleUntilAtLeast, out, version);
@@ -177,11 +177,11 @@ public long serializedSize(RedundantBefore.Entry t, int version)
177177
long size = KeySerializers.range.serializedSize(t.range, version);
178178
size += TypeSizes.sizeofUnsignedVInt(t.startOwnershipEpoch);
179179
size += TypeSizes.sizeofUnsignedVInt(t.endOwnershipEpoch == Long.MAX_VALUE ? 0 : 1 + t.endOwnershipEpoch - t.startOwnershipEpoch);
180-
size += CommandSerializers.txnId.serializedSize(t.locallyWitnessedOrInvalidatedBefore, version);
181-
size += CommandSerializers.txnId.serializedSize(t.locallyAppliedOrInvalidatedBefore, version);
182-
size += CommandSerializers.txnId.serializedSize(t.locallyDecidedAndAppliedOrInvalidatedBefore, version);
183-
size += CommandSerializers.txnId.serializedSize(t.shardOnlyAppliedOrInvalidatedBefore, version);
184-
size += CommandSerializers.txnId.serializedSize(t.shardAppliedOrInvalidatedBefore, version);
180+
size += CommandSerializers.txnId.serializedSize(t.locallyWitnessedBefore, version);
181+
size += CommandSerializers.txnId.serializedSize(t.locallyAppliedBefore, version);
182+
size += CommandSerializers.txnId.serializedSize(t.locallyDecidedAndAppliedBefore, version);
183+
size += CommandSerializers.txnId.serializedSize(t.shardOnlyAppliedBefore, version);
184+
size += CommandSerializers.txnId.serializedSize(t.shardAppliedBefore, version);
185185
size += CommandSerializers.txnId.serializedSize(t.gcBefore, version);
186186
size += CommandSerializers.txnId.serializedSize(t.bootstrappedAt, version);
187187
size += CommandSerializers.nullableTimestamp.serializedSize(t.staleUntilAtLeast, version);

test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@
5353
import org.apache.cassandra.utils.concurrent.Condition;
5454
import org.awaitility.Awaitility;
5555

56-
import static accord.primitives.TxnId.FastPath.UNOPTIMISED;
56+
import static accord.primitives.TxnId.FastPath.Unoptimised;
5757
import static org.apache.cassandra.Util.spinUntilSuccess;
5858
import static org.apache.cassandra.service.accord.AccordTestUtils.createTxn;
5959

@@ -136,7 +136,7 @@ public void inflight() throws ExecutionException, InterruptedException
136136
public void blocked() throws ExecutionException, InterruptedException
137137
{
138138
ProtocolModifiers.Toggles.setPermitLocalExecution(false);
139-
ProtocolModifiers.Toggles.setPermittedFastPaths(new TxnId.FastPaths(UNOPTIMISED));
139+
ProtocolModifiers.Toggles.setPermittedFastPaths(new TxnId.FastPaths(Unoptimised));
140140
AccordMsgFilter filter = new AccordMsgFilter();
141141
MessagingService.instance().outboundSink.add(filter);
142142
try

0 commit comments

Comments
 (0)