Skip to content

Conversation

@bdeggleston
Copy link
Member

No description provided.

bdeggleston and others added 30 commits January 25, 2023 11:29
patch by Blake Eggleston; reviewed by Benedict Elliott Smith, David Capwell for CASSANDRA-17103
…an rely on a version column, and fixed a few feedback related issues

patch by David Capwell; reviewed by Blake Eggleston for CASSANDRA-17103
The replication of transactions and their processing state must be optimal, i.e. linearly proportional to the size of the transaction and the number of keys involved. This patch ensures a transaction is replicated in slices, with each slice being written (replication factor) times only to the shard that owns the keys, and only once (except in the case the transaction is recovered).

patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-17109
Ensure old or abandoned transactions are invalidated wherever possible.

patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-18041
…ce dedicated property tests; re-activate fast-path during range movements

patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-18056
 - Integrate accord-core changes for CASSANDRA-18057

patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-18057
  - Integrate accord-core changes for CASSANDRA-18087

patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-18087
patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-17719

Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com>
Co-authored-by: Blake Eggleston <bdeggleston@gmail.com>
Co-authored-by: David Capwell <dcapwell@apache.org>
…ion, will revert before trunk but needed for debugging
make sure to update this.kind in order to make tests stable
AsyncWriterTest#commandsPerKeyDenormalization to use SaveStatus rather
than Status
…decreaseSlightly to get back to the old behavior for now
…utingKey was added but does not work in all cases in C* due to sentinal values, added logic to return a C* friendly token
Some tests may be flaky now due to Preempted being thrown.
…ng IN clauses to return multiple partitions/rows

patch by David Capwell; reviewed by Caleb Rackliffe for CASSANDRA-18154
patch by Benedict; reviewed by Aleksey Yeschenko for CASSANDRA-18142
 - Combine real and logical into a single 64-but HLC
 - Introduce 16 flag bits
 - Pack epoch (48-bits), HLC (64-bits) and flags (16-bits) into two longs in memory

patch by Benedict; reviewed by Aleksey Yeschenko for CASSANDRA-18172
patch by Ariel Weisberg; Reviewed by Caleb Rackliffe for CASSANDRA-18100
…and clustering IN clauses to return multiple partitions/rows
Refactor Deps into KeyDeps and RangeDeps

patch by Benedict; reviewed by Aleksey Yeschenko for CASSANDRA-18173
patch by Benedict; reviewed by Ariel Weisberg for CASSANDRA-18174
patch by Caleb Rackliffe; reviewed by David Capwell for CASSANDRA-18195
jacek-lewandowski and others added 2 commits February 16, 2023 19:36
patch by <jacek-lewandowski>; reviewed by <maedhroz> and <dcapwell> for CASSANDRA-18241
patch by David Capwell; reviewed by Caleb Rackliffe, Michael Semb Wever for CASSANDRA-18204
@dcapwell
Copy link
Contributor

Ran stress and here are exceptions that I saw

java.lang.NullPointerException: null
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:718)
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:651)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:322)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:309)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:207)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)
        at accord.utils.async.AsyncCallbacks.lambda$null$0(AsyncCallbacks.java:31)
        at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)
java.lang.NullPointerException: null
        at accord.primitives.Range$EndInclusive.compareTo(Range.java:47)
        at accord.primitives.AbstractKeys.lambda$findNextIntersection$1(AbstractKeys.java:130)
        at accord.utils.SortedArrays.exponentialSearch(SortedArrays.java:604)
        at accord.utils.SortedArrays.findNextIntersection(SortedArrays.java:801)
        at accord.utils.SortedArrays.findNextIntersectionWithMultipleMatches(SortedArrays.java:744)
        at accord.primitives.AbstractKeys.findNextIntersection(AbstractKeys.java:130)
        at accord.topology.Topology.subsetFor(Topology.java:257)
        at accord.topology.Topology.forSelection(Topology.java:203)
        at accord.topology.Topology.forSelection(Topology.java:198)
        at accord.topology.TopologyManager.preciseEpochs(TopologyManager.java:359)
        at accord.coordinate.CheckShards.topologyFor(CheckShards.java:57)
        at accord.coordinate.CheckShards.<init>(CheckShards.java:49)
        at accord.coordinate.MaybeRecover.<init>(MaybeRecover.java:50)
        at accord.coordinate.MaybeRecover.maybeRecover(MaybeRecover.java:60)
        at accord.local.Node.maybeRecover(Node.java:486)
        at accord.impl.SimpleProgressLog$Instance$State$CoordinateState.lambda$run$6(SimpleProgressLog.java:243)
        at accord.local.Node.withEpoch(Node.java:199)
        at accord.impl.SimpleProgressLog$Instance$State$CoordinateState.run(SimpleProgressLog.java:241)
        at accord.impl.SimpleProgressLog$Instance.lambda$run$3(SimpleProgressLog.java:829)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:322)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:309)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:207)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
        at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
        at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)
java.lang.IllegalStateException: null
        at accord.utils.Invariants.illegalState(Invariants.java:44)
        at accord.utils.Invariants.illegalState(Invariants.java:49)
        at accord.utils.Invariants.checkState(Invariants.java:91)
        at accord.impl.SimpleProgressLog$Instance.invalidated(SimpleProgressLog.java:745)
        at accord.local.Commands.commitInvalidate(Commands.java:354)
        at accord.messages.Commit$Invalidate.lambda$process$0(Commit.java:285)
        at accord.local.CommandStores$1.apply(CommandStores.java:333)
        at accord.local.CommandStores$1.apply(CommandStores.java:329)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:300)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:287)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:207)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
        at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
        at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)
java.lang.IllegalStateException: null
        at accord.utils.Invariants.illegalState(Invariants.java:44)
        at accord.utils.Invariants.illegalState(Invariants.java:49)
        at accord.utils.Invariants.checkState(Invariants.java:91)
        at accord.impl.SimpleProgressLog$Instance.invalidated(SimpleProgressLog.java:745)
        at accord.local.Commands.commitInvalidate(Commands.java:354)
        at accord.coordinate.Invalidate.lambda$commitInvalidate$2(Invalidate.java:271)
        at accord.local.CommandStores$1.apply(CommandStores.java:333)
        at accord.local.CommandStores$1.apply(CommandStores.java:329)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:300)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:287)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:207)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:96)
        at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61)
        at org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)
java.lang.IllegalArgumentException: Unsupported listener type: accord.messages.Defer
        at org.apache.cassandra.service.accord.serializers.ListenerSerializers$Kind.of(ListenerSerializers.java:47)
        at org.apache.cassandra.service.accord.serializers.ListenerSerializers$Kind.access$000(ListenerSerializers.java:35)
        at org.apache.cassandra.service.accord.serializers.ListenerSerializers$3.serializedSize(ListenerSerializers.java:134)
        at org.apache.cassandra.service.accord.serializers.ListenerSerializers$3.serializedSize(ListenerSerializers.java:95)
        at org.apache.cassandra.io.LocalVersionedSerializer.serializedSize(LocalVersionedSerializer.java:91)
        at org.apache.cassandra.service.accord.AccordKeyspace.serialize(AccordKeyspace.java:317)
        at org.apache.cassandra.service.accord.AccordKeyspace.lambda$getCommandMutation$5(AccordKeyspace.java:510)
        at org.apache.cassandra.service.accord.AccordKeyspace.addSetChanges(AccordKeyspace.java:440)
        at org.apache.cassandra.service.accord.AccordKeyspace.getCommandMutation(AccordKeyspace.java:510)
        at org.apache.cassandra.service.accord.async.AsyncWriter.lambda$assembleWrites$1(AsyncWriter.java:88)
        at java.base/java.util.HashMap.forEach(HashMap.java:1337)
        at org.apache.cassandra.service.accord.async.AsyncWriter.assembleWrites(AsyncWriter.java:85)
        at org.apache.cassandra.service.accord.async.AsyncWriter.maybeDispatchWrites(AsyncWriter.java:116)
        at org.apache.cassandra.service.accord.async.AsyncWriter.save(AsyncWriter.java:155)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:213)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)
        at accord.utils.async.AsyncCallbacks.lambda$null$0(AsyncCallbacks.java:31)
        at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)
java.lang.NullPointerException: null
        at org.apache.cassandra.service.accord.serializers.CommandSerializers$TimestampSerializer.serialize(CommandSerializers.java:81)
        at org.apache.cassandra.service.accord.serializers.CommandsForKeySerializer$AccordCFKLoader.saveForCFK(CommandsForKeySerializer.java:139)
        at org.apache.cassandra.service.accord.serializers.CommandsForKeySerializer$AccordCFKLoader.saveForCFK(CommandsForKeySerializer.java:79)
        at accord.impl.CommandsForKey$CommandTimeseries$Update.add(CommandsForKey.java:214)
        at accord.impl.SafeCommandsForKey.listenerUpdate(SafeCommandsForKey.java:116)
        at accord.impl.CommandsForKey$Listener.onChange(CommandsForKey.java:271)
        at accord.local.SafeCommandStore.lambda$notifyListeners$1(SafeCommandStore.java:115)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:322)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:309)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:207)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:247)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)
        at accord.utils.async.AsyncCallbacks.lambda$null$0(AsyncCallbacks.java:31)
        at org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:133)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
        at java.base/java.lang.Thread.run(Thread.java:829)

dcapwell and others added 2 commits February 22, 2023 09:55
…ative paths during development

patch by David Capwell; reviewed by Caleb Rackliffe, Michael Semb Wever for CASSANDRA-18204
patch by Jacek Lewandowski; reviewed by Caleb Rackliffe for CASSANDRA-18240
Copy link
Contributor

@dcapwell dcapwell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

small changes, but LGTM +1

default void checkNotInvalidated()
{
if (invalidated())
throw new IllegalStateException("Cannot access invalidated " + this);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixing the toString we get

ERROR [CommandStore[12]:1] 2023-03-02 18:00:33,137 AsyncOperation.java:274 - Operation AsyncOperation{SAVING}-0x7bd3b9c2 failed
java.lang.IllegalStateException: Cannot access invalidated AccordSafeCommand{invalidated=true, global=Node{LOADED, key=[1,1677808833023004,2,2130706433], references=1}@2de5a0d2, original=null, current=Command@1167976198{[1,1677808833023004,2,213
0706433]:PreAccepted}}
        at org.apache.cassandra.service.accord.AccordSafeState.checkNotInvalidated(AccordSafeState.java:74)
        at org.apache.cassandra.service.accord.AccordSafeCommand.original(AccordSafeCommand.java:94)
        at org.apache.cassandra.service.accord.AccordSafeCommand.original(AccordSafeCommand.java:29)
        at org.apache.cassandra.service.accord.AccordSafeState.hasUpdate(AccordSafeState.java:38)
        at org.apache.cassandra.service.accord.async.AsyncWriter.lambda$assembleWrites$1(AsyncWriter.java:85)
        at java.base/java.util.HashMap.forEach(HashMap.java:1337)
        at org.apache.cassandra.service.accord.async.AsyncWriter.assembleWrites(AsyncWriter.java:84)
        at org.apache.cassandra.service.accord.async.AsyncWriter.maybeDispatchWrites(AsyncWriter.java:116)
        at org.apache.cassandra.service.accord.async.AsyncWriter.save(AsyncWriter.java:155)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:235)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

java.lang.IllegalStateException: Cannot access invalidated AccordSafeCommand{invalidated=true, global=Node{LOADED, key=[1,1677809078513005,2,2130706433], references=3}@36b1ca9e, original=Command@1483650346{[1,1677809078513005,2,2130706433]:PreAc
cepted}, current=Command@1483650346{[1,1677809078513005,2,2130706433]:PreAccepted}}
        at org.apache.cassandra.service.accord.AccordSafeState.checkNotInvalidated(AccordSafeState.java:74)
        at org.apache.cassandra.service.accord.AccordSafeCommand.current(AccordSafeCommand.java:80)
        at org.apache.cassandra.service.accord.AccordSafeCommand.current(AccordSafeCommand.java:29)
        at org.apache.cassandra.service.accord.AccordSafeState.hasUpdate(AccordSafeState.java:38)
        at org.apache.cassandra.service.accord.async.AsyncWriter.lambda$assembleWrites$1(AsyncWriter.java:85)
        at java.base/java.util.HashMap.forEach(HashMap.java:1337)
        at org.apache.cassandra.service.accord.async.AsyncWriter.assembleWrites(AsyncWriter.java:84)
        at org.apache.cassandra.service.accord.async.AsyncWriter.maybeDispatchWrites(AsyncWriter.java:116)
        at org.apache.cassandra.service.accord.async.AsyncWriter.save(AsyncWriter.java:155)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:235)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)```

Copy link
Contributor

@dcapwell dcapwell Mar 3, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

org.apache.cassandra.service.accord.async.AsyncOperation#runInternal calls

commandStore.completeOperation(safeStore, context.commands, context.commandsForKeys);

which makes it unsafe for writer to have access to org.apache.cassandra.service.accord.async.AsyncOperation.Context#commands

So, for this to work we need to do completeOperation after the write, or make the writer context not have access; or have the read methods supported... which kinda invalidates... this logic?

Copy link
Contributor

@dcapwell dcapwell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

removing +1 due to issues found while running bin/cassandra -f

@dcapwell
Copy link
Contributor

dcapwell commented Mar 3, 2023

Errors seen while trying to run with load

ERROR [CommandStore[19]:1] 2023-03-02 18:21:17,400 JVMStabilityInspector.java:68 - Exception in thread Thread[CommandStore[19]:1,5,CommandStore[19]]
java.lang.ClassCastException: class accord.local.Command$NotWitnessed cannot be cast to class accord.local.Command$Committed (accord.local.Command$NotWitnessed and accord.local.Command$Committed are in unnamed module of loader 'app')
        at accord.local.Command.asCommitted(Command.java:448)
        at accord.local.Commands.updatePredecessorAndMaybeExecute(Commands.java:589)
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:675)
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:642)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:345)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:332)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:229)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)
ERROR [CommandStore[23]:1] 2023-03-02 18:21:17,778 AsyncOperation.java:274 - Operation AsyncOperation{RUNNING}-0x177ef150 failed
java.lang.IllegalStateException: null
        at accord.utils.Invariants.illegalState(Invariants.java:44)
        at accord.utils.Invariants.illegalState(Invariants.java:49)
        at accord.utils.Invariants.checkState(Invariants.java:91)
        at accord.impl.SimpleProgressLog$Instance.invalidated(SimpleProgressLog.java:756)
        at accord.local.Commands.commitInvalidate(Commands.java:355)
        at accord.coordinate.Invalidate.lambda$commitInvalidate$2(Invalidate.java:271)
        at accord.local.CommandStores$1.apply(CommandStores.java:326)
        at accord.local.CommandStores$1.apply(CommandStores.java:322)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:323)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForFunction.apply(AsyncOperation.java:310)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:229)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)

@dcapwell
Copy link
Contributor

dcapwell commented Mar 3, 2023

ERROR [CommandStore[18]:1] 2023-03-02 18:27:35,801 AsyncOperation.java:274 - Operation AsyncOperation{RUNNING}-0x679ceee failed
java.lang.NullPointerException: null
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:709)
        at accord.local.Commands$NotifyWaitingOn.accept(Commands.java:642)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:345)
        at org.apache.cassandra.service.accord.async.AsyncOperation$ForConsumer.apply(AsyncOperation.java:332)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:229)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)
        at org.apache.cassandra.service.accord.async.AsyncOperation.callback(AsyncOperation.java:158)

this next one is a known one

ERROR [CommandStore[8]:1] 2023-03-02 18:28:24,339 AsyncOperation.java:274 - Operation AsyncOperation{SAVING}-0x488a7cea failed
java.lang.IllegalArgumentException: Unhandled listener type: class accord.messages.Defer
        at org.apache.cassandra.service.accord.AccordObjectSizes.listener(AccordObjectSizes.java:257)
        at org.apache.cassandra.service.accord.AccordObjectSizes.command(AccordObjectSizes.java:310)
        at org.apache.cassandra.service.accord.AccordStateCache$Node.estimatedSizeOnHeap(AccordStateCache.java:100)
        at org.apache.cassandra.service.accord.AccordStateCache$Node.estimatedSizeOnHeapDelta(AccordStateCache.java:108)
        at org.apache.cassandra.service.accord.AccordStateCache.updateSize(AccordStateCache.java:239)
        at org.apache.cassandra.service.accord.AccordStateCache.access$1000(AccordStateCache.java:48)
        at org.apache.cassandra.service.accord.AccordStateCache$Instance.release(AccordStateCache.java:473)
        at java.base/java.util.HashMap$Values.forEach(HashMap.java:977)
        at org.apache.cassandra.service.accord.async.AsyncOperation$Context.releaseResources(AsyncOperation.java:62)
        at org.apache.cassandra.service.accord.async.AsyncOperation.runInternal(AsyncOperation.java:241)
        at org.apache.cassandra.service.accord.async.AsyncOperation.run(AsyncOperation.java:270)

@dcapwell
Copy link
Contributor

dcapwell commented Mar 3, 2023

updated SPL to show details when it fails, this is what I got

ERROR [CommandStore[31]:1] 2023-03-02 18:32:14,474 JVMStabilityInspector.java:68 - Exception in thread Thread[CommandStore[31]:1,5,CommandStore[31]]
java.lang.IllegalStateException: Shard = Unsure, state = {Done,Done}
        at accord.utils.Invariants.illegalState(Invariants.java:44)
        at accord.utils.Invariants.checkState(Invariants.java:133)
        at accord.impl.SimpleProgressLog$Instance.invalidated(SimpleProgressLog.java:756)

for code

Invariants.checkState(shard == Home || state == null || state.coordinateState == null, "Shard = %s, state = %s", shard, state);

this.commandStore = commandStore;
this.txnIds = txnIds;
this.keys = keys;
this.txnIds = Lists.newArrayList(txnIds);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we copy these to a new List? If we need a List (or a Set somewhere else) we should try to specify this in PreLoadContext imo.

this.txnIds = txnIds;
this.keys = keys;
this.txnIds = Lists.newArrayList(txnIds);
this.keys = Lists.newArrayList(keys);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we just accept an AbstractKeys<RoutingKey, ?> instead?

private final NamedMap<Object, Future<?>> writeFutures = new NamedMap<>("writeFutures");
private final NamedMap<Object, AsyncResult<Void>> saveResults = new NamedMap<>("saveResults");

private int linked = 0;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

eligibleForEviction?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

that's misleading, they're not neccesarily eligibleForEviction. unreferenced is probably clearer than linked though

Copy link
Contributor

@belliottsmith belliottsmith Mar 6, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Doesn’t unreferenced basically mean eligible for eviction (if last in queue)? Except for the condition that it’s loaded (that we can discuss in another thread)

either way, unreferenced is def an improvement

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Almost. To be eligible for eviction, there also needs to be no unfinished operations, it needs to be in the right state, etc.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we even be inserting into the list if we aren’t yet eligible for collection? Doesn’t that cause inefficiencies when actually evicting?

Copy link
Contributor

@belliottsmith belliottsmith Mar 7, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Once a minute is definitely not enough. You do not need much traffic to exhaust the cache in that time interval. Let's say you have 128MiB cache, 5k qps, you only 450 bytes in cache per query to exhaust the cache. Just 50 bytes per query wastes 10% of your cache space.

I'm strongly -1 on timed cache clearing. Cache management should dynamic based on actual demand and conditions. If we decide to accept the pointer chasing cost that's one thing, but a cache clearance frequency is another tunable we shouldn't have, and we have enough of those.

I don't mind punting on this, with a TODO (expected) to revisit, perhaps when we address persistence. But not with a periodic clearance, let's keep doing this on demand.

Copy link
Contributor

@belliottsmith belliottsmith Mar 7, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(There remain some other very simple possibilities, such as dequeuing the tail elements that we want to evict but can't, and place them in a separate queue for eviction as soon as they are ready, so that we don't repeatedly walk over them.)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok one minute was a bit of an exaggeration. I am in favor of accepting the pointer chasing cost until we can demonstrate it's a bottleneck though.

Copy link
Contributor

@belliottsmith belliottsmith Mar 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm happy to punt it as explicit technical debt, but don’t want to waive it away with a "prove it’s a problem" mentality. That shouldn’t be how we address concerns like this.

I think we probably agree that we would prefer our design not to have this property? and that we’re mostly willing to accept it because we have bigger fish to fry. In which case it’s probably a worse investment of our time to hash out the relevant standard of proof than it would be to address the issue, should it come to it.

But I agree it's not an optimal use of our time right now.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not trying to wave it away, or get into any Mexican standoffs :)

}

boolean isEmpty()
private boolean isUnlinked()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

isEligibleForEviction?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd prefer to keep as is. This is literally checking that the prev/next fields are null, and is used as a sanity check before updating the linked list

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unlinked felt ambiguous to me as a reader - the next and prev are potentially links, but so potentially are the references. Perhaps isInEvictionQueue?

enum State
{
INITIALIZED,
SUBMITTED,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unused? Would be good in general to comment these, at least the not 100% obvious ones like difference between SAVING and AWAITING_SAVE, LOADING and PREPARING.

*/
public class AccordLoadingState<K, V>
{
public enum LoadingState { NOT_FOUND, PENDING, LOADED, FAILED }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

NOT_FOUND -> UNINITIALIZED?

I spent a while this morning thinking this meant it wasn't found on disk, and trying to figure out how this differed from LOADED.

private final NamedMap<Object, AsyncResult<Void>> saveResults = new NamedMap<>("saveResults");

private int unreferenced = 0;
Node<?, ?> head;
Copy link
Contributor

@belliottsmith belliottsmith Mar 7, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We have a doubly-linked list, and idiomatically these have a dummy header (embedded in the owning class, i.e. so that AccordStateCache would extend Node or some super class). We have templates for this already around the codebase, e.g. IntrusiveLinkedList.

I'd be happy to convert if you've no concerns, or there's a reason I'm missing.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Node extends AccordLoadingState so it can't also extend some base node class. We also don't want the state cache inheriting the accord loading state stuff.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can invert that hierarchy quite easily can't we? The only class extending it is Node, so if we are doing it to separate concerns why not just have a LoadingNode extends Node or something?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Node also has a bunch of stuff we don’t want on the cache, and I don’t like the idea of AccordLoadingState extending a Node type. It could be done with interfaces, albeit more awkwardly, but I don’t really see the benefit, aside from slightly simpler push/pop methods?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why don't you like inverting the inheritance order?

It's not just that we get simpler push/pop methods: we can reuse the existing intrusive linked lists, so we are duplicating less functionality. It's also more idiomatic, so it's less surprising to a reader familiar with such lists.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you don't want to swap the inheritance order of course, we don't need to have AccordStateCache extend Node, we can just have a dummy member field. It's just very marginally less efficient/neat, and we also avoid the aforementioned code-reuse benefits.

So, I still think it would be better to inherit since we can. But we can still gain the benefits of a circular-linked list without doing so,

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why don't you like inverting the inheritance order?

Because Node is an implementation detail of the cache, and not the logical ancestor of accord state objects.

It's also more idiomatic, so it's less surprising to a reader familiar with such lists.

I’d disagree with that. Having explicit head/tail pointers on the list container is much more idiomatic than a container also being a dummy node. You could argue that this does put head/tail fields on the container, but it’s not nearly as straightforward or easy to identify. It does enable cleaner list manipulation methods, but is not the least surprising approach.

It also breaks the idiomatic linked list traversal pattern, and creates a special node you need to be aware of.

Copy link
Contributor

@belliottsmith belliottsmith Mar 8, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

🤷 I guess this is kind of like a blue/white dress thing. I thought it was pretty widely accepted that sentinel nodes and circular lists were the most common way of implementing doubly-linked lists. Certainly not singly linked lists, in which I would expect separate start/end pointers. But, no point arguing if we don't have the same ground truths there, we will get nowhere.

Because Node is an implementation detail of the cache, and not the logical ancestor of accord state objects.

I'm confused. The AccordLoadingState is only extended by Node, and is only used to expose the inner enum LoadingState AFAICT outside of the cache hierarchy. What accord state objects would we be the logical ancestor of?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm confused. The AccordLoadingState is only extended by Node, and is only used to expose the inner enum LoadingState AFAICT outside of the cache hierarchy. What accord state objects would we be the logical ancestor of?

Well none*, it's a conceptual and structural thing. It keeps things neater and more understandable if they can be kept separate

  • a test class does extend it, but it's not really consequential.

{
F r = futuresMap.get(key);
logger.trace("Evicting {} {} - {}", evict.state(), evict.key(), evict.isLoaded() ? evict.value() : null);
if (unlink) unlink(evict);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Might it be neater to just have unlink no-op if it's not in the queue?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wrote this to maintain assumption checks, if we believe it should be linked but it isn't, this is a bug we should fail on.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't this more brittle, since we could be linked and think we aren't, and do nothing?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Blake added a patch to verify this assumption in the else block.

since we could be linked and think we aren't,

This would be a bug, and not detecting that could cause a use-after-free issue (which leads to a NPE later on). There was a similar issue earlier on where the node was not in cache but it was in the linked list, this eventually NPE.

patch by David Capwell; reviewed by Alex Petrov, Caleb Rackliffe, Jacek Lewandowski for CASSANDRA-18299
node.isLoaded() &&
!hasActiveAsyncResult(saveResults, node.key());
Invariants.checkState(node.references == 0);
return node.state() == FAILED || !hasActiveAsyncResult(saveResults, node.key());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this change wasn't replicated to org.apache.cassandra.service.accord.AccordStateCache.Instance#checkCanEvict. This is only used in org.apache.cassandra.service.accord.async.AsyncOperationTest#assertCanEvict and was there to have a more human readable understanding of why we think we can not evict...

I am ok dropping org.apache.cassandra.service.accord.AccordStateCache.Instance#checkCanEvict as its just for tests and causes duplication of logic; but does make bugs harder to track down when tests do finally fail...


private static RuntimeException throwPreempted(TxnId txnId, Txn txn, ConsistencyLevel consistencyLevel)
{
throw txn.isWrite() ? new WritePreemptedException(WriteType.TRANSACTION, consistencyLevel, 0, 0, txnId.toString())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

when you rebase this will become a regression, can you use CAS? I document why in CASSANDRA-18299; TRANSACTION requires driver/protocol changes

@dcapwell
Copy link
Contributor

dcapwell commented Mar 8, 2023

I am mostly cool with all the changes since last approval. I still have 2 concerns but think it is ok to merge while we try to figure out what happened; while running perf tests I hit 2 issues: many transactions depended on a PreAccepted one that never made progress, and 2 transactions depended on NotWitnessed transactions (150 and 260) but this was a single node test only

I know burn tests have been running and no repo there yet, and once this patch rebases to pick up CASSANDRA-18299 I can rerun benchmarks to see if I can repo again...

Patch by Blake Eggleston; Reviewed by David Capwell & Benedict Elliott Smith for CASSANDRA-18004
Patch by Blake Eggleston; Reviewed by David Capwell and Benedict Elliott Smith for Cassandra-18192
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants