From 7b31886c7f912719b06021e44c05f9ccf5ce39d8 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Tue, 26 Nov 2024 15:26:54 +0100 Subject: [PATCH 01/10] Implement field saving/loading in AccordJournal --- .gitmodules | 4 +- modules/accord | 2 +- .../service/accord/AccordCommandStore.java | 8 +- .../service/accord/AccordJournal.java | 4 +- .../accord/AccordJournalValueSerializers.java | 45 ++++--- .../accord/AccordSafeCommandStore.java | 110 ++---------------- .../service/accord/AccordService.java | 2 +- .../serializers/CommandStoreSerializers.java | 7 +- .../test/accord/AccordBootstrapTest.java | 4 +- .../accord/AccordJournalCompactionTest.java | 4 +- .../CompactionAccordIteratorsTest.java | 6 +- .../service/accord/AccordTestUtils.java | 9 +- .../accord/SimulatedAccordCommandStore.java | 2 +- .../service/accord/SimulatedDepsTest.java | 4 +- .../accord/SimulatedMultiKeyAndRangeTest.java | 2 +- ...ulatedRandomKeysWithRangeConflictTest.java | 2 +- .../cassandra/utils/AccordGenerators.java | 4 +- 17 files changed, 69 insertions(+), 150 deletions(-) diff --git a/.gitmodules b/.gitmodules index 616dacf610a7..6c948af217bf 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord - url = https://github.com/apache/cassandra-accord.git - branch = trunk + url = https://github.com/ifesdjeen/cassandra-accord.git + branch = CASSANDRA-20114 diff --git a/modules/accord b/modules/accord index bd0761c567d1..ce83a567fce3 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit bd0761c567d153995a3db8da686ffdc940247200 +Subproject commit ce83a567fce34e6a20d645426cd6c416f045ae22 diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java index 0fbfea91c01c..588b729c20e9 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java @@ -461,10 +461,10 @@ public void registerTransitive(SafeCommandStore safeStore, RangeDeps rangeDeps) return; RedundantBefore redundantBefore = unsafeGetRedundantBefore(); - CommandStores.RangesForEpoch ranges = safeStore.ranges(); + CommandStores.RangesForEpoch ranges = safeStore.rangesForEpoch(); // used in places such as accord.local.CommandStore.fetchMajorityDeps // We find a set of dependencies for a range then update CommandsFor to know about them - Ranges allRanges = safeStore.ranges().all(); + Ranges allRanges = safeStore.rangesForEpoch().all(); Ranges coordinateRanges = Ranges.EMPTY; long coordinateEpoch = -1; try (ExclusiveCaches caches = lockCaches()) @@ -611,9 +611,9 @@ void loadSafeToRead(NavigableMap safeToRead) unsafeSetSafeToRead(safeToRead); } - void loadRangesForEpoch(CommandStores.RangesForEpoch.Snapshot rangesForEpoch) + void loadRangesForEpoch(CommandStores.RangesForEpoch rangesForEpoch) { if (rangesForEpoch != null) - unsafeSetRangesForEpoch(new CommandStores.RangesForEpoch(rangesForEpoch.epochs, rangesForEpoch.ranges, this)); + unsafeSetRangesForEpoch(rangesForEpoch); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index 331ad04ab3f7..6e2e57fc40d0 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -241,9 +241,9 @@ public NavigableMap loadSafeToRead(int store) } @Override - public CommandStores.RangesForEpoch.Snapshot loadRangesForEpoch(int store) + public CommandStores.RangesForEpoch loadRangesForEpoch(int store) { - IdentityAccumulator accumulator = readAll(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, store)); + IdentityAccumulator accumulator = readAll(new JournalKey(TxnId.NONE, JournalKey.Type.RANGES_FOR_EPOCH, store)); return accumulator.get(); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java index 54af375f17aa..ad8d2594dd4b 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java @@ -28,6 +28,7 @@ import accord.primitives.Ranges; import accord.primitives.Timestamp; import accord.primitives.TxnId; +import accord.utils.Invariants; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.net.MessagingService; @@ -280,44 +281,50 @@ public void deserialize(JournalKey key, IdentityAccumulator> + implements FlyweightSerializer> { - public IdentityAccumulator mergerFor(JournalKey key) + public IdentityAccumulator mergerFor(JournalKey key) { return new IdentityAccumulator<>(null); } @Override - public void serialize(JournalKey key, RangesForEpoch.Snapshot from, DataOutputPlus out, int userVersion) throws IOException + public void serialize(JournalKey key, RangesForEpoch from, DataOutputPlus out, int userVersion) throws IOException { - out.writeUnsignedVInt32(from.ranges.length); - for (Ranges ranges : from.ranges) - KeySerializers.ranges.serialize(ranges, out, messagingVersion); - - out.writeUnsignedVInt32(from.epochs.length); - for (long epoch : from.epochs) - out.writeLong(epoch); + out.writeUnsignedVInt32(from.size()); + from.forEach((epoch, ranges) -> { + try + { + out.writeLong(epoch); + KeySerializers.ranges.serialize(ranges, out, messagingVersion); + } + catch (Throwable t) + { + throw new IllegalStateException("Serialization error", t); + } + }); } @Override - public void reserialize(JournalKey key, IdentityAccumulator from, DataOutputPlus out, int userVersion) throws IOException + public void reserialize(JournalKey key, IdentityAccumulator from, DataOutputPlus out, int userVersion) throws IOException { serialize(key, from.get(), out, messagingVersion); } @Override - public void deserialize(JournalKey key, IdentityAccumulator into, DataInputPlus in, int userVersion) throws IOException + public void deserialize(JournalKey key, IdentityAccumulator into, DataInputPlus in, int userVersion) throws IOException { - Ranges[] ranges = new Ranges[in.readUnsignedVInt32()]; + int size = in.readUnsignedVInt32(); + Ranges[] ranges = new Ranges[size]; + long[] epochs = new long[size]; for (int i = 0; i < ranges.length; i++) + { ranges[i] = KeySerializers.ranges.deserialize(in, messagingVersion); - - long[] epochs = new long[in.readUnsignedVInt32()]; - for (int i = 0; i < epochs.length; i++) - epochs[i] = in.readLong(); // TODO: assert lengths equal? - - into.update(new RangesForEpoch.Snapshot(epochs, ranges)); + epochs[i] = in.readLong(); + } + Invariants.checkState(ranges.length == epochs.length); + into.update(new RangesForEpoch(epochs, ranges)); } } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java index c33279e925f9..edc473cf3570 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.Map; -import java.util.NavigableMap; import java.util.Set; import java.util.function.BiFunction; import javax.annotation.Nullable; @@ -34,12 +33,9 @@ import accord.impl.AbstractSafeCommandStore; import accord.impl.CommandsSummary; import accord.local.CommandStores; -import accord.local.CommandStores.RangesForEpoch; import accord.local.NodeCommandStoreService; -import accord.local.RedundantBefore; import accord.local.cfk.CommandsForKey; import accord.primitives.AbstractKeys; -import accord.primitives.Ranges; import accord.primitives.Routables; import accord.primitives.Timestamp; import accord.primitives.Txn; @@ -48,7 +44,6 @@ import accord.utils.Invariants; import org.apache.cassandra.service.accord.AccordCommandStore.ExclusiveCaches; -import static accord.api.Journal.*; import static accord.utils.Invariants.illegalState; public class AccordSafeCommandStore extends AbstractSafeCommandStore @@ -56,20 +51,26 @@ public class AccordSafeCommandStore extends AbstractSafeCommandStore task; private final @Nullable CommandsForRanges commandsForRanges; private final AccordCommandStore commandStore; - private RangesForEpoch ranges; - private FieldUpdates fieldUpdates; private AccordSafeCommandStore(AccordTask task, @Nullable CommandsForRanges commandsForRanges, AccordCommandStore commandStore) { - super(task.preLoadContext()); + super(task.preLoadContext(), commandStore); this.task = task; this.commandsForRanges = commandsForRanges; this.commandStore = commandStore; commandStore.updateRangesForEpoch(this); - if (this.ranges == null) - this.ranges = Invariants.nonNull(commandStore.unsafeRangesForEpoch()); + } + + @Override + public CommandStores.RangesForEpoch rangesForEpoch() + { + CommandStores.RangesForEpoch ranges = super.rangesForEpoch(); + if (ranges != null) + return ranges; + + return commandStore.unsafeGetRangesForEpoch(); } public static AccordSafeCommandStore create(AccordTask operation, @@ -196,12 +197,6 @@ public NodeCommandStoreService node() return commandStore.node(); } - @Override - public RangesForEpoch ranges() - { - return ranges; - } - private O mapReduce(Routables keysOrRanges, BiFunction map, O accumulate) { Invariants.checkState(context.keys().containsAll(keysOrRanges), "Attempted to access keysOrRanges outside of what was asked for; asked for %s, accessed %s", context.keys(), keysOrRanges); @@ -278,87 +273,4 @@ public String toString() { return "AccordSafeCommandStore(id=" + commandStore().id() + ")"; } - - @Override - public void upsertRedundantBefore(RedundantBefore addRedundantBefore) - { - // TODO (required): this is a temporary measure, see comment on AccordJournalValueSerializers; upsert instead - // when modifying, only modify together with AccordJournalValueSerializers - ensureFieldUpdates().newRedundantBefore = RedundantBefore.merge(redundantBefore(), addRedundantBefore); - } - - @Override - public void setBootstrapBeganAt(NavigableMap newBootstrapBeganAt) - { - ensureFieldUpdates().newBootstrapBeganAt = newBootstrapBeganAt; - } - - @Override - public void setSafeToRead(NavigableMap newSafeToRead) - { - ensureFieldUpdates().newSafeToRead = newSafeToRead; - } - - @Override - public void setRangesForEpoch(CommandStores.RangesForEpoch rangesForEpoch) - { - ensureFieldUpdates().newRangesForEpoch = rangesForEpoch.snapshot(); - ranges = rangesForEpoch; - } - - @Override - public NavigableMap bootstrapBeganAt() - { - if (fieldUpdates != null && fieldUpdates.newBootstrapBeganAt != null) - return fieldUpdates.newBootstrapBeganAt; - - return super.bootstrapBeganAt(); - } - - @Override - public NavigableMap safeToReadAt() - { - if (fieldUpdates != null && fieldUpdates.newSafeToRead != null) - return fieldUpdates.newSafeToRead; - - return super.safeToReadAt(); - } - - @Override - public RedundantBefore redundantBefore() - { - if (fieldUpdates != null && fieldUpdates.newRedundantBefore != null) - return fieldUpdates.newRedundantBefore; - - return super.redundantBefore(); - } - - private FieldUpdates ensureFieldUpdates() - { - if (fieldUpdates == null) fieldUpdates = new FieldUpdates(); - return fieldUpdates; - } - - public FieldUpdates fieldUpdates() - { - return fieldUpdates; - } - - public void postExecute() - { - if (fieldUpdates == null) - return; - - if (fieldUpdates.newRedundantBefore != null) - super.unsafeSetRedundantBefore(fieldUpdates.newRedundantBefore); - - if (fieldUpdates.newBootstrapBeganAt != null) - super.setBootstrapBeganAt(fieldUpdates.newBootstrapBeganAt); - - if (fieldUpdates.newSafeToRead != null) - super.setSafeToRead(fieldUpdates.newSafeToRead); - - if (fieldUpdates.newRangesForEpoch != null) - super.setRangesForEpoch(ranges); - } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 06fb49cda9a3..3755348c66a2 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -1253,7 +1253,7 @@ public CompactionInfo getCompactionInfo() synchronized (redundantBefores) { redundantBefores.put(safeStore.commandStore().id(), safeStore.redundantBefore()); - ranges.put(safeStore.commandStore().id(), safeStore.ranges()); + ranges.put(safeStore.commandStore().id(), safeStore.rangesForEpoch()); durableBefores.put(safeStore.commandStore().id(), safeStore.durableBefore()); } })); diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java index 4e4053103314..10433feaf471 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java @@ -38,7 +38,6 @@ import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.utils.CollectionSerializers; import org.apache.cassandra.utils.NullableSerializer; @@ -138,7 +137,7 @@ public long serializedSize(DurableBefore.Entry t, int version) @Override public void serialize(RedundantBefore.Entry t, DataOutputPlus out, int version) throws IOException { - TokenRange.serializer.serialize((TokenRange) t.range, out, version); + KeySerializers.range.serialize(t.range, out, version); Invariants.checkState(t.startOwnershipEpoch <= t.endOwnershipEpoch); out.writeUnsignedVInt(t.startOwnershipEpoch); if (t.endOwnershipEpoch == Long.MAX_VALUE) out.writeUnsignedVInt(0L); @@ -156,7 +155,7 @@ public void serialize(RedundantBefore.Entry t, DataOutputPlus out, int version) @Override public RedundantBefore.Entry deserialize(DataInputPlus in, int version) throws IOException { - Range range = TokenRange.serializer.deserialize(in, version); + Range range = KeySerializers.range.deserialize(in, version); long startEpoch = in.readUnsignedVInt(); long endEpoch = in.readUnsignedVInt(); if (endEpoch == 0) endEpoch = Long.MAX_VALUE; @@ -175,7 +174,7 @@ public RedundantBefore.Entry deserialize(DataInputPlus in, int version) throws I @Override public long serializedSize(RedundantBefore.Entry t, int version) { - long size = TokenRange.serializer.serializedSize((TokenRange) t.range, version); + long size = KeySerializers.range.serializedSize(t.range, version); size += TypeSizes.sizeofUnsignedVInt(t.startOwnershipEpoch); size += TypeSizes.sizeofUnsignedVInt(t.endOwnershipEpoch == Long.MAX_VALUE ? 0 : 1 + t.endOwnershipEpoch - t.startOwnershipEpoch); size += CommandSerializers.txnId.serializedSize(t.locallyWitnessedOrInvalidatedBefore, version); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java index 57e452009dd8..82371a84bb57 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java @@ -320,7 +320,7 @@ public void bootstrapTest() throws Throwable Assert.assertEquals(key, row.getInt("v")); awaitUninterruptiblyAndRethrow(service().node().commandStores().forEach(safeStore -> { - if (safeStore.ranges().currentRanges().contains(partitionKey)) + if (safeStore.rangesForEpoch().currentRanges().contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; Assert.assertFalse(ss.bootstrapBeganAt().isEmpty()); @@ -465,7 +465,7 @@ public void moveTest() throws Throwable awaitUninterruptiblyAndRethrow(service().node().commandStores().forEach(PreLoadContext.contextFor(partitionKey.toUnseekable()), partitionKey.toUnseekable(), moveMax, moveMax, safeStore -> { - if (!safeStore.ranges().allAt(preMove).contains(partitionKey)) + if (!safeStore.rangesForEpoch().allAt(preMove).contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; Assert.assertFalse(ss.bootstrapBeganAt().isEmpty()); diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java index 1a790eece5c5..647ba1c8142b 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalCompactionTest.java @@ -90,12 +90,12 @@ public void segmentMergeTest() throws InterruptedException DurableBeforeAccumulator durableBeforeAccumulator = new DurableBeforeAccumulator(); NavigableMap safeToReadAtAccumulator = ImmutableSortedMap.of(Timestamp.NONE, Ranges.EMPTY); NavigableMap bootstrapBeganAtAccumulator = ImmutableSortedMap.of(TxnId.NONE, Ranges.EMPTY); - RangesForEpoch.Snapshot rangesForEpochAccumulator = null; + RangesForEpoch rangesForEpochAccumulator = null; Gen redundantBeforeGen = AccordGenerators.redundantBefore(DatabaseDescriptor.getPartitioner()); Gen durableBeforeGen = AccordGenerators.durableBeforeGen(DatabaseDescriptor.getPartitioner()); Gen> safeToReadGen = AccordGenerators.safeToReadGen(DatabaseDescriptor.getPartitioner()); - Gen rangesForEpochGen = AccordGenerators.rangesForEpoch(DatabaseDescriptor.getPartitioner()); + Gen rangesForEpochGen = AccordGenerators.rangesForEpoch(DatabaseDescriptor.getPartitioner()); Gen rangeGen = AccordGenerators.range(DatabaseDescriptor.getPartitioner()); Gen historicalTransactionsGen = depsGen(); diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionAccordIteratorsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionAccordIteratorsTest.java index fbcfee12f385..b6ecc521406a 100644 --- a/test/unit/org/apache/cassandra/db/compaction/CompactionAccordIteratorsTest.java +++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAccordIteratorsTest.java @@ -440,7 +440,7 @@ private static IAccordService mockAccordService(CommandStore commandStore, Redun if (durableBefore != null) durableBefores.put(commandStore.id(), durableBefore); Int2ObjectHashMap rangesForEpochs = new Int2ObjectHashMap<>(); - rangesForEpochs.put(commandStore.id(), commandStore.unsafeRangesForEpoch()); + rangesForEpochs.put(commandStore.id(), commandStore.unsafeGetRangesForEpoch()); when(mockAccordService.getCompactionInfo()).thenReturn(new IAccordService.CompactionInfo(redundantBefores, rangesForEpochs, durableBefores)); return mockAccordService; } @@ -491,8 +491,8 @@ private void testWithCommandStoreInternal(TestWithCommandStore test, boolean add { Txn txn = txnId.kind().isWrite() ? writeTxn : readTxn; PartialDeps partialDeps = Deps.NONE.intersecting(AccordTestUtils.fullRange(txn)); - PartialTxn partialTxn = txn.slice(commandStore.unsafeRangesForEpoch().currentRanges(), true); - Route partialRoute = route.slice(commandStore.unsafeRangesForEpoch().currentRanges()); + PartialTxn partialTxn = txn.slice(commandStore.unsafeGetRangesForEpoch().currentRanges(), true); + Route partialRoute = route.slice(commandStore.unsafeGetRangesForEpoch().currentRanges()); getUninterruptibly(commandStore.execute(contextFor(txnId, route, SYNC), safe -> { CheckedCommands.preaccept(safe, txnId, partialTxn, route, appendDiffToKeyspace(commandStore)); }).beginAsResult()); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java index c3ac81e1a90e..7e10e8ed5c56 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java @@ -350,12 +350,13 @@ public SingleEpochRanges(Ranges ranges) this.ranges = ranges; } - private void set(CommandStore store) + private void set() { - add(1, new CommandStores.RangesForEpoch(1, ranges, store), ranges); + add(1, new CommandStores.RangesForEpoch(1, ranges), ranges); } } + // TODO: why unused? public static InMemoryCommandStore.Synchronized createInMemoryCommandStore(LongSupplier now, String keyspace, String table) { TableMetadata metadata = Schema.instance.getTableMetadata(keyspace, table); @@ -378,7 +379,7 @@ public static InMemoryCommandStore.Synchronized createInMemoryCommandStore(LongS SingleEpochRanges holder = new SingleEpochRanges(Ranges.of(range)); InMemoryCommandStore.Synchronized result = new InMemoryCommandStore.Synchronized(0, time, new AccordAgent(), null, null, cs -> null, holder); - holder.set(result); + holder.set(); return result; } @@ -421,7 +422,7 @@ public static AccordCommandStore createAccordCommandStore( cs -> new NoOpProgressLog(), cs -> new DefaultLocalListeners(new NoOpRemoteListeners(), new NoOpNotifySink()), holder, journal, executor); - holder.set(result); + holder.set(); result.unsafeUpdateRangesForEpoch(); return result; } diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java index 5c5285f1d035..fe82e3b009f6 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java @@ -220,7 +220,7 @@ public void onUncaughtException(Throwable t) }); this.topology = AccordTopology.createAccordTopology(ClusterMetadata.current()); this.topologies = new Topologies.Single(SizeOfIntersectionSorter.SUPPLIER, topology); - var rangesForEpoch = new CommandStores.RangesForEpoch(topology.epoch(), topology.ranges(), commandStore); + CommandStores.RangesForEpoch rangesForEpoch = new CommandStores.RangesForEpoch(topology.epoch(), topology.ranges()); updateHolder.add(topology.epoch(), rangesForEpoch, topology.ranges()); updateHolder.updateGlobal(topology.ranges()); commandStore.unsafeUpdateRangesForEpoch(); diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedDepsTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedDepsTest.java index 770af5b314e7..5de604aff944 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedDepsTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedDepsTest.java @@ -186,7 +186,7 @@ public void simpleRangeConflicts() FullRangeRoute rangeRoute = ranges.toRoute(pk.toUnseekable()); Txn rangeTxn = createTxn(Txn.Kind.ExclusiveSyncPoint, ranges); - DepsModel model = new DepsModel(instance.commandStore.unsafeRangesForEpoch().currentRanges()); + DepsModel model = new DepsModel(instance.commandStore.unsafeGetRangesForEpoch().currentRanges()); for (int i = 0; i < numSamples; i++) { instance.maybeCacheEvict(keyRoute, ranges); @@ -258,7 +258,7 @@ public void overlappingRangeConflicts() Range left = tokenRange(tbl.id, token - 10, token + 5); Range right = tokenRange(tbl.id, token - 5, token + 10); - DepsModel model = new DepsModel(instance.commandStore.unsafeRangesForEpoch().currentRanges()); + DepsModel model = new DepsModel(instance.commandStore.unsafeGetRangesForEpoch().currentRanges()); for (int i = 0; i < numSamples; i++) { Ranges partialRange = Ranges.of(rs.nextBoolean() ? left : right); diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedMultiKeyAndRangeTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedMultiKeyAndRangeTest.java index cf09d8eab8a0..485f7d98b949 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedMultiKeyAndRangeTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedMultiKeyAndRangeTest.java @@ -71,7 +71,7 @@ public void test() Gen.IntGen keyCountGen = keyDistribution.next(rs); Gen.IntGen rangeCountGen = rangeDistribution.next(rs); - DepsModel model = new DepsModel(instance.commandStore.unsafeRangesForEpoch().currentRanges()); + DepsModel model = new DepsModel(instance.commandStore.unsafeGetRangesForEpoch().currentRanges()); for (int i = 0; i < numSamples; i++) { diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedRandomKeysWithRangeConflictTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedRandomKeysWithRangeConflictTest.java index 68a2b3cef327..d91b0fe013bc 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedRandomKeysWithRangeConflictTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedRandomKeysWithRangeConflictTest.java @@ -88,7 +88,7 @@ public State(RandomSource rs) AccordKeyspace.unsafeClear(); this.instance = new SimulatedAccordCommandStore(rs); this.instance.commandStore.executor().cacheUnsafe().setShrinkingOn(false); - this.model = new DepsModel(instance.commandStore.unsafeRangesForEpoch().currentRanges()); + this.model = new DepsModel(instance.commandStore.unsafeGetRangesForEpoch().currentRanges()); } @Override diff --git a/test/unit/org/apache/cassandra/utils/AccordGenerators.java b/test/unit/org/apache/cassandra/utils/AccordGenerators.java index e4e08ac180a9..93ad3b923224 100644 --- a/test/unit/org/apache/cassandra/utils/AccordGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AccordGenerators.java @@ -517,7 +517,7 @@ public static Gen> safeToReadGen(IPartitioner pa }; } - public static Gen rangesForEpoch(IPartitioner partitioner) + public static Gen rangesForEpoch(IPartitioner partitioner) { Gen rangesGen = ranges(partitioner); @@ -529,7 +529,7 @@ public static Gen rangesForEpoch(IPartitioner partition Ranges[] ranges = new Ranges[size]; for (int i = 0; i < size; i++) ranges[i] = rangesGen.next(rs); - return new RangesForEpoch.Snapshot(epochs, ranges); + return new RangesForEpoch(epochs, ranges); }; } From 98de52cfa6744262ce2eb46ac0bfe5e99d9cdaaa Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Tue, 26 Nov 2024 15:26:54 +0100 Subject: [PATCH 02/10] Implement field saving/loading in AccordJournal --- .../apache/cassandra/service/accord/AccordSafeCommandStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java index edc473cf3570..d5c03c8e0a04 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java @@ -32,7 +32,6 @@ import accord.api.RoutingKey; import accord.impl.AbstractSafeCommandStore; import accord.impl.CommandsSummary; -import accord.local.CommandStores; import accord.local.NodeCommandStoreService; import accord.local.cfk.CommandsForKey; import accord.primitives.AbstractKeys; From 1ab6e7fd1040c9492d62bd79bdf5b6a0ecd26912 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Mon, 2 Dec 2024 20:15:21 +0100 Subject: [PATCH 03/10] Migrate in memory journal to CommandChange logic shared with AccordJournal Patch by Alex Petrov; reviewed by Benedict Elliott Smith for CASSANDRA-20115 --- .gitmodules | 2 +- modules/accord | 2 +- .../cassandra/service/accord/SavedCommand.java | 13 ++++++++----- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/.gitmodules b/.gitmodules index 6c948af217bf..1500851ee94c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord url = https://github.com/ifesdjeen/cassandra-accord.git - branch = CASSANDRA-20114 + branch = CASSANDRA-20115 diff --git a/modules/accord b/modules/accord index ce83a567fce3..bbd95649fba0 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit ce83a567fce34e6a20d645426cd6c416f045ae22 +Subproject commit bbd95649fba007771bd9b42e0a8370258c61dfff diff --git a/src/java/org/apache/cassandra/service/accord/SavedCommand.java b/src/java/org/apache/cassandra/service/accord/SavedCommand.java index d00cde113f9d..dfa872c4fe6c 100644 --- a/src/java/org/apache/cassandra/service/accord/SavedCommand.java +++ b/src/java/org/apache/cassandra/service/accord/SavedCommand.java @@ -398,7 +398,6 @@ public static class Builder PartialTxn partialTxn; PartialDeps partialDeps; - byte[] waitingOnBytes; SavedCommand.WaitingOnProvider waitingOn; Writes writes; Result result; @@ -510,7 +509,6 @@ public void clear() partialTxn = null; partialDeps = null; - waitingOnBytes = null; waitingOn = null; writes = null; result = null; @@ -703,8 +701,12 @@ public void serialize(DataOutputPlus out, int userVersion) throws IOException DepsSerializers.partialDeps.serialize(partialDeps(), out, userVersion); break; case WAITING_ON: - out.writeInt(waitingOnBytes.length); - out.write(waitingOnBytes); + Command.WaitingOn waitingOn = waitingOn().provide(txnId, partialDeps); + long size = WaitingOnSerializer.serializedSize(txnId, waitingOn); + ByteBuffer serialized = WaitingOnSerializer.serialize(txnId, waitingOn); + Invariants.checkState(serialized.remaining() == size); + out.writeInt((int) size); + out.write(serialized); break; case WRITES: CommandSerializers.writes.serialize(writes(), out, userVersion); @@ -789,7 +791,8 @@ private void deserialize(Fields field, DataInputPlus in, int userVersion) throws break; case WAITING_ON: int size = in.readInt(); - waitingOnBytes = new byte[size]; + + byte[] waitingOnBytes = new byte[size]; in.readFully(waitingOnBytes); ByteBuffer buffer = ByteBuffer.wrap(waitingOnBytes); waitingOn = (localTxnId, deps) -> { From 643a5c7496ff81bea47a19f2ad5acf07dec069e7 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Tue, 3 Dec 2024 17:21:35 +0100 Subject: [PATCH 04/10] Integrate --- modules/accord | 2 +- .../db/compaction/CompactionIterator.java | 4 +- .../cassandra/service/accord/AccordCache.java | 4 +- .../service/accord/AccordCommandStore.java | 10 +- .../service/accord/AccordJournal.java | 442 ++++++-- .../accord/AccordJournalValueSerializers.java | 14 +- .../accord/AccordSafeCommandStore.java | 1 + .../service/accord/CommandsForRanges.java | 4 +- .../cassandra/service/accord/IJournal.java | 38 - .../service/accord/SavedCommand.java | 983 ------------------ .../test/accord/AccordLoadTest.java | 4 - .../service/accord/AccordJournalBurnTest.java | 4 +- .../accord/AccordJournalOrderTest.java | 2 +- ...ommandTest.java => CommandChangeTest.java} | 22 +- .../accord/SimulatedAccordCommandStore.java | 21 +- 15 files changed, 417 insertions(+), 1138 deletions(-) delete mode 100644 src/java/org/apache/cassandra/service/accord/IJournal.java delete mode 100644 src/java/org/apache/cassandra/service/accord/SavedCommand.java rename test/unit/org/apache/cassandra/service/accord/{SavedCommandTest.java => CommandChangeTest.java} (88%) diff --git a/modules/accord b/modules/accord index bbd95649fba0..16d6cbe520e9 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit bbd95649fba007771bd9b42e0a8370258c61dfff +Subproject commit 16d6cbe520e9ffe0c3226fed178a819e3a97555d diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java index 4feefac642cd..6a26d795ef34 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java @@ -93,6 +93,7 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.accord.AccordJournal; import org.apache.cassandra.service.accord.AccordJournalValueSerializers; import org.apache.cassandra.service.accord.AccordJournalValueSerializers.FlyweightSerializer; import org.apache.cassandra.service.accord.AccordKeyspace; @@ -103,7 +104,6 @@ import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.JournalKey; -import org.apache.cassandra.service.accord.SavedCommand; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.AccordRoutingKey.TokenKey; import org.apache.cassandra.service.paxos.PaxosRepairHistory; @@ -1099,7 +1099,7 @@ protected UnfilteredRowIterator applyToPartition(UnfilteredRowIterator partition return newVersion.build().unfilteredIterator(); } - SavedCommand.Builder commandBuilder = (SavedCommand.Builder) builder; + AccordJournal.Builder commandBuilder = (AccordJournal.Builder) builder; if (commandBuilder.isEmpty()) { Invariants.checkState(rows.isEmpty()); diff --git a/src/java/org/apache/cassandra/service/accord/AccordCache.java b/src/java/org/apache/cassandra/service/accord/AccordCache.java index 614054c03161..7fdb3c35a615 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCache.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCache.java @@ -1211,7 +1211,7 @@ public Object fullShrink(TxnId txnId, Command value) try { - return SavedCommand.asSerializedDiff(null, value, current_version); + return AccordJournal.asSerializedChange(null, value, current_version); } catch (IOException e) { @@ -1223,7 +1223,7 @@ public Object fullShrink(TxnId txnId, Command value) @Override public @Nullable Command inflate(TxnId key, Object serialized) { - SavedCommand.Builder builder = new SavedCommand.Builder(key); + AccordJournal.Builder builder = new AccordJournal.Builder(key); ByteBuffer buffer = (ByteBuffer) serialized; buffer.mark(); try (DataInputBuffer buf = new DataInputBuffer(buffer, false)) diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java index 588b729c20e9..c9df8b49b9cf 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java @@ -35,6 +35,7 @@ import accord.api.Agent; import accord.api.DataStore; +import accord.api.Journal; import accord.api.LocalListeners; import accord.api.ProgressLog; import accord.api.RoutingKey; @@ -64,7 +65,6 @@ import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; import org.apache.cassandra.db.Mutation; -import org.apache.cassandra.service.accord.SavedCommand.MinimalCommand; import org.apache.cassandra.service.accord.api.AccordRoutingKey.TokenKey; import org.apache.cassandra.service.accord.txn.TxnRead; import org.apache.cassandra.utils.Clock; @@ -72,12 +72,12 @@ import static accord.api.Journal.CommandUpdate; import static accord.api.Journal.FieldUpdates; +import static accord.api.Journal.Load.MINIMAL; import static accord.api.Journal.Loader; import static accord.api.Journal.OnDone; import static accord.local.KeyHistory.SYNC; import static accord.primitives.Status.Committed; import static accord.utils.Invariants.checkState; -import static org.apache.cassandra.service.accord.SavedCommand.Load.MINIMAL; public class AccordCommandStore extends CommandStore { @@ -155,7 +155,7 @@ public void close() } public final String loggingId; - private final IJournal journal; + private final Journal journal; private final AccordExecutor executor; private final Executor taskExecutor; private final ExclusiveCaches caches; @@ -174,7 +174,7 @@ public AccordCommandStore(int id, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenerFactory, EpochUpdateHolder epochUpdateHolder, - IJournal journal, + Journal journal, AccordExecutor executor) { super(id, node, agent, dataStore, progressLogFactory, listenerFactory, epochUpdateHolder); @@ -523,7 +523,7 @@ public static Command prepareToCache(Command command) return command; } - public MinimalCommand loadMinimal(TxnId txnId) + public Command.Minimal loadMinimal(TxnId txnId) { return journal.loadMinimal(id, txnId, MINIMAL, unsafeGetRedundantBefore(), durableBefore()); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index 6e2e57fc40d0..dcec5d6b1a1e 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -18,19 +18,20 @@ package org.apache.cassandra.service.accord; import java.io.IOException; -import java.util.ArrayList; +import java.nio.ByteBuffer; import java.util.Collections; -import java.util.List; import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.impl.CommandChange; import accord.impl.ErasedSafeCommand; import accord.local.Cleanup; import accord.local.Command; @@ -54,6 +55,7 @@ import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.io.util.DataInputBuffer; import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputBuffer; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.io.util.File; import org.apache.cassandra.journal.Compactor; @@ -65,14 +67,30 @@ import org.apache.cassandra.service.accord.AccordJournalValueSerializers.IdentityAccumulator; import org.apache.cassandra.service.accord.JournalKey.JournalKeySupport; import org.apache.cassandra.service.accord.api.AccordAgent; +import org.apache.cassandra.service.accord.serializers.CommandSerializers; +import org.apache.cassandra.service.accord.serializers.DepsSerializers; +import org.apache.cassandra.service.accord.serializers.ResultSerializers; +import org.apache.cassandra.service.accord.serializers.WaitingOnSerializer; import org.apache.cassandra.utils.ExecutorUtils; +import org.apache.cassandra.utils.Throwables; import org.apache.cassandra.utils.concurrent.AsyncPromise; +import static accord.impl.CommandChange.anyFieldChanged; +import static accord.impl.CommandChange.getFieldChanged; +import static accord.impl.CommandChange.getFieldIsNull; +import static accord.impl.CommandChange.getFlags; +import static accord.impl.CommandChange.getWaitingOn; +import static accord.impl.CommandChange.nextSetField; +import static accord.impl.CommandChange.setFieldChanged; +import static accord.impl.CommandChange.setFieldIsNull; +import static accord.impl.CommandChange.toIterableSetFields; +import static accord.impl.CommandChange.unsetIterableFields; +import static accord.impl.CommandChange.validateFlags; import static accord.primitives.SaveStatus.ErasedOrVestigial; import static accord.primitives.Status.Truncated; import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.DurableBeforeAccumulator; -public class AccordJournal implements IJournal, Shutdownable +public class AccordJournal implements accord.api.Journal, Shutdownable { static { @@ -188,7 +206,7 @@ public boolean awaitTermination(long timeout, TimeUnit units) throws Interrupted @Override public Command loadCommand(int commandStoreId, TxnId txnId, RedundantBefore redundantBefore, DurableBefore durableBefore) { - SavedCommand.Builder builder = loadDiffs(commandStoreId, txnId); + Builder builder = load(commandStoreId, txnId); Cleanup cleanup = builder.shouldCleanup(agent, redundantBefore, durableBefore); switch (cleanup) { @@ -201,10 +219,10 @@ public Command loadCommand(int commandStoreId, TxnId txnId, RedundantBefore redu } @Override - public SavedCommand.MinimalCommand loadMinimal(int commandStoreId, TxnId txnId, SavedCommand.Load load, RedundantBefore redundantBefore, DurableBefore durableBefore) + public Command.Minimal loadMinimal(int commandStoreId, TxnId txnId, Load load, RedundantBefore redundantBefore, DurableBefore durableBefore) { - SavedCommand.Builder builder = loadDiffs(commandStoreId, txnId, load); - if (!builder.nextCalled) + Builder builder = loadDiffs(commandStoreId, txnId, load); + if (builder.isEmpty()) return null; Cleanup cleanup = builder.shouldCleanup(node.agent(), redundantBefore, durableBefore); @@ -215,11 +233,11 @@ public SavedCommand.MinimalCommand loadMinimal(int commandStoreId, TxnId txnId, case ERASE: return null; } - Invariants.checkState(builder.saveStatus != null, "No saveSatus loaded, but next was called and cleanup was not: %s", builder); + Invariants.checkState(builder.saveStatus() != null, "No saveSatus loaded, but next was called and cleanup was not: %s", builder); return builder.asMinimal(); } - @VisibleForTesting + @Override public RedundantBefore loadRedundantBefore(int store) { IdentityAccumulator accumulator = readAll(new JournalKey(TxnId.NONE, JournalKey.Type.REDUNDANT_BEFORE, store)); @@ -250,7 +268,7 @@ public CommandStores.RangesForEpoch loadRangesForEpoch(int store) @Override public void saveCommand(int store, CommandUpdate update, Runnable onFlush) { - SavedCommand.Writer diff = SavedCommand.diff(update.before, update.after); + Writer diff = Writer.make(update.before, update.after); if (diff == null || status == Status.REPLAY) { if (onFlush != null) @@ -315,18 +333,18 @@ public void saveStoreState(int store, FieldUpdates fieldUpdates, Runnable onFlus onFlush.run(); } - @VisibleForTesting - public SavedCommand.Builder loadDiffs(int commandStoreId, TxnId txnId, SavedCommand.Load load) + private Builder loadDiffs(int commandStoreId, TxnId txnId, Load load) { JournalKey key = new JournalKey(txnId, JournalKey.Type.COMMAND_DIFF, commandStoreId); - SavedCommand.Builder builder = new SavedCommand.Builder(txnId, load); + Builder builder = new Builder(txnId, load); journalTable.readAll(key, builder::deserializeNext); return builder; } - public SavedCommand.Builder loadDiffs(int commandStoreId, TxnId txnId) + @VisibleForTesting + public Builder load(int commandStoreId, TxnId txnId) { - return loadDiffs(commandStoreId, txnId, SavedCommand.Load.ALL); + return loadDiffs(commandStoreId, txnId, Load.ALL); } private BUILDER readAll(JournalKey key) @@ -353,15 +371,15 @@ public void closeCurrentSegmentForTestingIfNonEmpty() public void sanityCheck(int commandStoreId, Command orig) { - SavedCommand.Builder diffs = loadDiffs(commandStoreId, orig.txnId()); - diffs.forceResult(orig.result()); + Builder builder = load(commandStoreId, orig.txnId()); + builder.forceResult(orig.result()); // We can only use strict equality if we supply result. - Command reconstructed = diffs.construct(); + Command reconstructed = builder.construct(); Invariants.checkState(orig.equals(reconstructed), '\n' + "Original: %s\n" + "Reconstructed: %s\n" + - "Diffs: %s", orig, reconstructed, diffs); + "Diffs: %s", orig, reconstructed, builder); } @VisibleForTesting @@ -391,7 +409,7 @@ public void replay(CommandStores commandStores) try (AccordJournalTable.KeyOrderIterator iter = journalTable.readAll()) { JournalKey key; - SavedCommand.Builder builder = new SavedCommand.Builder(); + Builder builder = new Builder(); while ((key = iter.key()) != null) { @@ -417,7 +435,7 @@ public void replay(CommandStores commandStores) } }); - if (builder.nextCalled) + if (!builder.isEmpty()) { Command command = builder.construct(); Invariants.checkState(command.saveStatus() != SaveStatus.Uninitialised, @@ -454,65 +472,361 @@ public void failure(Throwable t) return future; } - // TODO: this is here temporarily; for debugging purposes + public static @Nullable ByteBuffer asSerializedChange(Command before, Command after, int userVersion) throws IOException + { + try (DataOutputBuffer out = new DataOutputBuffer()) + { + Writer writer = Writer.make(before, after); + if (writer == null) + return null; + + writer.write(out, userVersion); + return out.asNewBuffer(); + } + } + @VisibleForTesting - public void checkAllCommands() + public void unsafeSetStarted() { - try (AccordJournalTable.KeyOrderIterator iter = journalTable.readAll()) + status = Status.STARTED; + } + + public static class Writer implements Journal.Writer + { + private final Command after; + private final int flags; + + private Writer(Command after, int flags) { - IAccordService.CompactionInfo compactionInfo = AccordService.instance().getCompactionInfo(); - JournalKey key; - SavedCommand.Builder builder = new SavedCommand.Builder(); - while ((key = iter.key()) != null) + this.after = after; + this.flags = flags; + } + + public static Writer make(Command before, Command after) + { + if (before == after + || after == null + || after.saveStatus() == SaveStatus.Uninitialised) + return null; + + int flags = validateFlags(getFlags(before, after)); + if (!anyFieldChanged(flags)) + return null; + + return new Writer(after, flags); + } + + @Override + public void write(DataOutputPlus out, int userVersion) throws IOException + { + Invariants.checkState(flags != 0); + out.writeInt(flags); + + int iterable = toIterableSetFields(flags); + while (iterable != 0) { - builder.reset(key.id); - if (key.type != JournalKey.Type.COMMAND_DIFF) + CommandChange.Fields field = nextSetField(iterable); + if (getFieldIsNull(field, flags)) { - // TODO (required): add "skip" for the key to avoid getting stuck - iter.readAllForKey(key, (segment, position, key1, buffer, hosts, userVersion) -> {}); + iterable = unsetIterableFields(field, iterable); continue; } - JournalKey finalKey = key; - List pointers = new ArrayList<>(); - try + switch (field) + { + case EXECUTE_AT: + CommandSerializers.timestamp.serialize(after.executeAt(), out, userVersion); + break; + case EXECUTES_AT_LEAST: + CommandSerializers.timestamp.serialize(after.executesAtLeast(), out, userVersion); + break; + case SAVE_STATUS: + out.writeShort(after.saveStatus().ordinal()); + break; + case DURABILITY: + out.writeByte(after.durability().ordinal()); + break; + case ACCEPTED: + CommandSerializers.ballot.serialize(after.acceptedOrCommitted(), out, userVersion); + break; + case PROMISED: + CommandSerializers.ballot.serialize(after.promised(), out, userVersion); + break; + case PARTICIPANTS: + CommandSerializers.participants.serialize(after.participants(), out, userVersion); + break; + case PARTIAL_TXN: + CommandSerializers.partialTxn.serialize(after.partialTxn(), out, userVersion); + break; + case PARTIAL_DEPS: + DepsSerializers.partialDeps.serialize(after.partialDeps(), out, userVersion); + break; + case WAITING_ON: + Command.WaitingOn waitingOn = getWaitingOn(after); + long size = WaitingOnSerializer.serializedSize(after.txnId(), waitingOn); + ByteBuffer serialized = WaitingOnSerializer.serialize(after.txnId(), waitingOn); + Invariants.checkState(serialized.remaining() == size); + out.writeInt((int) size); + out.write(serialized); + break; + case WRITES: + CommandSerializers.writes.serialize(after.writes(), out, userVersion); + break; + case RESULT: + ResultSerializers.result.serialize(after.result(), out, userVersion); + break; + case CLEANUP: + throw new IllegalStateException(); + } + + iterable = unsetIterableFields(field, iterable); + } + } + } + + public static class Builder extends CommandChange.Builder + { + public Builder() + { + super(); + } + + public Builder(TxnId txnId) + { + super(txnId); + } + + public Builder(TxnId txnId, Load load) + { + super(txnId, load); + } + + public ByteBuffer asByteBuffer(int userVersion) throws IOException + { + try (DataOutputBuffer out = new DataOutputBuffer()) + { + serialize(out, userVersion); + return out.asNewBuffer(); + } + } + + public Builder maybeCleanup(Cleanup cleanup) + { + super.maybeCleanup(cleanup); + return this; + } + + public void serialize(DataOutputPlus out, int userVersion) throws IOException + { + Invariants.checkState(mask == 0); + Invariants.checkState(flags != 0); + out.writeInt(validateFlags(flags)); + + int iterable = toIterableSetFields(flags); + while (iterable != 0) + { + CommandChange.Fields field = nextSetField(iterable); + if (getFieldIsNull(field, flags)) + { + iterable = unsetIterableFields(field, iterable); + continue; + } + + switch (field) + { + case EXECUTE_AT: + CommandSerializers.timestamp.serialize(executeAt(), out, userVersion); + break; + case EXECUTES_AT_LEAST: + CommandSerializers.timestamp.serialize(executeAtLeast(), out, userVersion); + break; + case SAVE_STATUS: + out.writeShort(saveStatus().ordinal()); + break; + case DURABILITY: + out.writeByte(durability().ordinal()); + break; + case ACCEPTED: + CommandSerializers.ballot.serialize(acceptedOrCommitted(), out, userVersion); + break; + case PROMISED: + CommandSerializers.ballot.serialize(promised(), out, userVersion); + break; + case PARTICIPANTS: + CommandSerializers.participants.serialize(participants(), out, userVersion); + break; + case PARTIAL_TXN: + CommandSerializers.partialTxn.serialize(partialTxn(), out, userVersion); + break; + case PARTIAL_DEPS: + DepsSerializers.partialDeps.serialize(partialDeps(), out, userVersion); + break; + case WAITING_ON: + Command.WaitingOn waitingOn = waitingOn().provide(txnId, partialDeps); + long size = WaitingOnSerializer.serializedSize(txnId, waitingOn); + ByteBuffer serialized = WaitingOnSerializer.serialize(txnId, waitingOn); + Invariants.checkState(serialized.remaining() == size); + out.writeInt((int) size); + out.write(serialized); + break; + case WRITES: + CommandSerializers.writes.serialize(writes(), out, userVersion); + break; + case CLEANUP: + out.writeByte(cleanup.ordinal()); + break; + case RESULT: + ResultSerializers.result.serialize(result(), out, userVersion); + break; + } + + iterable = unsetIterableFields(field, iterable); + } + } + + public void deserializeNext(DataInputPlus in, int userVersion) throws IOException + { + Invariants.checkState(txnId != null); + int flags = in.readInt(); + Invariants.checkState(flags != 0); + nextCalled = true; + count++; + + int iterable = toIterableSetFields(flags); + while (iterable != 0) + { + CommandChange.Fields field = nextSetField(iterable); + if (getFieldChanged(field, this.flags) || getFieldIsNull(field, mask)) + { + if (!getFieldIsNull(field, flags)) + skip(field, in, userVersion); + + iterable = unsetIterableFields(field, iterable); + continue; + } + this.flags = setFieldChanged(field, this.flags); + + if (getFieldIsNull(field, flags)) { - iter.readAllForKey(key, (segment, position, local, buffer, hosts, userVersion) -> { - pointers.add(new RecordPointer(segment, position)); - Invariants.checkState(finalKey.equals(local)); - try (DataInputBuffer in = new DataInputBuffer(buffer, false)) + this.flags = setFieldIsNull(field, this.flags); + } + else + { + deserialize(field, in, userVersion); + } + + iterable = unsetIterableFields(field, iterable); + } + } + + private void deserialize(CommandChange.Fields field, DataInputPlus in, int userVersion) throws IOException + { + switch (field) + { + case EXECUTE_AT: + executeAt = CommandSerializers.timestamp.deserialize(in, userVersion); + break; + case EXECUTES_AT_LEAST: + executeAtLeast = CommandSerializers.timestamp.deserialize(in, userVersion); + break; + case SAVE_STATUS: + saveStatus = SaveStatus.values()[in.readShort()]; + break; + case DURABILITY: + durability = accord.primitives.Status.Durability.values()[in.readByte()]; + break; + case ACCEPTED: + acceptedOrCommitted = CommandSerializers.ballot.deserialize(in, userVersion); + break; + case PROMISED: + promised = CommandSerializers.ballot.deserialize(in, userVersion); + break; + case PARTICIPANTS: + participants = CommandSerializers.participants.deserialize(in, userVersion); + break; + case PARTIAL_TXN: + partialTxn = CommandSerializers.partialTxn.deserialize(in, userVersion); + break; + case PARTIAL_DEPS: + partialDeps = DepsSerializers.partialDeps.deserialize(in, userVersion); + break; + case WAITING_ON: + int size = in.readInt(); + + byte[] waitingOnBytes = new byte[size]; + in.readFully(waitingOnBytes); + ByteBuffer buffer = ByteBuffer.wrap(waitingOnBytes); + waitingOn = (localTxnId, deps) -> { + try { - builder.deserializeNext(in, userVersion); + Invariants.nonNull(deps); + return WaitingOnSerializer.deserialize(localTxnId, deps.keyDeps.keys(), deps.rangeDeps, deps.directKeyDeps, buffer); } catch (IOException e) { - // can only throw if serializer is buggy - throw new RuntimeException(e); + throw Throwables.unchecked(e); } - }); - - Cleanup cleanup = builder.shouldCleanup(node.agent(), compactionInfo.redundantBefores.get(key.commandStoreId), compactionInfo.durableBefores.get(key.commandStoreId)); - switch (cleanup) - { - case ERASE: - case EXPUNGE: - case EXPUNGE_PARTIAL: - case VESTIGIAL: - continue; - } - builder.construct(); - } - catch (Throwable t) - { - throw new RuntimeException(String.format("Caught an exception after iterating over: %s", pointers), - t); - } + }; + break; + case WRITES: + writes = CommandSerializers.writes.deserialize(in, userVersion); + break; + case CLEANUP: + Cleanup newCleanup = Cleanup.forOrdinal(in.readByte()); + if (cleanup == null || newCleanup.compareTo(cleanup) > 0) + cleanup = newCleanup; + break; + case RESULT: + result = ResultSerializers.result.deserialize(in, userVersion); + break; } } - } - public void unsafeSetStarted() - { - status = Status.STARTED; + private void skip(CommandChange.Fields field, DataInputPlus in, int userVersion) throws IOException + { + switch (field) + { + case EXECUTE_AT: + case EXECUTES_AT_LEAST: + CommandSerializers.timestamp.skip(in, userVersion); + break; + case SAVE_STATUS: + in.readShort(); + break; + case DURABILITY: + in.readByte(); + break; + case ACCEPTED: + case PROMISED: + CommandSerializers.ballot.skip(in, userVersion); + break; + case PARTICIPANTS: + // TODO (expected): skip + CommandSerializers.participants.deserialize(in, userVersion); + break; + case PARTIAL_TXN: + CommandSerializers.partialTxn.deserialize(in, userVersion); + break; + case PARTIAL_DEPS: + // TODO (expected): skip + DepsSerializers.partialDeps.deserialize(in, userVersion); + break; + case WAITING_ON: + int size = in.readInt(); + in.skipBytesFully(size); + break; + case WRITES: + // TODO (expected): skip + CommandSerializers.writes.deserialize(in, userVersion); + break; + case CLEANUP: + in.readByte(); + break; + case RESULT: + // TODO (expected): skip + result = ResultSerializers.result.deserialize(in, userVersion); + break; + } + } } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java index ad8d2594dd4b..b313e7e924e7 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java @@ -35,8 +35,8 @@ import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; import org.apache.cassandra.service.accord.serializers.KeySerializers; +import static accord.api.Journal.Load.ALL; import static accord.local.CommandStores.RangesForEpoch; -import static org.apache.cassandra.service.accord.SavedCommand.Load.ALL; // TODO (required): test with large collection values, and perhaps split out some fields if they have a tendency to grow larger // TODO (required): alert on metadata size @@ -56,16 +56,16 @@ public interface FlyweightSerializer } public static class CommandDiffSerializer - implements FlyweightSerializer + implements FlyweightSerializer { @Override - public SavedCommand.Builder mergerFor(JournalKey journalKey) + public AccordJournal.Builder mergerFor(JournalKey journalKey) { - return new SavedCommand.Builder(journalKey.id, ALL); + return new AccordJournal.Builder(journalKey.id, ALL); } @Override - public void serialize(JournalKey key, SavedCommand.Writer writer, DataOutputPlus out, int userVersion) + public void serialize(JournalKey key, AccordJournal.Writer writer, DataOutputPlus out, int userVersion) { try { @@ -78,13 +78,13 @@ public void serialize(JournalKey key, SavedCommand.Writer writer, DataOutputPlus } @Override - public void reserialize(JournalKey key, SavedCommand.Builder from, DataOutputPlus out, int userVersion) throws IOException + public void reserialize(JournalKey key, AccordJournal.Builder from, DataOutputPlus out, int userVersion) throws IOException { from.serialize(out, userVersion); } @Override - public void deserialize(JournalKey journalKey, SavedCommand.Builder into, DataInputPlus in, int userVersion) throws IOException + public void deserialize(JournalKey journalKey, AccordJournal.Builder into, DataInputPlus in, int userVersion) throws IOException { into.deserializeNext(in, userVersion); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java index d5c03c8e0a04..edc473cf3570 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java @@ -32,6 +32,7 @@ import accord.api.RoutingKey; import accord.impl.AbstractSafeCommandStore; import accord.impl.CommandsSummary; +import accord.local.CommandStores; import accord.local.NodeCommandStoreService; import accord.local.cfk.CommandsForKey; import accord.primitives.AbstractKeys; diff --git a/src/java/org/apache/cassandra/service/accord/CommandsForRanges.java b/src/java/org/apache/cassandra/service/accord/CommandsForRanges.java index 13b21b5da8c5..7c2eddff8df3 100644 --- a/src/java/org/apache/cassandra/service/accord/CommandsForRanges.java +++ b/src/java/org/apache/cassandra/service/accord/CommandsForRanges.java @@ -334,7 +334,7 @@ public Summary load(TxnId txnId) { if (findAsDep == null) { - SavedCommand.MinimalCommand cmd = manager.commandStore.loadMinimal(txnId); + Command.Minimal cmd = manager.commandStore.loadMinimal(txnId); if (cmd != null) return from(cmd); } @@ -390,7 +390,7 @@ public Summary from(Command cmd) return from(cmd.txnId(), cmd.executeAt(), cmd.saveStatus(), cmd.participants(), cmd.partialDeps()); } - public Summary from(SavedCommand.MinimalCommand cmd) + public Summary from(Command.Minimal cmd) { Invariants.checkState(findAsDep == null); return from(cmd.txnId, cmd.executeAt, cmd.saveStatus, cmd.participants, null); diff --git a/src/java/org/apache/cassandra/service/accord/IJournal.java b/src/java/org/apache/cassandra/service/accord/IJournal.java deleted file mode 100644 index a96bfc7afdd7..000000000000 --- a/src/java/org/apache/cassandra/service/accord/IJournal.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.service.accord; - -import accord.api.Journal; -import accord.local.Command; -import accord.local.DurableBefore; -import accord.local.RedundantBefore; -import accord.primitives.TxnId; -import accord.utils.PersistentField.Persister; - -public interface IJournal extends Journal -{ - // TODO (required): migrate to accord.api.Journal - default SavedCommand.MinimalCommand loadMinimal(int commandStoreId, TxnId txnId, SavedCommand.Load load, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - Command command = loadCommand(commandStoreId, txnId, redundantBefore, durableBefore); - return new SavedCommand.MinimalCommand(command.txnId(), command.saveStatus(), command.participants(), command.durability(), command.executeAt(), command.writes()); - } - - Persister durableBeforePersister(); -} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/SavedCommand.java b/src/java/org/apache/cassandra/service/accord/SavedCommand.java deleted file mode 100644 index dfa872c4fe6c..000000000000 --- a/src/java/org/apache/cassandra/service/accord/SavedCommand.java +++ /dev/null @@ -1,983 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.service.accord; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.function.Function; -import javax.annotation.Nullable; - -import com.google.common.annotations.VisibleForTesting; - -import accord.api.Agent; -import accord.api.Result; -import accord.local.Cleanup; -import accord.local.Command; -import accord.local.CommonAttributes; -import accord.local.DurableBefore; -import accord.local.RedundantBefore; -import accord.local.StoreParticipants; -import accord.primitives.Ballot; -import accord.primitives.PartialDeps; -import accord.primitives.PartialTxn; -import accord.primitives.SaveStatus; -import accord.primitives.Status; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.primitives.Writes; -import accord.utils.Invariants; -import org.apache.cassandra.io.util.DataInputPlus; -import org.apache.cassandra.io.util.DataOutputBuffer; -import org.apache.cassandra.io.util.DataOutputPlus; -import org.apache.cassandra.journal.Journal; -import org.apache.cassandra.service.accord.serializers.CommandSerializers; -import org.apache.cassandra.service.accord.serializers.DepsSerializers; -import org.apache.cassandra.service.accord.serializers.ResultSerializers; -import org.apache.cassandra.service.accord.serializers.WaitingOnSerializer; -import org.apache.cassandra.utils.Throwables; - -import static accord.local.Cleanup.NO; -import static accord.local.Cleanup.TRUNCATE_WITH_OUTCOME; -import static accord.primitives.Known.KnownDeps.DepsErased; -import static accord.primitives.Known.KnownDeps.DepsUnknown; -import static accord.primitives.Known.KnownDeps.NoDeps; -import static accord.primitives.SaveStatus.TruncatedApplyWithOutcome; -import static accord.primitives.Status.Durability.NotDurable; -import static accord.utils.Invariants.illegalState; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.DURABILITY; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.EXECUTE_AT; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.PARTICIPANTS; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.RESULT; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.SAVE_STATUS; -import static org.apache.cassandra.service.accord.SavedCommand.Fields.WRITES; -import static org.apache.cassandra.service.accord.SavedCommand.Load.ALL; - -public class SavedCommand -{ - // This enum is order-dependent - public enum Fields - { - PARTICIPANTS, // stored first so we can index it - SAVE_STATUS, - PARTIAL_DEPS, - EXECUTE_AT, - EXECUTES_AT_LEAST, - DURABILITY, - ACCEPTED, - PROMISED, - WAITING_ON, - PARTIAL_TXN, - WRITES, - CLEANUP, - RESULT, - ; - - public static final Fields[] FIELDS = values(); - } - - // TODO: maybe rename this and enclosing classes? - public static class Writer implements Journal.Writer - { - private final Command after; - private final TxnId txnId; - private final int flags; - - @VisibleForTesting - public Writer(Command after, int flags) - { - this(after.txnId(), after, flags); - } - - @VisibleForTesting - public Writer(TxnId txnId, Command after, int flags) - { - this.txnId = txnId; - this.after = after; - this.flags = flags; - } - - @VisibleForTesting - public Command after() - { - return after; - } - - public void write(DataOutputPlus out, int userVersion) throws IOException - { - serialize(after, flags, out, userVersion); - } - - public TxnId key() - { - return txnId; - } - } - - public static @Nullable ByteBuffer asSerializedDiff(Command before, Command after, int userVersion) throws IOException - { - try (DataOutputBuffer out = new DataOutputBuffer()) - { - Writer writer = diff(before, after); - if (writer == null) - return null; - - writer.write(out, userVersion); - return out.asNewBuffer(); - } - } - - @Nullable - public static Writer diff(Command before, Command after) - { - if (before == after - || after == null - || after.saveStatus() == SaveStatus.Uninitialised) - return null; - - int flags = validateFlags(getFlags(before, after)); - if (!anyFieldChanged(flags)) - return null; - - return new Writer(after, flags); - } - - // TODO (required): calculate flags once - private static boolean anyFieldChanged(int flags) - { - return (flags >>> 16) != 0; - } - - private static int validateFlags(int flags) - { - Invariants.checkState(0 == (~(flags >>> 16) & (flags & 0xffff))); - return flags; - } - - public static void serialize(Command after, int flags, DataOutputPlus out, int userVersion) throws IOException - { - Invariants.checkState(flags != 0); - out.writeInt(flags); - - int iterable = toIterableSetFields(flags); - while (iterable != 0) - { - Fields field = nextSetField(iterable); - if (getFieldIsNull(field, flags)) - { - iterable = unsetIterableFields(field, iterable); - continue; - } - - switch (field) - { - case EXECUTE_AT: - CommandSerializers.timestamp.serialize(after.executeAt(), out, userVersion); - break; - case EXECUTES_AT_LEAST: - CommandSerializers.timestamp.serialize(after.executesAtLeast(), out, userVersion); - break; - case SAVE_STATUS: - out.writeShort(after.saveStatus().ordinal()); - break; - case DURABILITY: - out.writeByte(after.durability().ordinal()); - break; - case ACCEPTED: - CommandSerializers.ballot.serialize(after.acceptedOrCommitted(), out, userVersion); - break; - case PROMISED: - CommandSerializers.ballot.serialize(after.promised(), out, userVersion); - break; - case PARTICIPANTS: - CommandSerializers.participants.serialize(after.participants(), out, userVersion); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.serialize(after.partialTxn(), out, userVersion); - break; - case PARTIAL_DEPS: - DepsSerializers.partialDeps.serialize(after.partialDeps(), out, userVersion); - break; - case WAITING_ON: - Command.WaitingOn waitingOn = getWaitingOn(after); - long size = WaitingOnSerializer.serializedSize(after.txnId(), waitingOn); - ByteBuffer serialized = WaitingOnSerializer.serialize(after.txnId(), waitingOn); - Invariants.checkState(serialized.remaining() == size); - out.writeInt((int) size); - out.write(serialized); - break; - case WRITES: - CommandSerializers.writes.serialize(after.writes(), out, userVersion); - break; - case RESULT: - ResultSerializers.result.serialize(after.result(), out, userVersion); - break; - case CLEANUP: - throw new IllegalStateException(); - } - - iterable = unsetIterableFields(field, iterable); - } - } - - @VisibleForTesting - public static int getFlags(Command before, Command after) - { - int flags = 0; - - flags = collectFlags(before, after, Command::executeAt, true, Fields.EXECUTE_AT, flags); - flags = collectFlags(before, after, Command::executesAtLeast, true, Fields.EXECUTES_AT_LEAST, flags); - flags = collectFlags(before, after, Command::saveStatus, false, SAVE_STATUS, flags); - flags = collectFlags(before, after, Command::durability, false, DURABILITY, flags); - - flags = collectFlags(before, after, Command::acceptedOrCommitted, false, Fields.ACCEPTED, flags); - flags = collectFlags(before, after, Command::promised, false, Fields.PROMISED, flags); - - flags = collectFlags(before, after, Command::participants, true, PARTICIPANTS, flags); - flags = collectFlags(before, after, Command::partialTxn, false, Fields.PARTIAL_TXN, flags); - flags = collectFlags(before, after, Command::partialDeps, false, Fields.PARTIAL_DEPS, flags); - - // TODO: waitingOn vs WaitingOnWithExecutedAt? - flags = collectFlags(before, after, SavedCommand::getWaitingOn, true, Fields.WAITING_ON, flags); - - flags = collectFlags(before, after, Command::writes, false, WRITES, flags); - - // Special-cased for Journal BurnTest integration - if ((before != null && before.result() != null && before.result() != ResultSerializers.APPLIED) || - (after.result() != null && after.result() != ResultSerializers.APPLIED)) - { - flags = collectFlags(before, after, Command::writes, false, RESULT, flags); - } - - return flags; - } - - static Command.WaitingOn getWaitingOn(Command command) - { - if (command instanceof Command.Committed) - return command.asCommitted().waitingOn(); - - return null; - } - - private static int collectFlags(OBJ lo, OBJ ro, Function convert, boolean allowClassMismatch, Fields field, int flags) - { - VAL l = null; - VAL r = null; - if (lo != null) l = convert.apply(lo); - if (ro != null) r = convert.apply(ro); - - if (l == r) - return flags; // no change - - if (r == null) - flags = setFieldIsNull(field, flags); - - if (l == null || r == null) - return setFieldChanged(field, flags); - - assert allowClassMismatch || l.getClass() == r.getClass() : String.format("%s != %s", l.getClass(), r.getClass()); - - if (l.equals(r)) - return flags; // no change - - return setFieldChanged(field, flags); - } - - private static int setFieldChanged(Fields field, int oldFlags) - { - return oldFlags | (0x10000 << field.ordinal()); - } - - @VisibleForTesting - static boolean getFieldChanged(Fields field, int oldFlags) - { - return (oldFlags & (0x10000 << field.ordinal())) != 0; - } - - static int toIterableSetFields(int flags) - { - return flags >>> 16; - } - - static Fields nextSetField(int iterable) - { - int i = Integer.numberOfTrailingZeros(Integer.lowestOneBit(iterable)); - return i == 32 ? null : Fields.FIELDS[i]; - } - - static int unsetIterableFields(Fields field, int iterable) - { - return iterable & ~(1 << field.ordinal()); - } - - @VisibleForTesting - static boolean getFieldIsNull(Fields field, int oldFlags) - { - return (oldFlags & (1 << field.ordinal())) != 0; - } - - private static int setFieldIsNull(Fields field, int oldFlags) - { - return oldFlags | (1 << field.ordinal()); - } - - public enum Load - { - ALL(0), - PURGEABLE(SAVE_STATUS, PARTICIPANTS, DURABILITY, EXECUTE_AT, WRITES), - MINIMAL(SAVE_STATUS, PARTICIPANTS, EXECUTE_AT); - - final int mask; - - Load(int mask) - { - this.mask = mask; - } - - Load(Fields ... fields) - { - int mask = -1; - for (Fields field : fields) - mask &= ~(1<< field.ordinal()); - this.mask = mask; - } - } - - public static class MinimalCommand - { - public final TxnId txnId; - public final SaveStatus saveStatus; - public final StoreParticipants participants; - public final Status.Durability durability; - public final Timestamp executeAt; - public final Writes writes; - - public MinimalCommand(TxnId txnId, SaveStatus saveStatus, StoreParticipants participants, Status.Durability durability, Timestamp executeAt, Writes writes) - { - this.txnId = txnId; - this.saveStatus = saveStatus; - this.participants = participants; - this.durability = durability; - this.executeAt = executeAt; - this.writes = writes; - } - } - - public static class Builder - { - final int mask; - int flags; - - TxnId txnId; - - Timestamp executeAt; - Timestamp executeAtLeast; - SaveStatus saveStatus; - Status.Durability durability; - - Ballot acceptedOrCommitted; - Ballot promised; - - StoreParticipants participants; - PartialTxn partialTxn; - PartialDeps partialDeps; - - SavedCommand.WaitingOnProvider waitingOn; - Writes writes; - Result result; - Cleanup cleanup; - - boolean nextCalled; - int count; - - public Builder(TxnId txnId, Load load) - { - this.mask = load.mask; - init(txnId); - } - - public Builder(TxnId txnId) - { - this(txnId, ALL); - } - - public Builder(Load load) - { - this.mask = load.mask; - } - - public Builder() - { - this(ALL); - } - - public TxnId txnId() - { - return txnId; - } - - public Timestamp executeAt() - { - return executeAt; - } - - public Timestamp executeAtLeast() - { - return executeAtLeast; - } - - public SaveStatus saveStatus() - { - return saveStatus; - } - - public Status.Durability durability() - { - return durability; - } - - public Ballot acceptedOrCommitted() - { - return acceptedOrCommitted; - } - - public Ballot promised() - { - return promised; - } - - public StoreParticipants participants() - { - return participants; - } - - public PartialTxn partialTxn() - { - return partialTxn; - } - - public PartialDeps partialDeps() - { - return partialDeps; - } - - public SavedCommand.WaitingOnProvider waitingOn() - { - return waitingOn; - } - - public Writes writes() - { - return writes; - } - - public Result result() - { - return result; - } - - public void clear() - { - flags = 0; - txnId = null; - - executeAt = null; - executeAtLeast = null; - saveStatus = null; - durability = null; - - acceptedOrCommitted = null; - promised = null; - - participants = null; - partialTxn = null; - partialDeps = null; - - waitingOn = null; - writes = null; - result = null; - cleanup = null; - - nextCalled = false; - count = 0; - } - - public void reset(TxnId txnId) - { - clear(); - init(txnId); - } - - public void init(TxnId txnId) - { - this.txnId = txnId; - durability = NotDurable; - acceptedOrCommitted = promised = Ballot.ZERO; - waitingOn = (txn, deps) -> null; - result = ResultSerializers.APPLIED; - } - - public boolean isEmpty() - { - return !nextCalled; - } - - public int count() - { - return count; - } - - public Cleanup shouldCleanup(Agent agent, RedundantBefore redundantBefore, DurableBefore durableBefore) - { - if (!nextCalled) - return NO; - - if (saveStatus == null || participants == null) - return Cleanup.NO; - - Cleanup cleanup = Cleanup.shouldCleanupPartial(agent, txnId, saveStatus, durability, participants, redundantBefore, durableBefore); - if (this.cleanup != null && this.cleanup.compareTo(cleanup) > 0) - cleanup = this.cleanup; - return cleanup; - } - - // TODO (expected): avoid allocating new builder - public Builder maybeCleanup(Cleanup cleanup) - { - if (saveStatus() == null) - return this; - - switch (cleanup) - { - case EXPUNGE: - case ERASE: - return null; - - case EXPUNGE_PARTIAL: - return expungePartial(cleanup, saveStatus, true); - - case VESTIGIAL: - case INVALIDATE: - return saveStatusOnly(); - - case TRUNCATE_WITH_OUTCOME: - case TRUNCATE: - return expungePartial(cleanup, cleanup.appliesIfNot, cleanup == TRUNCATE_WITH_OUTCOME); - - case NO: - return this; - default: - throw new IllegalStateException("Unknown cleanup: " + cleanup);} - } - - private Builder expungePartial(Cleanup cleanup, SaveStatus saveStatus, boolean includeOutcome) - { - Invariants.checkState(txnId != null); - Builder builder = new Builder(txnId, ALL); - - builder.count++; - builder.nextCalled = true; - - Invariants.checkState(saveStatus != null); - builder.flags = setFieldChanged(SAVE_STATUS, builder.flags); - builder.saveStatus = saveStatus; - builder.flags = setFieldChanged(Fields.CLEANUP, builder.flags); - builder.cleanup = cleanup; - if (executeAt != null) - { - builder.flags = setFieldChanged(Fields.EXECUTE_AT, builder.flags); - builder.executeAt = executeAt; - } - if (durability != null) - { - builder.flags = setFieldChanged(DURABILITY, builder.flags); - builder.durability = durability; - } - if (participants != null) - { - builder.flags = setFieldChanged(PARTICIPANTS, builder.flags); - builder.participants = participants; - } - if (includeOutcome && builder.writes != null) - { - builder.flags = setFieldChanged(WRITES, builder.flags); - builder.writes = writes; - } - - return builder; - } - - private Builder saveStatusOnly() - { - Invariants.checkState(txnId != null); - Builder builder = new Builder(txnId, ALL); - - builder.count++; - builder.nextCalled = true; - - // TODO: these accesses can be abstracted away - if (saveStatus != null) - { - builder.flags = setFieldChanged(SAVE_STATUS, builder.flags); - builder.saveStatus = saveStatus; - } - - return builder; - } - - public ByteBuffer asByteBuffer(int userVersion) throws IOException - { - try (DataOutputBuffer out = new DataOutputBuffer()) - { - serialize(out, userVersion); - return out.asNewBuffer(); - } - } - - public MinimalCommand asMinimal() - { - return new MinimalCommand(txnId, saveStatus, participants, durability, executeAt, writes); - } - - public void serialize(DataOutputPlus out, int userVersion) throws IOException - { - Invariants.checkState(mask == 0); - Invariants.checkState(flags != 0); - out.writeInt(validateFlags(flags)); - - int iterable = toIterableSetFields(flags); - while (iterable != 0) - { - Fields field = nextSetField(iterable); - if (getFieldIsNull(field, flags)) - { - iterable = unsetIterableFields(field, iterable); - continue; - } - - switch (field) - { - case EXECUTE_AT: - CommandSerializers.timestamp.serialize(executeAt(), out, userVersion); - break; - case EXECUTES_AT_LEAST: - CommandSerializers.timestamp.serialize(executeAtLeast(), out, userVersion); - break; - case SAVE_STATUS: - out.writeShort(saveStatus().ordinal()); - break; - case DURABILITY: - out.writeByte(durability().ordinal()); - break; - case ACCEPTED: - CommandSerializers.ballot.serialize(acceptedOrCommitted(), out, userVersion); - break; - case PROMISED: - CommandSerializers.ballot.serialize(promised(), out, userVersion); - break; - case PARTICIPANTS: - CommandSerializers.participants.serialize(participants(), out, userVersion); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.serialize(partialTxn(), out, userVersion); - break; - case PARTIAL_DEPS: - DepsSerializers.partialDeps.serialize(partialDeps(), out, userVersion); - break; - case WAITING_ON: - Command.WaitingOn waitingOn = waitingOn().provide(txnId, partialDeps); - long size = WaitingOnSerializer.serializedSize(txnId, waitingOn); - ByteBuffer serialized = WaitingOnSerializer.serialize(txnId, waitingOn); - Invariants.checkState(serialized.remaining() == size); - out.writeInt((int) size); - out.write(serialized); - break; - case WRITES: - CommandSerializers.writes.serialize(writes(), out, userVersion); - break; - case CLEANUP: - out.writeByte(cleanup.ordinal()); - break; - case RESULT: - ResultSerializers.result.serialize(result(), out, userVersion); - break; - } - - iterable = unsetIterableFields(field, iterable); - } - } - - public void deserializeNext(DataInputPlus in, int userVersion) throws IOException - { - Invariants.checkState(txnId != null); - int flags = in.readInt(); - Invariants.checkState(flags != 0); - nextCalled = true; - count++; - - int iterable = toIterableSetFields(flags); - while (iterable != 0) - { - Fields field = nextSetField(iterable); - if (getFieldChanged(field, this.flags) || getFieldIsNull(field, mask)) - { - if (!getFieldIsNull(field, flags)) - skip(field, in, userVersion); - - iterable = unsetIterableFields(field, iterable); - continue; - } - this.flags = setFieldChanged(field, this.flags); - - if (getFieldIsNull(field, flags)) - { - this.flags = setFieldIsNull(field, this.flags); - } - else - { - deserialize(field, in, userVersion); - } - - iterable = unsetIterableFields(field, iterable); - } - } - - private void deserialize(Fields field, DataInputPlus in, int userVersion) throws IOException - { - switch (field) - { - case EXECUTE_AT: - executeAt = CommandSerializers.timestamp.deserialize(in, userVersion); - break; - case EXECUTES_AT_LEAST: - executeAtLeast = CommandSerializers.timestamp.deserialize(in, userVersion); - break; - case SAVE_STATUS: - saveStatus = SaveStatus.values()[in.readShort()]; - break; - case DURABILITY: - durability = Status.Durability.values()[in.readByte()]; - break; - case ACCEPTED: - acceptedOrCommitted = CommandSerializers.ballot.deserialize(in, userVersion); - break; - case PROMISED: - promised = CommandSerializers.ballot.deserialize(in, userVersion); - break; - case PARTICIPANTS: - participants = CommandSerializers.participants.deserialize(in, userVersion); - break; - case PARTIAL_TXN: - partialTxn = CommandSerializers.partialTxn.deserialize(in, userVersion); - break; - case PARTIAL_DEPS: - partialDeps = DepsSerializers.partialDeps.deserialize(in, userVersion); - break; - case WAITING_ON: - int size = in.readInt(); - - byte[] waitingOnBytes = new byte[size]; - in.readFully(waitingOnBytes); - ByteBuffer buffer = ByteBuffer.wrap(waitingOnBytes); - waitingOn = (localTxnId, deps) -> { - try - { - Invariants.nonNull(deps); - return WaitingOnSerializer.deserialize(localTxnId, deps.keyDeps.keys(), deps.rangeDeps, deps.directKeyDeps, buffer); - } - catch (IOException e) - { - throw Throwables.unchecked(e); - } - }; - break; - case WRITES: - writes = CommandSerializers.writes.deserialize(in, userVersion); - break; - case CLEANUP: - Cleanup newCleanup = Cleanup.forOrdinal(in.readByte()); - if (cleanup == null || newCleanup.compareTo(cleanup) > 0) - cleanup = newCleanup; - break; - case RESULT: - result = ResultSerializers.result.deserialize(in, userVersion); - break; - } - } - - private void skip(Fields field, DataInputPlus in, int userVersion) throws IOException - { - switch (field) - { - case EXECUTE_AT: - case EXECUTES_AT_LEAST: - CommandSerializers.timestamp.skip(in, userVersion); - break; - case SAVE_STATUS: - in.readShort(); - break; - case DURABILITY: - in.readByte(); - break; - case ACCEPTED: - case PROMISED: - CommandSerializers.ballot.skip(in, userVersion); - break; - case PARTICIPANTS: - // TODO (expected): skip - CommandSerializers.participants.deserialize(in, userVersion); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.deserialize(in, userVersion); - break; - case PARTIAL_DEPS: - // TODO (expected): skip - DepsSerializers.partialDeps.deserialize(in, userVersion); - break; - case WAITING_ON: - int size = in.readInt(); - in.skipBytesFully(size); - break; - case WRITES: - // TODO (expected): skip - CommandSerializers.writes.deserialize(in, userVersion); - break; - case CLEANUP: - in.readByte(); - break; - case RESULT: - // TODO (expected): skip - result = ResultSerializers.result.deserialize(in, userVersion); - break; - } - } - - public void forceResult(Result newValue) - { - this.result = newValue; - } - - public Command construct() - { - if (!nextCalled) - return null; - - Invariants.checkState(txnId != null); - CommonAttributes.Mutable attrs = new CommonAttributes.Mutable(txnId); - if (partialTxn != null) - attrs.partialTxn(partialTxn); - if (durability != null) - attrs.durability(durability); - if (participants != null) - attrs.setParticipants(participants); - else - attrs.setParticipants(StoreParticipants.empty(txnId)); - if (partialDeps != null && - (saveStatus.known.deps != NoDeps && - saveStatus.known.deps != DepsErased && - saveStatus.known.deps != DepsUnknown)) - attrs.partialDeps(partialDeps); - - switch (saveStatus.known.outcome) - { - case Erased: - case WasApply: - writes = null; - result = null; - break; - } - - Command.WaitingOn waitingOn = null; - if (this.waitingOn != null) - waitingOn = this.waitingOn.provide(txnId, partialDeps); - - switch (saveStatus.status) - { - case NotDefined: - return saveStatus == SaveStatus.Uninitialised ? Command.NotDefined.uninitialised(attrs.txnId()) - : Command.NotDefined.notDefined(attrs, promised); - case PreAccepted: - return Command.PreAccepted.preAccepted(attrs, executeAt, promised); - case AcceptedInvalidate: - case Accepted: - case PreCommitted: - if (saveStatus == SaveStatus.AcceptedInvalidate) - return Command.AcceptedInvalidateWithoutDefinition.acceptedInvalidate(attrs, promised, acceptedOrCommitted); - else - return Command.Accepted.accepted(attrs, saveStatus, executeAt, promised, acceptedOrCommitted); - case Committed: - case Stable: - return Command.Committed.committed(attrs, saveStatus, executeAt, promised, acceptedOrCommitted, waitingOn); - case PreApplied: - case Applied: - return Command.Executed.executed(attrs, saveStatus, executeAt, promised, acceptedOrCommitted, waitingOn, writes, result); - case Truncated: - case Invalidated: - return truncated(attrs, saveStatus, executeAt, executeAtLeast, writes, result); - default: - throw new IllegalStateException(); - } - } - - private static Command.Truncated truncated(CommonAttributes.Mutable attrs, SaveStatus status, Timestamp executeAt, Timestamp executesAtLeast, Writes writes, Result result) - { - switch (status) - { - default: - throw illegalState("Unhandled SaveStatus: " + status); - case TruncatedApplyWithOutcome: - case TruncatedApplyWithDeps: - case TruncatedApply: - if (status != TruncatedApplyWithOutcome) - result = null; - if (attrs.txnId().kind().awaitsOnlyDeps()) - return Command.Truncated.truncatedApply(attrs, status, executeAt, writes, result, executesAtLeast); - return Command.Truncated.truncatedApply(attrs, status, executeAt, writes, result, null); - case ErasedOrVestigial: - return Command.Truncated.erasedOrInvalidOrVestigial(attrs.txnId(), attrs.durability(), attrs.participants()); - case Erased: - return Command.Truncated.erased(attrs.txnId(), attrs.durability(), attrs.participants()); - case Invalidated: - return Command.Truncated.invalidated(attrs.txnId()); - } - } - - public String toString() - { - return "Diff {" + - "txnId=" + txnId + - ", executeAt=" + executeAt + - ", saveStatus=" + saveStatus + - ", durability=" + durability + - ", acceptedOrCommitted=" + acceptedOrCommitted + - ", promised=" + promised + - ", participants=" + participants + - ", partialTxn=" + partialTxn + - ", partialDeps=" + partialDeps + - ", waitingOn=" + waitingOn + - ", writes=" + writes + - '}'; - } - } - - public interface WaitingOnProvider - { - Command.WaitingOn provide(TxnId txnId, PartialDeps deps); - } -} \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java index 6ad2f09d2a00..2a99a350b58e 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordLoadTest.java @@ -186,9 +186,6 @@ else if (initialised.get(k)) System.out.println("compacting accord..."); cluster.forEach(i -> { i.nodetool("compact", "system_accord.journal"); - i.runOnInstance(() -> { - ((AccordService) AccordService.instance()).journal().checkAllCommands(); - }); }); } @@ -198,7 +195,6 @@ else if (initialised.get(k)) System.out.println("flushing journal..."); cluster.forEach(i -> i.runOnInstance(() -> { ((AccordService) AccordService.instance()).journal().closeCurrentSegmentForTestingIfNonEmpty(); - ((AccordService) AccordService.instance()).journal().checkAllCommands(); })); } diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java index 255c2e075886..ad33206a2766 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java @@ -137,7 +137,7 @@ public void testOne() operations, 10 + random.nextInt(30), new RandomDelayQueue.Factory(random).get(), - (node) -> { + (node, agent) -> { try { File directory = new File(Files.createTempDirectory(Integer.toString(counter.incrementAndGet()))); @@ -176,6 +176,4 @@ public boolean enableCompaction() throw SimulationException.wrap(seed, t); } } - - } diff --git a/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java b/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java index 34bb215c0ac8..3c655133fab0 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordJournalOrderTest.java @@ -89,7 +89,7 @@ public void simpleKeyTest() Runnable check = () -> { for (JournalKey key : res.keySet()) { - SavedCommand.Builder diffs = accordJournal.loadDiffs(key.commandStoreId, key.id); + AccordJournal.Builder diffs = accordJournal.load(key.commandStoreId, key.id); Assert.assertEquals(String.format("%d != %d for key %s", diffs.count(), res.get(key).intValue(), key), diffs.count(), res.get(key).intValue()); } diff --git a/test/unit/org/apache/cassandra/service/accord/SavedCommandTest.java b/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java similarity index 88% rename from test/unit/org/apache/cassandra/service/accord/SavedCommandTest.java rename to test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java index 1760286ba3d7..2d7383466177 100644 --- a/test/unit/org/apache/cassandra/service/accord/SavedCommandTest.java +++ b/test/unit/org/apache/cassandra/service/accord/CommandChangeTest.java @@ -26,6 +26,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import accord.impl.CommandChange; import accord.local.Command; import accord.primitives.SaveStatus; import accord.primitives.TxnId; @@ -39,17 +40,17 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.service.StorageService; -import org.apache.cassandra.service.accord.SavedCommand.Fields; -import org.apache.cassandra.service.accord.SavedCommand.Load; import org.apache.cassandra.service.consensus.TransactionalMode; import org.apache.cassandra.utils.AccordGenerators; import org.assertj.core.api.SoftAssertions; +import static accord.api.Journal.*; +import static accord.impl.CommandChange.*; +import static accord.impl.CommandChange.getFlags; import static accord.utils.Property.qt; import static org.apache.cassandra.cql3.statements.schema.CreateTableStatement.parse; -import static org.apache.cassandra.service.accord.SavedCommand.getFlags; -public class SavedCommandTest +public class CommandChangeTest { private static final EnumSet ALL = EnumSet.allOf(Fields.class); @@ -97,8 +98,9 @@ public void serde() if (saveStatus == SaveStatus.TruncatedApplyWithDeps) continue; out.clear(); Command orig = cmdBuilder.build(saveStatus); - SavedCommand.serialize(orig, getFlags(null, orig), out, userVersion); - SavedCommand.Builder builder = new SavedCommand.Builder(orig.txnId(), Load.ALL); + + AccordJournal.Writer.make(null, orig).write(out, userVersion); + AccordJournal.Builder builder = new AccordJournal.Builder(orig.txnId(), Load.ALL); builder.deserializeNext(new DataInputBuffer(out.unsafeGetBufferAndFlip(), false), userVersion); // We are not persisting the result, so force it for strict equality builder.forceResult(orig.result()); @@ -119,10 +121,10 @@ private void assertHas(int flags, Set missing) SoftAssertions checks = new SoftAssertions(); for (Fields field : missing) { - checks.assertThat(SavedCommand.getFieldChanged(field, flags)) + checks.assertThat(CommandChange.getFieldChanged(field, flags)) .describedAs("field %s changed", field). isTrue(); - checks.assertThat(SavedCommand.getFieldIsNull(field, flags)) + checks.assertThat(CommandChange.getFieldIsNull(field, flags)) .describedAs("field %s not null", field) .isFalse(); } @@ -135,11 +137,11 @@ private void assertMissing(int flags, Set missing) for (Fields field : missing) { if (field == Fields.CLEANUP) continue; - checks.assertThat(SavedCommand.getFieldChanged(field, flags)) + checks.assertThat(CommandChange.getFieldChanged(field, flags)) .describedAs("field %s changed", field) .isFalse(); // Is null flag can not be set on a field that has not changed - checks.assertThat(SavedCommand.getFieldIsNull(field, flags)) + checks.assertThat(CommandChange.getFieldIsNull(field, flags)) .describedAs("field %s not null", field) .isFalse(); } diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java index fe82e3b009f6..76d152f91520 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java @@ -30,6 +30,7 @@ import java.util.function.Predicate; import java.util.function.ToLongFunction; +import accord.api.Journal; import accord.api.LocalListeners; import accord.api.ProgressLog; import accord.api.RemoteListeners; @@ -39,6 +40,7 @@ import accord.impl.DefaultTimeouts; import accord.impl.SizeOfIntersectionSorter; import accord.impl.TestAgent; +import accord.impl.basic.InMemoryJournal; import accord.impl.basic.SimulatedFault; import accord.local.Command; import accord.local.CommandStore; @@ -68,7 +70,6 @@ import accord.topology.Topologies; import accord.topology.Topology; import accord.utils.Gens; -import accord.utils.PersistentField; import accord.utils.RandomSource; import accord.utils.async.AsyncChains; import accord.utils.async.AsyncResult; @@ -105,7 +106,7 @@ public class SimulatedAccordCommandStore implements AutoCloseable public final Node.Id nodeId; public final Topology topology; public final Topologies topologies; - public final IJournal journal; + public final Journal journal; public final ScheduledExecutorPlus unorderedScheduled; public final List evictions = new ArrayList<>(); public Predicate ignoreExceptions = ignore -> false; @@ -185,7 +186,6 @@ public Timestamp uniqueNow(Timestamp atLeast) } }; - this.journal = new InMemoryJournal(nodeId); TestAgent.RethrowAgent agent = new TestAgent.RethrowAgent() { @Override @@ -201,6 +201,8 @@ public void onUncaughtException(Throwable t) super.onUncaughtException(t); } }; + + this.journal = new InMemoryJournal(nodeId, agent); this.commandStore = new AccordCommandStore(0, storeService, agent, @@ -246,19 +248,6 @@ public void onEvict(AccordCacheEntry state) }); } - private final class InMemoryJournal extends accord.impl.basic.InMemoryJournal implements IJournal - { - public InMemoryJournal(Node.Id id) - { - super(id); - } - - public PersistentField.Persister durableBeforePersister() - { - throw new IllegalArgumentException("Not implemented"); - } - } - private void updateLoadFunction(AccordCache.Type i, FunctionWrapper wrapper) { i.unsafeSetLoadFunction(wrapper.wrap(i.unsafeGetLoadFunction())); From b149b54c5a3643a183a383a25d0b2e7f066cca3f Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 4 Dec 2024 11:50:49 +0100 Subject: [PATCH 05/10] Minor improvements --- modules/accord | 2 +- .../service/accord/AccordJournal.java | 158 +++++++++--------- 2 files changed, 84 insertions(+), 76 deletions(-) diff --git a/modules/accord b/modules/accord index 16d6cbe520e9..194af533c848 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit 16d6cbe520e9ffe0c3226fed178a819e3a97555d +Subproject commit 194af533c8484f9a05fcd0d4856f35317b7eb67c diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index dcec5d6b1a1e..eba04a030c89 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -518,6 +518,11 @@ public static Writer make(Command before, Command after) @Override public void write(DataOutputPlus out, int userVersion) throws IOException + { + serialize(after, flags, out, userVersion); + } + + private static void serialize(Command command, int flags, DataOutputPlus out, int userVersion) throws IOException { Invariants.checkState(flags != 0); out.writeInt(flags); @@ -535,45 +540,45 @@ public void write(DataOutputPlus out, int userVersion) throws IOException switch (field) { case EXECUTE_AT: - CommandSerializers.timestamp.serialize(after.executeAt(), out, userVersion); + CommandSerializers.timestamp.serialize(command.executeAt(), out, userVersion); break; case EXECUTES_AT_LEAST: - CommandSerializers.timestamp.serialize(after.executesAtLeast(), out, userVersion); + CommandSerializers.timestamp.serialize(command.executesAtLeast(), out, userVersion); break; case SAVE_STATUS: - out.writeShort(after.saveStatus().ordinal()); + out.writeShort(command.saveStatus().ordinal()); break; case DURABILITY: - out.writeByte(after.durability().ordinal()); + out.writeByte(command.durability().ordinal()); break; case ACCEPTED: - CommandSerializers.ballot.serialize(after.acceptedOrCommitted(), out, userVersion); + CommandSerializers.ballot.serialize(command.acceptedOrCommitted(), out, userVersion); break; case PROMISED: - CommandSerializers.ballot.serialize(after.promised(), out, userVersion); + CommandSerializers.ballot.serialize(command.promised(), out, userVersion); break; case PARTICIPANTS: - CommandSerializers.participants.serialize(after.participants(), out, userVersion); + CommandSerializers.participants.serialize(command.participants(), out, userVersion); break; case PARTIAL_TXN: - CommandSerializers.partialTxn.serialize(after.partialTxn(), out, userVersion); + CommandSerializers.partialTxn.serialize(command.partialTxn(), out, userVersion); break; case PARTIAL_DEPS: - DepsSerializers.partialDeps.serialize(after.partialDeps(), out, userVersion); + DepsSerializers.partialDeps.serialize(command.partialDeps(), out, userVersion); break; case WAITING_ON: - Command.WaitingOn waitingOn = getWaitingOn(after); - long size = WaitingOnSerializer.serializedSize(after.txnId(), waitingOn); - ByteBuffer serialized = WaitingOnSerializer.serialize(after.txnId(), waitingOn); + Command.WaitingOn waitingOn = getWaitingOn(command); + long size = WaitingOnSerializer.serializedSize(command.txnId(), waitingOn); + ByteBuffer serialized = WaitingOnSerializer.serialize(command.txnId(), waitingOn); Invariants.checkState(serialized.remaining() == size); out.writeInt((int) size); out.write(serialized); break; case WRITES: - CommandSerializers.writes.serialize(after.writes(), out, userVersion); + CommandSerializers.writes.serialize(command.writes(), out, userVersion); break; case RESULT: - ResultSerializers.result.serialize(after.result(), out, userVersion); + ResultSerializers.result.serialize(command.result(), out, userVersion); break; case CLEANUP: throw new IllegalStateException(); @@ -620,68 +625,71 @@ public void serialize(DataOutputPlus out, int userVersion) throws IOException { Invariants.checkState(mask == 0); Invariants.checkState(flags != 0); - out.writeInt(validateFlags(flags)); - - int iterable = toIterableSetFields(flags); - while (iterable != 0) - { - CommandChange.Fields field = nextSetField(iterable); - if (getFieldIsNull(field, flags)) - { - iterable = unsetIterableFields(field, iterable); - continue; - } - - switch (field) - { - case EXECUTE_AT: - CommandSerializers.timestamp.serialize(executeAt(), out, userVersion); - break; - case EXECUTES_AT_LEAST: - CommandSerializers.timestamp.serialize(executeAtLeast(), out, userVersion); - break; - case SAVE_STATUS: - out.writeShort(saveStatus().ordinal()); - break; - case DURABILITY: - out.writeByte(durability().ordinal()); - break; - case ACCEPTED: - CommandSerializers.ballot.serialize(acceptedOrCommitted(), out, userVersion); - break; - case PROMISED: - CommandSerializers.ballot.serialize(promised(), out, userVersion); - break; - case PARTICIPANTS: - CommandSerializers.participants.serialize(participants(), out, userVersion); - break; - case PARTIAL_TXN: - CommandSerializers.partialTxn.serialize(partialTxn(), out, userVersion); - break; - case PARTIAL_DEPS: - DepsSerializers.partialDeps.serialize(partialDeps(), out, userVersion); - break; - case WAITING_ON: - Command.WaitingOn waitingOn = waitingOn().provide(txnId, partialDeps); - long size = WaitingOnSerializer.serializedSize(txnId, waitingOn); - ByteBuffer serialized = WaitingOnSerializer.serialize(txnId, waitingOn); - Invariants.checkState(serialized.remaining() == size); - out.writeInt((int) size); - out.write(serialized); - break; - case WRITES: - CommandSerializers.writes.serialize(writes(), out, userVersion); - break; - case CLEANUP: - out.writeByte(cleanup.ordinal()); - break; - case RESULT: - ResultSerializers.result.serialize(result(), out, userVersion); - break; - } - iterable = unsetIterableFields(field, iterable); - } + int flags = validateFlags(this.flags); + Writer.serialize(construct(), flags, out, userVersion); +// out.writeInt(validateFlags(flags)); +// +// int iterable = toIterableSetFields(flags); +// while (iterable != 0) +// { +// CommandChange.Fields field = nextSetField(iterable); +// if (getFieldIsNull(field, flags)) +// { +// iterable = unsetIterableFields(field, iterable); +// continue; +// } +// +// switch (field) +// { +// case EXECUTE_AT: +// CommandSerializers.timestamp.serialize(executeAt(), out, userVersion); +// break; +// case EXECUTES_AT_LEAST: +// CommandSerializers.timestamp.serialize(executeAtLeast(), out, userVersion); +// break; +// case SAVE_STATUS: +// out.writeShort(saveStatus().ordinal()); +// break; +// case DURABILITY: +// out.writeByte(durability().ordinal()); +// break; +// case ACCEPTED: +// CommandSerializers.ballot.serialize(acceptedOrCommitted(), out, userVersion); +// break; +// case PROMISED: +// CommandSerializers.ballot.serialize(promised(), out, userVersion); +// break; +// case PARTICIPANTS: +// CommandSerializers.participants.serialize(participants(), out, userVersion); +// break; +// case PARTIAL_TXN: +// CommandSerializers.partialTxn.serialize(partialTxn(), out, userVersion); +// break; +// case PARTIAL_DEPS: +// DepsSerializers.partialDeps.serialize(partialDeps(), out, userVersion); +// break; +// case WAITING_ON: +// Command.WaitingOn waitingOn = waitingOn().provide(txnId, partialDeps); +// long size = WaitingOnSerializer.serializedSize(txnId, waitingOn); +// ByteBuffer serialized = WaitingOnSerializer.serialize(txnId, waitingOn); +// Invariants.checkState(serialized.remaining() == size); +// out.writeInt((int) size); +// out.write(serialized); +// break; +// case WRITES: +// CommandSerializers.writes.serialize(writes(), out, userVersion); +// break; +// case CLEANUP: +// out.writeByte(cleanup.ordinal()); +// break; +// case RESULT: +// ResultSerializers.result.serialize(result(), out, userVersion); +// break; +// } +// +// iterable = unsetIterableFields(field, iterable); +// } } public void deserializeNext(DataInputPlus in, int userVersion) throws IOException From 7fd644e48d30802031a262874e4d88ab76b6392b Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 4 Dec 2024 12:19:54 +0100 Subject: [PATCH 06/10] Allow saving commands on replay --- src/java/org/apache/cassandra/service/accord/AccordJournal.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index eba04a030c89..347ca63d7d04 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -269,7 +269,7 @@ public CommandStores.RangesForEpoch loadRangesForEpoch(int store) public void saveCommand(int store, CommandUpdate update, Runnable onFlush) { Writer diff = Writer.make(update.before, update.after); - if (diff == null || status == Status.REPLAY) + if (diff == null) { if (onFlush != null) onFlush.run(); From 7af90d4e257052dc7667eaba8cc98b884e078104 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 4 Dec 2024 12:23:21 +0100 Subject: [PATCH 07/10] Fix AccordJournalCompactionTest --- .../cassandra/service/accord/AccordJournalValueSerializers.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java index b313e7e924e7..cee74aba3dad 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java @@ -320,8 +320,8 @@ public void deserialize(JournalKey key, IdentityAccumulator into long[] epochs = new long[size]; for (int i = 0; i < ranges.length; i++) { - ranges[i] = KeySerializers.ranges.deserialize(in, messagingVersion); epochs[i] = in.readLong(); + ranges[i] = KeySerializers.ranges.deserialize(in, messagingVersion); } Invariants.checkState(ranges.length == epochs.length); into.update(new RangesForEpoch(epochs, ranges)); From 8fdcbe2e10eaf8b10dabb6e9a4bd0d4c595f008e Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Wed, 4 Dec 2024 12:25:14 +0100 Subject: [PATCH 08/10] Persist durable before during replay, too --- .../org/apache/cassandra/service/accord/AccordJournal.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index 347ca63d7d04..f4d87e5bde1f 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -290,9 +290,6 @@ public PersistentField.Persister durableBeforePers @Override public AsyncResult persist(DurableBefore addDurableBefore, DurableBefore newDurableBefore) { - if (status == Status.REPLAY) - return AsyncResults.success(null); - AsyncResult.Settable result = AsyncResults.settable(); JournalKey key = new JournalKey(TxnId.NONE, JournalKey.Type.DURABLE_BEFORE, 0); RecordPointer pointer = appendInternal(key, addDurableBefore); From a1e2c615b126c6d7444afb934868a36a03c07bec Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Thu, 5 Dec 2024 13:51:58 +0100 Subject: [PATCH 09/10] Fix topology replay during bootstrap and startup, decouple Accord from TCM Includes multiple changes, primary ones: * Removed nodes now live in TCM, no need to discover historic epochs in order to find removed nodes * CommandStore <-> RangesForEpochs mappings required for startup are now stored in journal, and CS can be set up _without_ topology replay * Topology replay is fully done via journal (where we store topologies themselves), and topology metadata table (where we store redundant/closed information) * Fixed various bugs related to propagation and staleness * TCM was previously relied on for "fetching" epoch: we can not rely on it as there's no guarantee we will see a consecutive epoch when grabbing Metadata#current * Redundant / closed during replay was set with incorrect ranges in 1 of the code paths * TCM was contacted multiple times for historical epochs, which made startup much longer under some circumstances --- .gitmodules | 2 +- modules/accord | 2 +- .../apache/cassandra/config/AccordSpec.java | 1 + .../cassandra/exceptions/RequestFailure.java | 3 + .../exceptions/RequestFailureReason.java | 3 +- .../locator/CMSPlacementStrategy.java | 2 +- .../apache/cassandra/net/MessageDelivery.java | 43 ++++- src/java/org/apache/cassandra/net/Verb.java | 4 + .../cassandra/repair/SharedContext.java | 1 + .../service/accord/AccordCommandStore.java | 2 +- .../service/accord/AccordCommandStores.java | 14 +- .../accord/AccordConfigurationService.java | 171 +++++++++--------- .../service/accord/AccordJournal.java | 16 ++ .../accord/AccordJournalValueSerializers.java | 103 ++++++++++- .../service/accord/AccordKeyspace.java | 22 +-- .../accord/AccordSafeCommandStore.java | 4 +- .../service/accord/AccordService.java | 170 +++++++---------- .../service/accord/AccordSyncPropagator.java | 31 ++-- .../service/accord/AccordTopology.java | 23 ++- .../service/accord/EndpointMapping.java | 27 ++- .../service/accord/FetchMinEpoch.java | 44 +++-- .../service/accord/FetchTopology.java | 114 ++++++++++++ .../cassandra/service/accord/JournalKey.java | 2 + .../apache/cassandra/tcm/ClusterMetadata.java | 4 +- .../org/apache/cassandra/tcm/Startup.java | 11 +- .../cassandra/tcm/membership/Directory.java | 114 +++++++++++- .../cassandra/tcm/serialization/Version.java | 7 + .../org/apache/cassandra/utils/HeapUtils.java | 2 +- .../distributed/shared/ClusterUtils.java | 2 +- .../fuzz/topology/AccordBootstrapTest.java | 161 +++++++++++++++++ .../apache/cassandra/harry/SchemaSpec.java | 5 + .../harry/execution/CQLVisitExecutor.java | 9 +- .../execution/InJvmDTestVisitExecutor.java | 7 +- .../cassandra/harry/gen/SchemaGenerators.java | 8 +- .../org/apache/cassandra/harry/op/Visit.java | 6 + .../cassandra/net/MessageDeliveryTest.java | 20 +- .../AccordConfigurationServiceTest.java | 48 ++--- .../service/accord/AccordTestUtils.java | 28 --- .../service/accord/EpochSyncTest.java | 9 +- 39 files changed, 883 insertions(+), 362 deletions(-) create mode 100644 src/java/org/apache/cassandra/service/accord/FetchTopology.java create mode 100644 test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java diff --git a/.gitmodules b/.gitmodules index 1500851ee94c..d06ec1ce80d0 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,4 +1,4 @@ [submodule "modules/accord"] path = modules/accord url = https://github.com/ifesdjeen/cassandra-accord.git - branch = CASSANDRA-20115 + branch = topology-rebuild-on-bounce diff --git a/modules/accord b/modules/accord index 194af533c848..aca7033a33f2 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit 194af533c8484f9a05fcd0d4856f35317b7eb67c +Subproject commit aca7033a33f2a638077e52d5ae95497bca253cd1 diff --git a/src/java/org/apache/cassandra/config/AccordSpec.java b/src/java/org/apache/cassandra/config/AccordSpec.java index f062c9a5703f..d550320727d9 100644 --- a/src/java/org/apache/cassandra/config/AccordSpec.java +++ b/src/java/org/apache/cassandra/config/AccordSpec.java @@ -194,6 +194,7 @@ public enum TransactionalRangeMigration public boolean ephemeralReadEnabled = true; public boolean state_cache_listener_jfr_enabled = true; public final JournalSpec journal = new JournalSpec(); + // TODO: do we need more retries; rename public final MinEpochRetrySpec minEpochSyncRetry = new MinEpochRetrySpec(); public static class MinEpochRetrySpec extends RetrySpec diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailure.java b/src/java/org/apache/cassandra/exceptions/RequestFailure.java index 2b2282731a87..7da818d3e5c9 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailure.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailure.java @@ -54,6 +54,7 @@ public class RequestFailure public static final RequestFailure COORDINATOR_BEHIND = new RequestFailure(RequestFailureReason.COORDINATOR_BEHIND); public static final RequestFailure READ_TOO_MANY_INDEXES = new RequestFailure(RequestFailureReason.READ_TOO_MANY_INDEXES); public static final RequestFailure RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM = new RequestFailure(RequestFailureReason.RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM); + public static final RequestFailure BOOTING = new RequestFailure(RequestFailureReason.RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM); static { @@ -131,6 +132,7 @@ public static RequestFailure forReason(RequestFailureReason reason) { switch (reason) { + // TODO: this really screws things up default: throw new IllegalStateException("Unhandled request failure reason " + reason); case UNKNOWN: return UNKNOWN; case READ_TOO_MANY_TOMBSTONES: return READ_TOO_MANY_TOMBSTONES; @@ -144,6 +146,7 @@ public static RequestFailure forReason(RequestFailureReason reason) case COORDINATOR_BEHIND: return COORDINATOR_BEHIND; case READ_TOO_MANY_INDEXES: return READ_TOO_MANY_INDEXES; case RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM: return RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM; + case BOOTING: return BOOTING; } } diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java index 77fc06ea18e6..b3b9bddfc448 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailureReason.java @@ -39,8 +39,9 @@ public enum RequestFailureReason NOT_CMS (7), INVALID_ROUTING (8), COORDINATOR_BEHIND (9), - READ_TOO_MANY_INDEXES (10), + READ_TOO_MANY_INDEXES (10), RETRY_ON_DIFFERENT_TRANSACTION_SYSTEM (11), + BOOTING (12), ; static diff --git a/src/java/org/apache/cassandra/locator/CMSPlacementStrategy.java b/src/java/org/apache/cassandra/locator/CMSPlacementStrategy.java index 6ba47ae04d77..d0d038200052 100644 --- a/src/java/org/apache/cassandra/locator/CMSPlacementStrategy.java +++ b/src/java/org/apache/cassandra/locator/CMSPlacementStrategy.java @@ -117,7 +117,7 @@ public Set reconfigure(ClusterMetadata metadata, Map Future> sendWithRetries(Backoff backoff, return promise; } + public default Future> sendWithRetries(SharedContext sharedContext, Verb verb, REQ request, Collection peers) + { + // TODO: move somewhere else + Iterator candidates = new Iterator<>() + { + boolean firstRun = true; + Iterator iter = peers.iterator(); + + public boolean hasNext() + { + return !peers.isEmpty(); + } + + public InetAddressAndPort next() + { + // At first, try all alive nodes + if (firstRun) + { + while (iter.hasNext()) + { + InetAddressAndPort candidate = iter.next(); + if (sharedContext.failureDetector().isAlive(candidate)) + return candidate; + } + firstRun = false; + } + + // After that, cycle through all nodes + if (!iter.hasNext()) + iter = peers.iterator(); + + return iter.next(); + } + }; + return sendWithRetries(Backoff.None.INSTANCE, ImmediateRetryScheduler.instance, verb, request, candidates, RetryPredicate.ALWAYS_RETRY, RetryErrorMessage.EMPTY); + } + public default void sendWithRetries(Backoff backoff, RetryScheduler retryThreads, Verb verb, REQ request, Iterator candidates, @@ -127,11 +164,15 @@ interface OnResult interface RetryPredicate { + public static RetryPredicate times(int n) { return (attempt, from, failure) -> attempt < n; } + RetryPredicate ALWAYS_RETRY = (i1, i2, i3) -> true; + RetryPredicate ALWAYS_REJECT = (i1, i2, i3) -> false; boolean test(int attempt, InetAddressAndPort from, RequestFailure failure); } interface RetryErrorMessage { + RetryErrorMessage EMPTY = (i1, i2, i3, i4) -> null; String apply(int attempt, ResponseFailureReason retryFailure, @Nullable InetAddressAndPort from, @Nullable RequestFailure reason); } diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index 732f0e5df0e4..30d1315a6276 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -81,6 +81,7 @@ import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.AccordSyncPropagator; import org.apache.cassandra.service.accord.AccordSyncPropagator.Notification; +import org.apache.cassandra.service.accord.FetchTopology; import org.apache.cassandra.service.accord.FetchMinEpoch; import org.apache.cassandra.service.accord.interop.AccordInteropApply; import org.apache.cassandra.service.accord.interop.AccordInteropCommit; @@ -356,8 +357,11 @@ public enum Verb ACCORD_INTEROP_READ_REPAIR_RSP (158, P2, writeTimeout, IMMEDIATE, () -> AccordInteropReadRepair.replySerializer, AccordService::responseHandlerOrNoop), ACCORD_INTEROP_READ_REPAIR_REQ (159, P2, writeTimeout, IMMEDIATE, () -> AccordInteropReadRepair.requestSerializer, AccordService::requestHandlerOrNoop, ACCORD_INTEROP_READ_REPAIR_RSP), ACCORD_INTEROP_APPLY_REQ (160, P2, writeTimeout, IMMEDIATE, () -> AccordInteropApply.serializer, AccordService::requestHandlerOrNoop, ACCORD_APPLY_RSP), + // TODO: swap IDS? ACCORD_FETCH_MIN_EPOCH_RSP (166, P2, writeTimeout, IMMEDIATE, () -> FetchMinEpoch.Response.serializer, RESPONSE_HANDLER), ACCORD_FETCH_MIN_EPOCH_REQ (165, P2, writeTimeout, IMMEDIATE, () -> FetchMinEpoch.serializer, () -> FetchMinEpoch.handler, ACCORD_FETCH_MIN_EPOCH_RSP), + ACCORD_FETCH_TOPOLOGY_RSP (167, P2, writeTimeout, IMMEDIATE, () -> FetchTopology.Response.serializer, RESPONSE_HANDLER), + ACCORD_FETCH_TOPOLOGY_REQ (168, P2, writeTimeout, IMMEDIATE, () -> FetchTopology.serializer, () -> FetchTopology.handler, ACCORD_FETCH_TOPOLOGY_RSP), // generic failure response FAILURE_RSP (99, P0, noTimeout, REQUEST_RESPONSE, () -> RequestFailure.serializer, RESPONSE_HANDLER ), diff --git a/src/java/org/apache/cassandra/repair/SharedContext.java b/src/java/org/apache/cassandra/repair/SharedContext.java index f9ca3c23600e..56b6489f7afb 100644 --- a/src/java/org/apache/cassandra/repair/SharedContext.java +++ b/src/java/org/apache/cassandra/repair/SharedContext.java @@ -51,6 +51,7 @@ * * See {@link Global#instance} for the main production path */ +// TODO: move to util or something? public interface SharedContext { InetAddressAndPort broadcastAddressAndPort(); diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java index c9df8b49b9cf..6b7c16dddd5f 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStore.java @@ -203,7 +203,7 @@ public AccordCommandStore(int id, loadRangesForEpoch(journal.loadRangesForEpoch(id())); } - static Factory factory(AccordJournal journal, IntFunction executorFactory) + static Factory factory(Journal journal, IntFunction executorFactory) { return (id, node, agent, dataStore, progressLogFactory, listenerFactory, rangesForEpoch) -> new AccordCommandStore(id, node, agent, dataStore, progressLogFactory, listenerFactory, rangesForEpoch, journal, executorFactory.apply(id)); diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java index 0a530bf4d3cd..718862877245 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java @@ -26,6 +26,7 @@ import accord.api.Agent; import accord.api.DataStore; +import accord.api.Journal; import accord.api.LocalListeners; import accord.api.ProgressLog; import accord.local.CommandStores; @@ -47,8 +48,8 @@ import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; import static org.apache.cassandra.config.AccordSpec.QueueShardModel.THREAD_PER_SHARD; -import static org.apache.cassandra.config.DatabaseDescriptor.getAccordQueueSubmissionModel; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordQueueShardCount; +import static org.apache.cassandra.config.DatabaseDescriptor.getAccordQueueSubmissionModel; import static org.apache.cassandra.service.accord.AccordExecutor.Mode.RUN_WITHOUT_LOCK; import static org.apache.cassandra.service.accord.AccordExecutor.Mode.RUN_WITH_LOCK; import static org.apache.cassandra.service.accord.AccordExecutor.constant; @@ -60,15 +61,16 @@ public class AccordCommandStores extends CommandStores implements CacheSize private final CacheSizeMetrics cacheSizeMetrics; private final AccordExecutor[] executors; + private long cacheSize, workingSetSize; private int maxQueuedLoads, maxQueuedRangeLoads; private boolean shrinkingOn; AccordCommandStores(NodeCommandStoreService node, Agent agent, DataStore store, RandomSource random, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenerFactory, - AccordJournal journal, AccordExecutor[] executors) + Journal journal, AccordExecutor[] executors) { - super(node, agent, store, random, shardDistributor, progressLogFactory, listenerFactory, + super(node, agent, store, random, journal, shardDistributor, progressLogFactory, listenerFactory, AccordCommandStore.factory(journal, id -> executors[id % executors.length])); this.executors = executors; this.cacheSizeMetrics = new CacheSizeMetrics(ACCORD_STATE_CACHE, this); @@ -80,9 +82,9 @@ public class AccordCommandStores extends CommandStores implements CacheSize refreshCapacities(); } - static Factory factory(AccordJournal journal) + static Factory factory() { - return (time, agent, store, random, shardDistributor, progressLogFactory, listenerFactory) -> { + return (NodeCommandStoreService time, Agent agent, DataStore store, RandomSource random, Journal journal, ShardDistributor shardDistributor, ProgressLog.Factory progressLogFactory, LocalListeners.Factory listenersFactory) -> { AccordExecutor[] executors = new AccordExecutor[getAccordQueueShardCount()]; AccordExecutorFactory factory; int maxThreads = Integer.MAX_VALUE; @@ -119,7 +121,7 @@ static Factory factory(AccordJournal journal) } } - return new AccordCommandStores(time, agent, store, random, shardDistributor, progressLogFactory, listenerFactory, journal, executors); + return new AccordCommandStores(time, agent, store, random, shardDistributor, progressLogFactory, listenersFactory, journal, executors); }; } diff --git a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java index 42ac0325a8a3..4678adfb0d52 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java @@ -18,12 +18,13 @@ package org.apache.cassandra.service.accord; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; import java.util.Objects; -import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; import java.util.stream.Collectors; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -31,6 +32,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import accord.api.Journal; import accord.impl.AbstractConfigurationService; import accord.local.Node; import accord.primitives.Ranges; @@ -39,6 +41,7 @@ import accord.utils.Invariants; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; +import org.agrona.collections.Long2ObjectHashMap; import org.agrona.collections.LongArrayList; import org.apache.cassandra.concurrent.ScheduledExecutorPlus; import org.apache.cassandra.concurrent.ScheduledExecutors; @@ -49,25 +52,26 @@ import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.service.accord.AccordKeyspace.EpochDiskState; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; -import org.apache.cassandra.tcm.Epoch; import org.apache.cassandra.tcm.listeners.ChangeListener; -import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.Simulate; import org.apache.cassandra.utils.concurrent.AsyncPromise; import org.apache.cassandra.utils.concurrent.Future; +import org.apache.cassandra.utils.concurrent.UncheckedInterruptedException; import static org.apache.cassandra.utils.Simulate.With.MONITORS; // TODO: listen to FailureDetector and rearrange fast path accordingly @Simulate(with=MONITORS) -public class AccordConfigurationService extends AbstractConfigurationService implements ChangeListener, AccordEndpointMapper, AccordSyncPropagator.Listener, Shutdownable +public class AccordConfigurationService extends AbstractConfigurationService implements AccordEndpointMapper, AccordSyncPropagator.Listener, Shutdownable { private final AccordSyncPropagator syncPropagator; private final DiskStateManager diskStateManager; + private final Journal journal; @GuardedBy("this") private EpochDiskState diskState = EpochDiskState.EMPTY; @@ -131,6 +135,7 @@ protected EpochState createEpochState(long epoch) interface DiskStateManager { EpochDiskState loadTopologies(AccordKeyspace.TopologyLoadConsumer consumer); + EpochDiskState setNotifyingLocalSync(long epoch, Set pending, EpochDiskState diskState); EpochDiskState setCompletedLocalSync(long epoch, EpochDiskState diskState); @@ -199,16 +204,27 @@ public EpochDiskState truncateTopologyUntil(long epoch, EpochDiskState diskState } } - public AccordConfigurationService(Node.Id node, MessageDelivery messagingService, IFailureDetector failureDetector, DiskStateManager diskStateManager, ScheduledExecutorPlus scheduledTasks) + private final MetadataChangeListener listener = new MetadataChangeListener(); + private class MetadataChangeListener implements ChangeListener + { + @Override + public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean fromSnapshot) + { + maybeReportMetadata(next); + } + } + + public AccordConfigurationService(Node.Id node, MessageDelivery messagingService, IFailureDetector failureDetector, DiskStateManager diskStateManager, ScheduledExecutorPlus scheduledTasks, Journal journal) { super(node); this.syncPropagator = new AccordSyncPropagator(localId, this, messagingService, failureDetector, scheduledTasks, this); this.diskStateManager = diskStateManager; + this.journal = journal; } - public AccordConfigurationService(Node.Id node) + public AccordConfigurationService(Node.Id node, Journal journal) { - this(node, MessagingService.instance(), FailureDetector.instance, SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + this(node, MessagingService.instance(), FailureDetector.instance, SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, journal); } @Override @@ -217,48 +233,51 @@ protected EpochHistory createEpochHistory() return new EpochHistory(); } - @VisibleForTesting + /** + * Re + */ public synchronized void start() - { - start(ignore -> {}); - } - - public synchronized void start(Consumer callback) { Invariants.checkState(state == State.INITIALIZED, "Expected state to be INITIALIZED but was %s", state); state = State.LOADING; - EndpointMapping snapshot = mapping; - //TODO (restart): if there are topologies loaded then there is likely failures if reporting is needed, as mapping is not setup yet - AtomicReference previousRef = new AtomicReference<>(null); - diskState = diskStateManager.loadTopologies(((epoch, metadata, topology, syncStatus, pendingSyncNotify, remoteSyncComplete, closed, redundant) -> { - updateMapping(metadata); - reportTopology(topology, syncStatus == SyncStatus.NOT_STARTED, true); - Topology previous = previousRef.get(); - if (previous != null) - { - // for all nodes removed, or pending removal, mark them as removed so we don't wait on their replies - Sets.SetView removedNodes = Sets.difference(previous.nodes(), topology.nodes()); - if (!removedNodes.isEmpty()) - onNodesRemoved(topology.epoch(), currentTopology(), removedNodes); - } - previousRef.set(topology); + Iterator iter = journal.replayTopologies(); + // Load all active topologies. No-op on bootstrap. Since we are loading them _after_ restoring shard state, there should be no back-writes to journal. + Long2ObjectHashMap topologies = new Long2ObjectHashMap<>(); + while (iter.hasNext()) + { + Journal.TopologyUpdate update = iter.next(); + topologies.put(update.global.epoch(), update.global); + } + EndpointMapping snapshot = mapping; + diskStateManager.loadTopologies((epoch, syncStatus, pendingSyncNotify, remoteSyncComplete, closed, redundant) -> { + Topology topology = Invariants.nonNull(topologies.get(epoch)); + reportTopology(topology, syncStatus == SyncStatus.NOT_STARTED, true); getOrCreateEpochState(epoch).setSyncStatus(syncStatus); - if (syncStatus == SyncStatus.NOTIFYING) + switch (syncStatus) { - // TODO (expected, correctness): since this is loading old topologies, might see nodes no longer present (host replacement, decom, shrink, etc.); attempt to remove unknown nodes - syncPropagator.reportSyncComplete(epoch, Sets.filter(pendingSyncNotify, snapshot::containsId), localId); + case NOTIFYING: + // TODO (expected, correctness): since this is loading old topologies, might see nodes no longer present (host replacement, decom, shrink, etc.); attempt to remove unknown nodes + syncPropagator.reportSyncComplete(epoch, Sets.filter(pendingSyncNotify, snapshot::containsId), localId); + break; + case COMPLETED: + break; } remoteSyncComplete.forEach(id -> receiveRemoteSyncComplete(id, epoch)); // TODO (required): disk doesn't get updated until we see our own notification, so there is an edge case where this instance notified others and fails in the middle, but Apply was already sent! This could leave partial closed/redudant accross the cluster receiveClosed(closed, epoch); receiveRedundant(redundant, epoch); - })); + }); state = State.STARTED; - callback.accept(diskState.isEmpty() ? OptionalLong.empty() : OptionalLong.of(diskState.maxEpoch)); - ClusterMetadataService.instance().log().addListener(this); + + // for all nodes removed, or pending removal, mark them as removed so we don't wait on their replies + Map removedNodes = mapping.removedNodes(); + for (Map.Entry e : removedNodes.entrySet()) + onNodeRemoved(e.getValue(), currentTopology(), e.getKey()); + + ClusterMetadataService.instance().log().addListener(listener); } @Override @@ -272,7 +291,7 @@ public synchronized void shutdown() { if (isTerminated()) return; - ClusterMetadataService.instance().log().removeListener(this); + ClusterMetadataService.instance().log().removeListener(listener); state = State.SHUTDOWN; } @@ -314,9 +333,9 @@ synchronized void updateMapping(EndpointMapping mapping) this.mapping = mapping; } - synchronized void updateMapping(ClusterMetadata metadata) + public synchronized void updateMapping(ClusterMetadata metadata) { - updateMapping(AccordTopology.directoryToMapping(mapping, metadata.epoch.getEpoch(), metadata.directory)); + updateMapping(AccordTopology.directoryToMapping(metadata.epoch.getEpoch(), metadata.directory)); } private void reportMetadata(ClusterMetadata metadata) @@ -325,11 +344,6 @@ private void reportMetadata(ClusterMetadata metadata) } void reportMetadataInternal(ClusterMetadata metadata) - { - reportMetadataInternal(metadata, false); - } - - void reportMetadataInternal(ClusterMetadata metadata, boolean isLoad) { updateMapping(metadata); Topology topology = AccordTopology.createAccordTopology(metadata); @@ -338,7 +352,7 @@ void reportMetadataInternal(ClusterMetadata metadata, boolean isLoad) for (Node.Id node : topology.nodes()) { if (mapping.mappedEndpointOrNull(node) == null) - throw new IllegalStateException("Epoch " + topology.epoch() + " has node " + node + " but mapping does not!"); + throw new IllegalStateException(String.format("Epoch %d has node %s but mapping does not!", topology.epoch(), node)); } } reportTopology(topology); @@ -352,8 +366,12 @@ private void checkIfNodesRemoved(Topology topology) Topology previous = getTopologyForEpoch(topology.epoch() - 1); // for all nodes removed, or pending removal, mark them as removed so we don't wait on their replies Sets.SetView removedNodes = Sets.difference(previous.nodes(), topology.nodes()); - if (!removedNodes.isEmpty()) - onNodesRemoved(topology.epoch(), previous, removedNodes); + // TODO: only notify for active epochs? + for (Node.Id removedNode : removedNodes) + { + if (topology.epoch() >= epochs.minEpoch()) + onNodeRemoved(topology.epoch(), previous, removedNode); + } } private static boolean shareShard(Topology current, Node.Id target, Node.Id self) @@ -366,22 +384,16 @@ private static boolean shareShard(Topology current, Node.Id target, Node.Id self return false; } - public void onNodesRemoved(long epoch, Topology current, Set removed) + public void onNodeRemoved(long epoch, Topology current, Node.Id removed) { - if (removed.isEmpty()) return; syncPropagator.onNodesRemoved(removed); for (long oldEpoch : nonCompletedEpochsBefore(epoch)) - { - for (Node.Id node : removed) - receiveRemoteSyncCompletePreListenerNotify(node, oldEpoch); - } - listeners.forEach(l -> l.onRemoveNodes(epoch, removed)); + receiveRemoteSyncCompletePreListenerNotify(removed, oldEpoch); - for (Node.Id node : removed) - { - if (shareShard(current, node, localId)) - AccordService.instance().tryMarkRemoved(current, node); - } + listeners.forEach(l -> l.onRemoveNode(epoch, removed)); + + if (shareShard(current, removed, localId)) + AccordService.instance().tryMarkRemoved(current, removed); } private long[] nonCompletedEpochsBefore(long max) @@ -416,35 +428,25 @@ void maybeReportMetadata(ClusterMetadata metadata) getOrCreateEpochState(epoch - 1).acknowledged().addCallback(() -> reportMetadata(metadata)); } - @Override - public void notifyPostCommit(ClusterMetadata prev, ClusterMetadata next, boolean fromSnapshot) - { - maybeReportMetadata(next); - } - @Override protected void fetchTopologyInternal(long epoch) { - ClusterMetadata metadata = ClusterMetadata.current(); - if (metadata.directory.peerIds().isEmpty()) - return; // just let CMS handle it when it's ready - - // TODO (desired): randomise - NodeId first = metadata.directory.peerIds().first(); - InetAddressAndPort peer = metadata.directory.getNodeAddresses(first).broadcastAddress; - if (FBUtilities.getBroadcastAddressAndPort().equals(peer)) + try { - NodeId second = metadata.directory.peerIds().higher(first); - if (second == null) - return; - - peer = metadata.directory.getNodeAddresses(second).broadcastAddress; + Set peers = new HashSet<>(ClusterMetadata.current().directory.allJoinedEndpoints()); + peers.remove(FBUtilities.getBroadcastAddressAndPort()); + Topology topology = FetchTopology.fetch(SharedContext.Global.instance, peers, epoch).get(); + reportTopology(topology); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e); + } + catch (ExecutionException e) + { + throw new RuntimeException(e.getCause()); } - ClusterMetadataService.instance().fetchLogFromPeerOrCMSAsync(metadata, peer, Epoch.create(epoch)) - .addCallback((success, fail) -> { - if (fail != null) - fetchTopologyInternal(epoch); - }); } @Override @@ -569,7 +571,7 @@ public enum ResultStatus { PENDING, SUCCESS, FAILURE; - static ResultStatus of (AsyncResult result) + static ResultStatus of(AsyncResult result) { if (result == null || !result.isDone()) return PENDING; @@ -680,3 +682,4 @@ public Future unsafeLocalSyncNotified(long epoch) return promise; } } +//SubstituteLogger.java:169 - DEBUG [node5_Messaging-EventLoop-3-1] node5 2024-12-04 20:15:07,842 HeapUtils.java:133 - Heap dump creation on uncaught exceptions is disabled. \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index f4d87e5bde1f..baada251c20c 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.Iterator; import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -282,6 +283,21 @@ public void saveCommand(int store, CommandUpdate update, Runnable onFlush) journal.onDurable(pointer, onFlush); } + @Override + public Iterator replayTopologies() + { + AccordJournalValueSerializers.ListAccumulator accumulator = readAll(new JournalKey(TxnId.NONE, JournalKey.Type.TOPOLOGY_UPDATE, -1)); + return accumulator.get().iterator(); + } + + @Override + public void saveTopology(TopologyUpdate topologyUpdate, Runnable onFlush) + { + RecordPointer pointer = appendInternal(new JournalKey(TxnId.NONE, JournalKey.Type.TOPOLOGY_UPDATE, -1), topologyUpdate); + if (onFlush != null) + journal.onDurable(pointer, onFlush); + } + @Override public PersistentField.Persister durableBeforePersister() { diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java index cee74aba3dad..6a3d882ba71a 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalValueSerializers.java @@ -19,21 +19,28 @@ package org.apache.cassandra.service.accord; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import java.util.NavigableMap; import com.google.common.collect.ImmutableSortedMap; +import accord.api.Journal; import accord.local.DurableBefore; import accord.local.RedundantBefore; import accord.primitives.Ranges; import accord.primitives.Timestamp; import accord.primitives.TxnId; +import accord.topology.Topology; import accord.utils.Invariants; +import org.agrona.collections.Int2ObjectHashMap; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.service.accord.serializers.CommandStoreSerializers; import org.apache.cassandra.service.accord.serializers.KeySerializers; +import org.apache.cassandra.service.accord.serializers.TopologySerializers; import static accord.api.Journal.Load.ALL; import static accord.local.CommandStores.RangesForEpoch; @@ -190,7 +197,8 @@ protected DurableBefore accumulate(DurableBefore oldValue, DurableBefore newValu } } - public static class DurableBeforeSerializer implements FlyweightSerializer + public static class DurableBeforeSerializer + implements FlyweightSerializer { public DurableBeforeAccumulator mergerFor(JournalKey journalKey) { @@ -253,7 +261,8 @@ public void deserialize(JournalKey key, IdentityAccumulator, IdentityAccumulator>> + public static class SafeToReadSerializer + implements FlyweightSerializer, IdentityAccumulator>> { @Override public IdentityAccumulator> mergerFor(JournalKey key) @@ -281,9 +290,8 @@ public void deserialize(JournalKey key, IdentityAccumulator> + implements FlyweightSerializer> { - public IdentityAccumulator mergerFor(JournalKey key) { return new IdentityAccumulator<>(null); @@ -307,13 +315,13 @@ public void serialize(JournalKey key, RangesForEpoch from, DataOutputPlus out, i } @Override - public void reserialize(JournalKey key, IdentityAccumulator from, DataOutputPlus out, int userVersion) throws IOException + public void reserialize(JournalKey key, Accumulator from, DataOutputPlus out, int userVersion) throws IOException { serialize(key, from.get(), out, messagingVersion); } @Override - public void deserialize(JournalKey key, IdentityAccumulator into, DataInputPlus in, int userVersion) throws IOException + public void deserialize(JournalKey key, Accumulator into, DataInputPlus in, int userVersion) throws IOException { int size = in.readUnsignedVInt32(); Ranges[] ranges = new Ranges[size]; @@ -327,4 +335,87 @@ public void deserialize(JournalKey key, IdentityAccumulator into into.update(new RangesForEpoch(epochs, ranges)); } } + + public static class ListAccumulator extends Accumulator, T> + { + public ListAccumulator() + { + super(new ArrayList<>()); + } + + @Override + protected List accumulate(List oldValue, T newValue) + { + oldValue.add(newValue); + return oldValue; + } + } + + public static class TopologyUpdateSerializer + implements FlyweightSerializer> + { + private final RangesForEpochSerializer rangesForEpochSerializer = new RangesForEpochSerializer(); + @Override + public ListAccumulator mergerFor(JournalKey key) + { + return new ListAccumulator<>(); + } + + @Override + public void serialize(JournalKey key, Journal.TopologyUpdate from, DataOutputPlus out, int userVersion) throws IOException + { + out.writeInt(1); + serializeOne(key, from, out, userVersion); + } + + private void serializeOne(JournalKey key, Journal.TopologyUpdate from, DataOutputPlus out, int userVersion) throws IOException + { + out.writeInt(from.commandStores.size()); + for (Map.Entry e : from.commandStores.entrySet()) + { + out.writeInt(e.getKey()); + rangesForEpochSerializer.serialize(key, e.getValue(), out, userVersion); + } + TopologySerializers.topology.serialize(from.local, out, userVersion); + TopologySerializers.topology.serialize(from.global, out, userVersion); + } + + @Override + public void reserialize(JournalKey key, ListAccumulator from, DataOutputPlus out, int userVersion) throws IOException + { + out.writeInt(from.accumulated.size()); + for (Journal.TopologyUpdate update : from.accumulated) + serializeOne(key, update, out, userVersion); + } + + @Override + public void deserialize(JournalKey key, ListAccumulator into, DataInputPlus in, int userVersion) throws IOException + { + int size = in.readInt(); + for (int i = 0; i < size; i++) + { + int commandStoresSize = in.readInt(); + Accumulator acc = new Accumulator<>(null) + { + @Override + protected RangesForEpoch accumulate(RangesForEpoch oldValue, RangesForEpoch newValue) + { + return this.accumulated = newValue; + } + }; + + Int2ObjectHashMap commandStores = new Int2ObjectHashMap<>(); + for (int j = 0; j < commandStoresSize; j++) + { + int commandStoreId = in.readInt(); + rangesForEpochSerializer.deserialize(key, acc, in, userVersion); + commandStores.put(commandStoreId, acc.accumulated); + acc.update(null); + } + Topology local = TopologySerializers.topology.deserialize(in, userVersion); + Topology global = TopologySerializers.topology.deserialize(in, userVersion); + into.update(new Journal.TopologyUpdate(commandStores, local, global)); + } + } + } } \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java index 82ef6f8519fd..3211136d0845 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java +++ b/src/java/org/apache/cassandra/service/accord/AccordKeyspace.java @@ -139,7 +139,6 @@ import org.apache.cassandra.service.accord.serializers.CommandSerializers; import org.apache.cassandra.service.accord.serializers.CommandsForKeySerializer; import org.apache.cassandra.service.accord.serializers.KeySerializers; -import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.utils.Clock.Global; import org.apache.cassandra.utils.CloseableIterator; import org.apache.cassandra.utils.btree.BTree; @@ -1498,6 +1497,7 @@ public static EpochDiskState loadEpochDiskState() * fails, we can detect and cleanup. If we updated disk state after an update and it failed, we could "forget" * about (now acked) topology updates after a restart. */ + // Should this one be synchronized? private static EpochDiskState maybeUpdateMaxEpoch(EpochDiskState diskState, long epoch) { if (diskState.isEmpty()) @@ -1596,22 +1596,18 @@ public static EpochDiskState truncateTopologyUntil(final long epoch, EpochDiskSt public interface TopologyLoadConsumer { - void load(long epoch, ClusterMetadata metadata, Topology topology, SyncStatus syncStatus, Set pendingSyncNotify, Set remoteSyncComplete, Ranges closed, Ranges redundant); + void load(long epoch, SyncStatus syncStatus, Set pendingSyncNotify, Set remoteSyncComplete, Ranges closed, Ranges redundant); } @VisibleForTesting - public static void loadEpoch(long epoch, ClusterMetadata metadata, TopologyLoadConsumer consumer) throws IOException + public static void loadEpoch(long epoch, TopologyLoadConsumer consumer) throws IOException { - Topology topology = AccordTopology.createAccordTopology(metadata); - - String cql = "SELECT * FROM " + ACCORD_KEYSPACE_NAME + '.' + TOPOLOGIES + ' ' + - "WHERE epoch=?"; - - UntypedResultSet result = executeInternal(cql, epoch); + UntypedResultSet result = executeInternal(String.format("SELECT * FROM %s.%s WHERE epoch=?", ACCORD_KEYSPACE_NAME, TOPOLOGIES), epoch); if (result.isEmpty()) { + // TODO: check if we still need this // topology updates disk state for epoch but doesn't save the topology to the table, so there maybe an epoch we know about, but no fields are present - consumer.load(epoch, metadata, topology, SyncStatus.NOT_STARTED, Collections.emptySet(), Collections.emptySet(), Ranges.EMPTY, Ranges.EMPTY); + consumer.load(epoch, SyncStatus.NOT_STARTED, Collections.emptySet(), Collections.emptySet(), Ranges.EMPTY, Ranges.EMPTY); return; } checkState(!result.isEmpty(), "Nothing found for epoch %d", epoch); @@ -1629,7 +1625,7 @@ public static void loadEpoch(long epoch, ClusterMetadata metadata, TopologyLoadC Ranges closed = row.has("closed") ? blobMapToRanges(row.getMap("closed", BytesType.instance, BytesType.instance)) : Ranges.EMPTY; Ranges redundant = row.has("redundant") ? blobMapToRanges(row.getMap("redundant", BytesType.instance, BytesType.instance)) : Ranges.EMPTY; - consumer.load(epoch, metadata, topology, syncStatus, pendingSyncNotify, remoteSyncComplete, closed, redundant); + consumer.load(epoch, syncStatus, pendingSyncNotify, remoteSyncComplete, closed, redundant); } public static EpochDiskState loadTopologies(TopologyLoadConsumer consumer) @@ -1640,8 +1636,8 @@ public static EpochDiskState loadTopologies(TopologyLoadConsumer consumer) if (diskState == null) return EpochDiskState.EMPTY; - for (ClusterMetadata metadata : AccordService.tcmLoadRange(diskState.minEpoch, diskState.maxEpoch)) - loadEpoch(metadata.epoch.getEpoch(), metadata, consumer); + for (long epoch = diskState.minEpoch; epoch < diskState.maxEpoch; epoch++) + loadEpoch(epoch, consumer); return diskState; } diff --git a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java index edc473cf3570..7c947958ea38 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSafeCommandStore.java @@ -199,7 +199,9 @@ public NodeCommandStoreService node() private O mapReduce(Routables keysOrRanges, BiFunction map, O accumulate) { - Invariants.checkState(context.keys().containsAll(keysOrRanges), "Attempted to access keysOrRanges outside of what was asked for; asked for %s, accessed %s", context.keys(), keysOrRanges); + Invariants.checkState(context.keys().containsAll(keysOrRanges), + "Attempted to access keysOrRanges outside of what was asked for; asked for %s, accessed %s. Context: %s", context.keys(), + keysOrRanges, context); accumulate = mapReduceForRange(keysOrRanges, map, accumulate); return mapReduceForKey(keysOrRanges, map, accumulate); } diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 3755348c66a2..746b5a7d9aac 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -133,7 +133,6 @@ import org.apache.cassandra.metrics.AccordClientRequestMetrics; import org.apache.cassandra.metrics.ClientRequestMetrics; import org.apache.cassandra.metrics.ClientRequestsMetricsHolder; -import org.apache.cassandra.metrics.TCMMetrics; import org.apache.cassandra.net.IVerbHandler; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessageDelivery; @@ -143,7 +142,6 @@ import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.service.StorageService; import org.apache.cassandra.service.accord.AccordSyncPropagator.Notification; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.AccordRoutingKey.KeyspaceSplitter; @@ -164,7 +162,6 @@ import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.tcm.ClusterMetadataService; import org.apache.cassandra.tcm.Epoch; -import org.apache.cassandra.tcm.Retry; import org.apache.cassandra.tcm.membership.NodeId; import org.apache.cassandra.tcm.ownership.DataPlacement; import org.apache.cassandra.tracing.Tracing; @@ -255,14 +252,15 @@ public synchronized static void startup(NodeId tcmId) } AccordService as = new AccordService(AccordTopology.tcmIdToAccord(tcmId)); as.startup(); - if (StorageService.instance.isReplacingSameAddress()) - { - // when replacing another node but using the same ip the hostId will also match, this causes no TCM transactions - // to be committed... - // In order to bootup correctly, need to pull in the current epoch - ClusterMetadata current = ClusterMetadata.current(); - as.configurationService().notifyPostCommit(current, current, false); - } + // TODO: check if this is still necessary (likely not) +// if (StorageService.instance.isReplacingSameAddress()) +// { +// // when replacing another node but using the same ip the hostId will also match, this causes no TCM transactions +// // to be committed... +// // In order to bootup correctly, need to pull in the current epoch +// ClusterMetadata current = ClusterMetadata.current(); +// as.configurationService().notifyPostCommit(current, current, false); +// } instance = as; replayJournal(as); @@ -273,18 +271,24 @@ private static void replayJournal(AccordService as) logger.info("Starting journal replay."); TimestampsForKey.unsafeSetReplay(true); CommandsForKey.disableLinearizabilityViolationsReporting(); - AccordKeyspace.truncateAllCaches(); - - as.journal().replay(as.node().commandStores()); + try + { + AccordKeyspace.truncateAllCaches(); + as.journal().replay(as.node().commandStores()); - logger.info("Waiting for command stores to quiesce."); - ((AccordCommandStores)as.node.commandStores()).waitForQuiescense(); - CommandsForKey.enableLinearizabilityViolationsReporting(); - TimestampsForKey.unsafeSetReplay(false); - as.journal.unsafeSetStarted(); + logger.info("Waiting for command stores to quiesce."); + ((AccordCommandStores)as.node.commandStores()).waitForQuiescense(); + as.journal.unsafeSetStarted(); + } + finally + { + TimestampsForKey.unsafeSetReplay(false); + CommandsForKey.enableLinearizabilityViolationsReporting(); + } logger.info("Finished journal replay."); } + public static void shutdownServiceAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException { IAccordService i = instance; @@ -308,6 +312,13 @@ public static IAccordService instance() return i; } + public static boolean started() + { + if (!DatabaseDescriptor.getAccordTransactionsEnabled()) + return false; + return instance != null; + } + private AccordService(Id localId) { Invariants.checkState(localId != null, "static localId must be set before instantiating AccordService"); @@ -316,13 +327,13 @@ private AccordService(Id localId) agent.setNodeId(localId); AccordTimeService time = new AccordTimeService(); final RequestCallbacks callbacks = new RequestCallbacks(time); - this.configService = new AccordConfigurationService(localId); - this.fastPathCoordinator = AccordFastPathCoordinator.create(localId, configService); - this.messageSink = new AccordMessageSink(agent, configService, callbacks); this.scheduler = new AccordScheduler(); this.dataStore = new AccordDataStore(); this.configuration = new AccordConfiguration(DatabaseDescriptor.getRawConfig()); this.journal = new AccordJournal(DatabaseDescriptor.getAccord().journal, agent); + this.configService = new AccordConfigurationService(localId, journal); + this.fastPathCoordinator = AccordFastPathCoordinator.create(localId, configService); + this.messageSink = new AccordMessageSink(agent, configService, callbacks); this.node = new Node(localId, messageSink, configService, @@ -338,10 +349,11 @@ private AccordService(Id localId) ignore -> callbacks, DefaultProgressLogs::new, DefaultLocalListeners.Factory::new, - AccordCommandStores.factory(journal), + AccordCommandStores.factory(), new AccordInteropFactory(agent, configService), journal.durableBeforePersister(), - configuration); + configuration, + journal); this.nodeShutdown = toShutdownable(node); this.requestHandler = new AccordVerbHandler<>(node, configService); this.responseHandler = new AccordResponseVerbHandler<>(callbacks, configService); @@ -354,50 +366,29 @@ public synchronized void startup() return; journal.start(node); node.load(); - ClusterMetadataService cms = ClusterMetadataService.instance(); - class Ref { List historic = Collections.emptyList();} - Ref ref = new Ref(); - configService.start((optMaxEpoch -> { - List historic = ref.historic = !optMaxEpoch.isEmpty() - ? tcmLoadRange(optMaxEpoch.getAsLong(), Long.MAX_VALUE) - : discoverHistoric(node, cms); - for (ClusterMetadata m : historic) - configService.reportMetadataInternal(m, true); - })); - ClusterMetadata current = cms.metadata(); - if (!ref.historic.isEmpty()) - { - List historic = ref.historic; - long lastHistoric = ref.historic.get(historic.size() - 1).epoch.getEpoch(); - if (lastHistoric + 1 < current.epoch.getEpoch()) - { - // new epochs added while loading... load the deltas - for (ClusterMetadata metadata : tcmLoadRange(lastHistoric + 1, current.epoch.getEpoch())) - { - historic.add(metadata); - configService.reportMetadataInternal(metadata); - } - } - // sync doesn't happen when this node isn't in the epoch - //TODO (now, correctness): sync points use "closed" and not "syncComplete", so need to call TM.epochRedundant or TM.onEpochClosed - // epochRedundant implies all txn that have been proposed for this epoch have been executed "globally" - we don't have this knowlege - // epochClosed implies no "new" txn can be proposed - for (ClusterMetadata m : historic) - { - Topology t = AccordTopology.createAccordTopology(m); - long epoch = t.epoch(); - for (Id id : t.nodes()) - node.onRemoteSyncComplete(id, epoch); - //TODO (correctness): is this true? - node.onEpochClosed(t.ranges(), t.epoch()); - node.onEpochRedundant(t.ranges(), t.epoch()); - } + ClusterMetadata metadata = ClusterMetadata.current(); + // TODO (review/discussion): Even though previous version was updating metadatas and topologies in the loop, in + // reality mappings were just "waved" through, while historical topologies would be saved by configuration + // service. In other words, it was _always_ possible for historical topologies to contain peers that are + // _not_ present in metadata anymore. + configService.updateMapping(metadata); + + node.commandStores().restoreShardStateUnsafe(); + configService.start(); + + long minEpoch = fetchMinEpoch(); + if (minEpoch >= 0) + { + for (long epoch = minEpoch; epoch <= metadata.epoch.getEpoch(); epoch++) + node.configService().fetchTopologyForEpoch(epoch); +// Invariants.checkState(node.configService().currentEpoch() >= metadata.epoch.getEpoch(), +// "Config service epoch is %s, but metadata epoch is reported to be %s", +// node.configService().currentEpoch(), metadata.epoch.getEpoch()); } - configService.reportMetadataInternal(current); fastPathCoordinator.start(); - cms.log().addListener(fastPathCoordinator); + ClusterMetadataService.instance().log().addListener(fastPathCoordinator); node.durabilityScheduling().setDefaultRetryDelay(Ints.checkedCast(DatabaseDescriptor.getAccordDefaultDurabilityRetryDelay(SECONDS)), SECONDS); node.durabilityScheduling().setMaxRetryDelay(Ints.checkedCast(DatabaseDescriptor.getAccordMaxDurabilityRetryDelay(SECONDS)), SECONDS); node.durabilityScheduling().setTargetShardSplits(Ints.checkedCast(DatabaseDescriptor.getAccordShardDurabilityTargetSplits())); @@ -408,25 +399,23 @@ class Ref { List historic = Collections.emptyList();} state = State.STARTED; } - private List discoverHistoric(Node node, ClusterMetadataService cms) + /** + * Queries peers to discover min epoch + */ + private long fetchMinEpoch() { - ClusterMetadata current = cms.metadata(); - Topology topology = AccordTopology.createAccordTopology(current); - Ranges localRanges = topology.rangesForNode(node.id()); - if (!localRanges.isEmpty()) // already joined, nothing to see here - return Collections.emptyList(); - + ClusterMetadata metadata = ClusterMetadata.current(); Map> peers = new HashMap<>(); - for (KeyspaceMetadata keyspace : current.schema.getKeyspaces()) + for (KeyspaceMetadata keyspace : metadata.schema.getKeyspaces()) { List tables = keyspace.tables.stream().filter(TableMetadata::requiresAccordSupport).collect(Collectors.toList()); if (tables.isEmpty()) continue; - DataPlacement placement = current.placements.get(keyspace.params.replication); - DataPlacement whenSettled = current.writePlacementAllSettled(keyspace); - Sets.SetView alive = Sets.intersection(whenSettled.writes.byEndpoint().keySet(), placement.writes.byEndpoint().keySet()); + DataPlacement current = metadata.placements.get(keyspace.params.replication); + DataPlacement settled = metadata.writePlacementAllSettled(keyspace); + Sets.SetView alive = Sets.intersection(settled.writes.byEndpoint().keySet(), current.writes.byEndpoint().keySet()); InetAddressAndPort self = FBUtilities.getBroadcastAddressAndPort(); - whenSettled.writes.forEach((range, group) -> { + settled.writes.forEach((range, group) -> { if (group.endpoints().contains(self)) { for (InetAddressAndPort peer : group.endpoints()) @@ -439,35 +428,12 @@ private List discoverHistoric(Node node, ClusterMetadataService }); } if (peers.isEmpty()) - return Collections.emptyList(); + return -1; Long minEpoch = findMinEpoch(SharedContext.Global.instance, peers); if (minEpoch == null) - return Collections.emptyList(); - return tcmLoadRange(minEpoch, current.epoch.getEpoch()); - } - - public static List tcmLoadRange(long min, long max) - { - List afterLoad = reconstruct(min, max); - - if (Invariants.isParanoid()) - Invariants.checkState(afterLoad.get(0).epoch.getEpoch() == min, "Unexpected epoch: expected %d but given %d", min, afterLoad.get(0).epoch.getEpoch()); - while (!afterLoad.isEmpty() && afterLoad.get(0).epoch.getEpoch() < min) - afterLoad.remove(0); - Invariants.checkState(!afterLoad.isEmpty(), "TCM was unable to return the needed epochs: %d -> %d", min, max); - Invariants.checkState(afterLoad.get(0).epoch.getEpoch() == min, "Unexpected epoch: expected %d but given %d", min, afterLoad.get(0).epoch.getEpoch()); - Invariants.checkState(max == Long.MAX_VALUE || afterLoad.get(afterLoad.size() - 1).epoch.getEpoch() == max, "Unexpected epoch: expected %d but given %d", max, afterLoad.get(afterLoad.size() - 1).epoch.getEpoch()); - return afterLoad; - } - - private static List reconstruct(long min, long max) - { - Epoch start = Epoch.create(min); - Epoch end = Epoch.create(max); - Retry.Deadline deadline = Retry.Deadline.wrap(new Retry.ExponentialBackoff(TCMMetrics.instance.fetchLogRetries)); - return ClusterMetadataService.instance().processor() - .reconstruct(start, end, deadline); + return -1; + return minEpoch; } @VisibleForTesting diff --git a/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java b/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java index 6f1e479f91a9..4c93ad64bc6d 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java +++ b/src/java/org/apache/cassandra/service/accord/AccordSyncPropagator.java @@ -232,25 +232,22 @@ public String toString() '}'; } - public synchronized void onNodesRemoved(Set removed) + public synchronized void onNodesRemoved(Node.Id removed) { - for (Node.Id node : removed) + PendingEpochs pendingEpochs = pending.get(removed.id); + if (pendingEpochs == null) return; + long[] toComplete = new long[pendingEpochs.size()]; + Long2ObjectHashMap.KeyIterator it = pendingEpochs.keySet().iterator(); + for (int i = 0; it.hasNext(); i++) + toComplete[i] = it.nextLong(); + Arrays.sort(toComplete); + for (long epoch : toComplete) + listener.onEndpointAck(removed, epoch); + pending.remove(removed.id); + for (long epoch : toComplete) { - PendingEpochs pendingEpochs = pending.get(node.id); - if (pendingEpochs == null) continue; - long[] toComplete = new long[pendingEpochs.size()]; - Long2ObjectHashMap.KeyIterator it = pendingEpochs.keySet().iterator(); - for (int i = 0; it.hasNext(); i++) - toComplete[i] = it.nextLong(); - Arrays.sort(toComplete); - for (long epoch : toComplete) - listener.onEndpointAck(node, epoch); - pending.remove(node.id); - for (long epoch : toComplete) - { - if (hasSyncCompletedFor(epoch)) - listener.onComplete(epoch); - } + if (hasSyncCompletedFor(epoch)) + listener.onComplete(epoch); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopology.java b/src/java/org/apache/cassandra/service/accord/AccordTopology.java index bced75a40172..62f39bb74aa4 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTopology.java +++ b/src/java/org/apache/cassandra/service/accord/AccordTopology.java @@ -235,7 +235,7 @@ public static Topology createAccordTopology(Epoch epoch, DistributedSchema schem Directory directory, AccordFastPath accordFastPath, ShardLookup lookup, AccordStaleReplicas staleReplicas) { - List shards = new ArrayList<>(); + List res = new ArrayList<>(); Set unavailable = accordFastPath.unavailableIds(); Map dcMap = createDCMap(directory); @@ -245,12 +245,15 @@ public static Topology createAccordTopology(Epoch epoch, DistributedSchema schem if (tables.isEmpty()) continue; List ksShards = KeyspaceShard.forKeyspace(keyspace, placements, directory); - tables.forEach(table -> ksShards.forEach(shard -> shards.add(shard.createForTable(table, unavailable, dcMap, lookup)))); + for (TableMetadata table : tables) + { + for (KeyspaceShard shard : ksShards) + res.add(shard.createForTable(table, unavailable, dcMap, lookup)); + } } - shards.sort((a, b) -> a.range.compare(b.range)); - - return new Topology(epoch.getEpoch(), SortedArrayList.copyUnsorted(staleReplicas.ids(), Id[]::new), shards.toArray(new Shard[0])); + res.sort((a, b) -> a.range.compare(b.range)); + return new Topology(epoch.getEpoch(), SortedArrayList.copyUnsorted(staleReplicas.ids(), Id[]::new), res.toArray(new Shard[0])); } public static Topology createAccordTopology(ClusterMetadata metadata, ShardLookup lookup) @@ -268,16 +271,16 @@ public static Topology createAccordTopology(ClusterMetadata metadata) return createAccordTopology(metadata, (Topology) null); } - public static EndpointMapping directoryToMapping(EndpointMapping mapping, long epoch, Directory directory) + public static EndpointMapping directoryToMapping(long epoch, Directory directory) { EndpointMapping.Builder builder = EndpointMapping.builder(epoch); for (NodeId id : directory.peerIds()) builder.add(directory.endpoint(id), tcmIdToAccord(id)); - // There are cases where nodes are removed from the cluster (host replacement, decom, etc.), but inflight events may still be happening; - // keep the ids around so pending events do not fail with a mapping error - for (Id id : mapping.differenceIds(builder)) - builder.add(mapping.mappedEndpoint(id), id); + // There are cases where nodes are removed from the cluster (host replacement, decom, etc.), but inflight events + // may still be happening; keep the ids around so pending events do not fail with a mapping error + for (Directory.RemovedNode removedNode : directory.removedNodes()) + builder.add(removedNode.endpoint, tcmIdToAccord(removedNode.id)); return builder.build(); } diff --git a/src/java/org/apache/cassandra/service/accord/EndpointMapping.java b/src/java/org/apache/cassandra/service/accord/EndpointMapping.java index 916cea43bf82..5745c551121d 100644 --- a/src/java/org/apache/cassandra/service/accord/EndpointMapping.java +++ b/src/java/org/apache/cassandra/service/accord/EndpointMapping.java @@ -18,12 +18,12 @@ package org.apache.cassandra.service.accord; -import java.util.Set; +import java.util.Map; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.ImmutableBiMap; -import com.google.common.collect.Sets; +import com.google.common.collect.ImmutableMap; import accord.local.Node; import accord.utils.Invariants; @@ -31,15 +31,18 @@ class EndpointMapping implements AccordEndpointMapper { - public static final EndpointMapping EMPTY = new EndpointMapping(0, ImmutableBiMap.of()); + public static final EndpointMapping EMPTY = new EndpointMapping(0, ImmutableBiMap.of(), ImmutableMap.of()); private final long epoch; private final ImmutableBiMap mapping; + private final ImmutableMap removedNodes; private EndpointMapping(long epoch, - ImmutableBiMap mapping) + ImmutableBiMap mapping, + ImmutableMap removedNodes) { this.epoch = epoch; this.mapping = mapping; + this.removedNodes = removedNodes; } long epoch() @@ -52,9 +55,9 @@ public boolean containsId(Node.Id id) return mapping.containsKey(id); } - public Set differenceIds(Builder builder) + public Map removedNodes() { - return Sets.difference(mapping.keySet(), builder.mapping.keySet()); + return removedNodes; } @Override @@ -73,6 +76,7 @@ static class Builder { private final long epoch; private final BiMap mapping = HashBiMap.create(); + private final ImmutableMap.Builder removed = new ImmutableMap.Builder<>(); public Builder(long epoch) { @@ -87,9 +91,18 @@ public Builder add(InetAddressAndPort endpoint, Node.Id id) return this; } + public Builder removed(InetAddressAndPort endpoint, Node.Id id, long epoch) + { + Invariants.checkArgument(!mapping.containsKey(id), "Mapping already exists for Node.Id %s", id); + Invariants.checkArgument(!mapping.containsValue(endpoint), "Mapping already exists for %s", endpoint); + mapping.put(id, endpoint); + removed.put(id, epoch); + return this; + } + public EndpointMapping build() { - return new EndpointMapping(epoch, ImmutableBiMap.copyOf(mapping)); + return new EndpointMapping(epoch, ImmutableBiMap.copyOf(mapping), removed.build()); } } diff --git a/src/java/org/apache/cassandra/service/accord/FetchMinEpoch.java b/src/java/org/apache/cassandra/service/accord/FetchMinEpoch.java index 0e40afc6c12d..dccfe58982de 100644 --- a/src/java/org/apache/cassandra/service/accord/FetchMinEpoch.java +++ b/src/java/org/apache/cassandra/service/accord/FetchMinEpoch.java @@ -25,7 +25,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; - import javax.annotation.Nullable; import com.google.common.annotations.VisibleForTesting; @@ -33,12 +32,13 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.TypeSizes; +import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.io.IVersionedSerializer; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.IVerbHandler; -import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.SharedContext; @@ -46,11 +46,15 @@ import org.apache.cassandra.utils.concurrent.Future; import org.apache.cassandra.utils.concurrent.FutureCombiner; +import static org.apache.cassandra.net.MessageDelivery.RetryErrorMessage; +import static org.apache.cassandra.net.MessageDelivery.RetryPredicate; +import static org.apache.cassandra.net.MessageDelivery.logger; + +// TODO: this can be simplifyed: we seem to always use "entire range" public class FetchMinEpoch { public static final IVersionedSerializer serializer = new IVersionedSerializer<>() { - @Override public void serialize(FetchMinEpoch t, DataOutputPlus out, int version) throws IOException { @@ -78,15 +82,20 @@ public long serializedSize(FetchMinEpoch t, int version) return size; } }; - public static final IVerbHandler handler = new IVerbHandler() - { - @Override - public void doVerb(Message message) throws IOException + + public static final IVerbHandler handler = message -> { + if (AccordService.started()) { Long epoch = AccordService.instance().minEpoch(message.payload.ranges); MessagingService.instance().respond(new Response(epoch), message); } + else + { + logger.error("Accord service is not started, resopnding with error to {}", message); + MessagingService.instance().respondWithFailure(RequestFailure.BOOTING, message); + } }; + public final Collection ranges; public FetchMinEpoch(Collection ranges) @@ -122,13 +131,14 @@ public static Future fetch(SharedContext context, Map> accum = new ArrayList<>(peers.size()); for (Map.Entry> e : peers.entrySet()) accum.add(fetch(context, e.getKey(), e.getValue())); - return FutureCombiner.successfulOf(accum).map(ls -> { + // TODO (required): we are collecting only successes, but we need some threshold + return FutureCombiner.successfulOf(accum).map(epochs -> { Long min = null; - for (Long l : ls) + for (Long epoch : epochs) { - if (l == null) continue; - if (min == null) min = l; - else min = Math.min(min, l); + if (epoch == null) continue; + if (min == null) min = epoch; + else min = Math.min(min, epoch); } return min; }); @@ -138,12 +148,12 @@ public static Future fetch(SharedContext context, Map fetch(SharedContext context, InetAddressAndPort to, Set value) { FetchMinEpoch req = new FetchMinEpoch(value); - Backoff backoff = Backoff.fromConfig(context, DatabaseDescriptor.getAccord().minEpochSyncRetry); - return context.messaging().sendWithRetries(backoff, context.optionalTasks()::schedule, + return context.messaging().sendWithRetries(Backoff.None.INSTANCE, + MessageDelivery.ImmediateRetryScheduler.instance, Verb.ACCORD_FETCH_MIN_EPOCH_REQ, req, Iterators.cycle(to), - (i1, i2, i3) -> true, - (i1, i2, i3, i4) -> null) + RetryPredicate.times(DatabaseDescriptor.getAccord().minEpochSyncRetry.maxAttempts.value), + RetryErrorMessage.EMPTY) .map(m -> m.payload.minEpoch); } @@ -206,4 +216,4 @@ public String toString() '}'; } } -} +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/FetchTopology.java b/src/java/org/apache/cassandra/service/accord/FetchTopology.java new file mode 100644 index 000000000000..bbe9a70f199b --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/FetchTopology.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service.accord; + +import java.io.IOException; +import java.util.Collection; + +import accord.topology.Topology; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.service.accord.serializers.TopologySerializers; +import org.apache.cassandra.utils.concurrent.Future; + +public class FetchTopology +{ + private final long epoch; + + public static final IVersionedSerializer serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(FetchTopology t, DataOutputPlus out, int version) throws IOException + { + out.writeLong(t.epoch); + } + + @Override + public FetchTopology deserialize(DataInputPlus in, int version) throws IOException + { + return new FetchTopology(in.readLong()); + } + + @Override + public long serializedSize(FetchTopology t, int version) + { + return Long.BYTES; + } + }; + + public FetchTopology(long epoch) + { + this.epoch = epoch; + } + + public static class Response + { + public static final IVersionedSerializer serializer = new IVersionedSerializer<>() + { + @Override + public void serialize(Response t, DataOutputPlus out, int version) throws IOException + { + out.writeLong(t.epoch); + TopologySerializers.topology.serialize(t.topology, out, version); // TODO: not messaging version though + } + + @Override + public Response deserialize(DataInputPlus in, int version) throws IOException + { + long epoch = in.readLong(); + Topology topology = TopologySerializers.topology.deserialize(in, version); // TODO: version + return new Response(epoch, topology); + } + + @Override + public long serializedSize(Response t, int version) + { + return Long.BYTES + TopologySerializers.topology.serializedSize(t.topology, version); // TODO: version + } + }; + + private final long epoch; + private final Topology topology; + + public Response(long epoch, Topology topology) + { + this.epoch = epoch; + this.topology = topology; + } + } + + public static final IVerbHandler handler = message -> { + long epoch = message.payload.epoch; + Topology topology = AccordService.instance().topology().globalForEpoch(epoch); + MessagingService.instance().respond(new Response(epoch, topology), message); + }; + + public static Future fetch(SharedContext context, Collection peers, long epoch) + { + FetchTopology req = new FetchTopology(epoch); + return context.messaging().sendWithRetries(SharedContext.Global.instance, Verb.ACCORD_FETCH_TOPOLOGY_REQ, req, peers) + .map(m -> m.payload.topology); + } +} \ No newline at end of file diff --git a/src/java/org/apache/cassandra/service/accord/JournalKey.java b/src/java/org/apache/cassandra/service/accord/JournalKey.java index ea95629456e4..68ab0a1c041c 100644 --- a/src/java/org/apache/cassandra/service/accord/JournalKey.java +++ b/src/java/org/apache/cassandra/service/accord/JournalKey.java @@ -40,6 +40,7 @@ import static org.apache.cassandra.db.TypeSizes.BYTE_SIZE; import static org.apache.cassandra.db.TypeSizes.INT_SIZE; import static org.apache.cassandra.db.TypeSizes.LONG_SIZE; +import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.*; import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.RangesForEpochSerializer; import static org.apache.cassandra.service.accord.AccordJournalValueSerializers.SafeToReadSerializer; @@ -266,6 +267,7 @@ public enum Type SAFE_TO_READ (3, new SafeToReadSerializer()), BOOTSTRAP_BEGAN_AT (4, new BootstrapBeganAtSerializer()), RANGES_FOR_EPOCH (5, new RangesForEpochSerializer()), + TOPOLOGY_UPDATE (6, new TopologyUpdateSerializer()), ; public final int id; diff --git a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java index 809ed7356a24..3e33661e1835 100644 --- a/src/java/org/apache/cassandra/tcm/ClusterMetadata.java +++ b/src/java/org/apache/cassandra/tcm/ClusterMetadata.java @@ -446,7 +446,7 @@ public Transformer register(NodeAddresses addresses, Location location, NodeVers public Transformer unregister(NodeId nodeId) { - directory = directory.without(nodeId); + directory = directory.without(epoch, nodeId); Node.Id accordId = AccordTopology.tcmIdToAccord(nodeId); if (accordStaleReplicas.contains(accordId)) @@ -516,7 +516,7 @@ public Transformer replaced(NodeId replaced, NodeId replacement) Collection transferringTokens = tokenMap.tokens(replaced); tokenMap = tokenMap.unassignTokens(replaced) .assignTokens(replacement, transferringTokens); - directory = directory.without(replaced) + directory = directory.without(epoch, replaced) .withRackAndDC(replacement) .withNodeState(replacement, NodeState.JOINED); diff --git a/src/java/org/apache/cassandra/tcm/Startup.java b/src/java/org/apache/cassandra/tcm/Startup.java index 02c7e5eb8900..9a3ae24417c8 100644 --- a/src/java/org/apache/cassandra/tcm/Startup.java +++ b/src/java/org/apache/cassandra/tcm/Startup.java @@ -373,14 +373,9 @@ public static void startup(Supplier initialTransformation, boole metadata = ClusterMetadata.current(); NodeState startingstate = metadata.directory.peerState(self); - switch (startingstate) - { - case REGISTERED: - case LEFT: - break; - default: - AccordService.startup(self); - } + if (startingstate != REGISTERED && startingstate != LEFT) + AccordService.startup(self); + switch (startingstate) { case REGISTERED: diff --git a/src/java/org/apache/cassandra/tcm/membership/Directory.java b/src/java/org/apache/cassandra/tcm/membership/Directory.java index d5784725bd10..f4ac63be4e57 100644 --- a/src/java/org/apache/cassandra/tcm/membership/Directory.java +++ b/src/java/org/apache/cassandra/tcm/membership/Directory.java @@ -33,6 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.utils.Invariants; +import accord.utils.btree.BTreeSet; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; @@ -60,6 +62,7 @@ public class Directory implements MetadataValue private final int nextId; private final Epoch lastModified; private final BTreeBiMap peers; + private final BTreeSet removedNodes; private final BTreeMap locations; public final BTreeMap states; public final BTreeMap versions; @@ -75,6 +78,7 @@ public Directory() this(1, Epoch.EMPTY, BTreeBiMap.empty(), + BTreeSet.empty(RemovedNode::compareTo), BTreeMap.empty(), BTreeMap.empty(), BTreeMap.empty(), @@ -87,6 +91,7 @@ public Directory() private Directory(int nextId, Epoch lastModified, BTreeBiMap peers, + BTreeSet removedNodes, BTreeMap locations, BTreeMap states, BTreeMap versions, @@ -98,6 +103,7 @@ private Directory(int nextId, this.nextId = nextId; this.lastModified = lastModified; this.peers = peers; + this.removedNodes = removedNodes; this.locations = locations; this.states = states; this.versions = versions; @@ -144,7 +150,7 @@ public Epoch lastModified() @Override public Directory withLastModified(Epoch epoch) { - return new Directory(nextId, epoch, peers, locations, states, versions, hostIds, addresses, endpointsByDC, racksByDC); + return new Directory(nextId, epoch, peers, removedNodes, locations, states, versions, hostIds, addresses, endpointsByDC, racksByDC); } public Directory withNonUpgradedNode(NodeAddresses addresses, @@ -188,6 +194,7 @@ private Directory with(NodeAddresses nodeAddresses, NodeId id, UUID hostId, Loca return new Directory(nextId + 1, lastModified, peers.without(id).with(id, nodeAddresses.broadcastAddress), + removedNodes, locations.withForce(id, location), states.withForce(id, NodeState.REGISTERED), versions.withForce(id, nodeVersion), @@ -199,14 +206,14 @@ private Directory with(NodeAddresses nodeAddresses, NodeId id, UUID hostId, Loca public Directory withNodeState(NodeId id, NodeState state) { - return new Directory(nextId, lastModified, peers, locations, states.withForce(id, state), versions, hostIds, addresses, endpointsByDC, racksByDC); + return new Directory(nextId, lastModified, peers, removedNodes, locations, states.withForce(id, state), versions, hostIds, addresses, endpointsByDC, racksByDC); } public Directory withNodeVersion(NodeId id, NodeVersion version) { if (Objects.equals(versions.get(id), version)) return this; - return new Directory(nextId, lastModified, peers, locations, states, versions.withForce(id, version), hostIds, addresses, endpointsByDC, racksByDC); + return new Directory(nextId, lastModified, peers, removedNodes, locations, states, versions.withForce(id, version), hostIds, addresses, endpointsByDC, racksByDC); } public Directory withNodeAddresses(NodeId id, NodeAddresses nodeAddresses) @@ -228,7 +235,7 @@ public Directory withNodeAddresses(NodeId id, NodeAddresses nodeAddresses) BTreeMap> updatedEndpointsByRack = racksByDC.withForce(location(id).datacenter, rackEP); return new Directory(nextId, lastModified, - peers.withForce(id,nodeAddresses.broadcastAddress), locations, states, versions, hostIds, addresses.withForce(id, nodeAddresses), + peers.withForce(id,nodeAddresses.broadcastAddress), removedNodes, locations, states, versions, hostIds, addresses.withForce(id, nodeAddresses), updatedEndpointsByDC, updatedEndpointsByRack); } @@ -243,7 +250,7 @@ public Directory withRackAndDC(NodeId id) rackEP = BTreeMultimap.empty(); rackEP = rackEP.with(location.rack, endpoint); - return new Directory(nextId, lastModified, peers, locations, states, versions, hostIds, addresses, + return new Directory(nextId, lastModified, peers, removedNodes, locations, states, versions, hostIds, addresses, endpointsByDC.with(location.datacenter, endpoint), racksByDC.withForce(location.datacenter, rackEP)); } @@ -259,12 +266,28 @@ public Directory withoutRackAndDC(NodeId id) newRacksByDC = racksByDC.without(location.datacenter); else newRacksByDC = racksByDC.withForce(location.datacenter, rackEP); - return new Directory(nextId, lastModified, peers, locations, states, versions, hostIds, addresses, + return new Directory(nextId, lastModified, peers, removedNodes, locations, states, versions, hostIds, addresses, endpointsByDC.without(location.datacenter, endpoint), newRacksByDC); } - public Directory without(NodeId id) + public Directory removed(Epoch removedIn, NodeId id, InetAddressAndPort addr) + { + Invariants.checkState(!peers.containsKey(id)); + return new Directory(nextId, + lastModified, + peers, + removedNodes.with(new RemovedNode(removedIn, id, addr)), + locations, + states, + versions, + hostIds, + addresses, + endpointsByDC, + racksByDC); + } + + public Directory without(Epoch removedIn, NodeId id) { InetAddressAndPort endpoint = peers.get(id); Location location = locations.get(id); @@ -276,6 +299,7 @@ public Directory without(NodeId id) return new Directory(nextId, lastModified, peers.without(id), + removedNodes.with(new RemovedNode(removedIn, id, peers.get(id))), locations.without(id), states.without(id), versions.without(id), @@ -292,6 +316,7 @@ public Directory without(NodeId id) return new Directory(nextId, lastModified, peers.without(id), + removedNodes.with(new RemovedNode(removedIn, id, peers.get(id))), locations.without(id), states.without(id), versions.without(id), @@ -337,6 +362,11 @@ public NavigableSet peerIds() return peers.keySet(); } + public BTreeSet removedNodes() + { + return removedNodes; + } + public NodeAddresses getNodeAddresses(NodeId id) { return addresses.get(id); @@ -554,6 +584,17 @@ public void serialize(Directory t, DataOutputPlus out, Version version) throws I } } Epoch.serializer.serialize(t.lastModified, out, version); + + if (version.isAtLeast(Version.V5)) + { + out.writeInt(t.removedNodes.size()); + for (RemovedNode removedNode : t.removedNodes) + { + out.writeLong(removedNode.removedIn.getEpoch()); + NodeId.serializer.serialize(removedNode.id, out, version); + InetAddressAndPort.MetadataSerializer.serializer.serialize(removedNode.endpoint, out, version); + } + } } public Directory deserialize(DataInputPlus in, Version version) throws IOException @@ -609,9 +650,23 @@ public Directory deserialize(DataInputPlus in, Version version) throws IOExcepti else nextId = maxId.id() + 1; } + + if (version.isAtLeast(Version.V5)) + { + int removedNodes = in.readInt(); + for (int i = 0; i < removedNodes; i++) + { + long epoch = in.readLong(); + NodeId nodeId = NodeId.serializer.deserialize(in, version); + InetAddressAndPort addr = InetAddressAndPort.MetadataSerializer.serializer.deserialize(in, version); + newDir.removed(Epoch.create(epoch), nodeId, addr); + } + } + return new Directory(nextId, lastModified, newDir.peers, + newDir.removedNodes, newDir.locations, newDir.states, newDir.versions, @@ -649,6 +704,18 @@ public long serializedSize(Directory t, Version version) } } size += Epoch.serializer.serializedSize(t.lastModified, version); + + if (version.isAtLeast(Version.V5)) + { + size += TypeSizes.INT_SIZE; + for (RemovedNode removedNode : t.removedNodes) + { + size += TypeSizes.LONG_SIZE; + size += NodeId.serializer.serializedSize(removedNode.id, version); + size += InetAddressAndPort.MetadataSerializer.serializer.serializedSize(removedNode.endpoint, version); + } + } + return size; } } @@ -758,4 +825,37 @@ public static void dumpDiff(Logger logger, Map l, Map r) logger.warn("Value for key {} is only present in the right set: {}", k, r.get(k)); } + + + public static class RemovedNode implements Comparable + { + public final Epoch removedIn; + public final NodeId id; + public final InetAddressAndPort endpoint; + + public RemovedNode(Epoch removedIn, NodeId id, InetAddressAndPort endpoint) + { + this.removedIn = removedIn; + this.id = id; + this.endpoint = endpoint; + } + + public boolean equals(Object object) + { + if (this == object) return true; + if (object == null || getClass() != object.getClass()) return false; + RemovedNode that = (RemovedNode) object; + return Objects.equals(removedIn, that.removedIn) && Objects.equals(id, that.id) && Objects.equals(endpoint, that.endpoint); + } + + public int hashCode() + { + return Objects.hash(removedIn, id, endpoint); + } + + public int compareTo(RemovedNode o) + { + return id.compareTo(o.id); + } + } } diff --git a/src/java/org/apache/cassandra/tcm/serialization/Version.java b/src/java/org/apache/cassandra/tcm/serialization/Version.java index add51eb944d3..00fdf9f65cec 100644 --- a/src/java/org/apache/cassandra/tcm/serialization/Version.java +++ b/src/java/org/apache/cassandra/tcm/serialization/Version.java @@ -50,6 +50,13 @@ public enum Version */ V3(3), + // Padding + V4(4), + /** + * - Accord + */ + V5(5), + UNKNOWN(Integer.MAX_VALUE); /** diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java index 38a696990845..d84365c3f269 100644 --- a/src/java/org/apache/cassandra/utils/HeapUtils.java +++ b/src/java/org/apache/cassandra/utils/HeapUtils.java @@ -130,7 +130,7 @@ public static String maybeCreateHeapDump() } else { - logger.debug("Heap dump creation on uncaught exceptions is disabled."); +// logger.debug("Heap dump creation on uncaught exceptions is disabled."); } } catch (Throwable e) diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 8b48dfe10c2f..8a3d6e9d1d6d 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -653,7 +653,7 @@ public String toString() } } - public static void waitForCMSToQuiesce(ICluster cluster, int[] cmsNodes) + public static void waitForCMSToQuiesce(ICluster cluster, int... cmsNodes) { // first step; find the largest epoch waitForCMSToQuiesce(cluster, maxEpoch(cluster, cmsNodes)); diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java new file mode 100644 index 000000000000..5f4551826381 --- /dev/null +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBootstrapTest.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.fuzz.topology; + +import java.util.HashSet; + +import org.junit.Test; + +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.Constants; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.Feature; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.TokenSupplier; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.shared.NetworkTopology; +import org.apache.cassandra.distributed.test.log.FuzzTestBase; +import org.apache.cassandra.harry.SchemaSpec; +import org.apache.cassandra.harry.dsl.HistoryBuilder; +import org.apache.cassandra.harry.dsl.HistoryBuilderHelper; +import org.apache.cassandra.harry.dsl.ReplayingHistoryBuilder; +import org.apache.cassandra.harry.execution.InJvmDTestVisitExecutor; +import org.apache.cassandra.harry.execution.QueryBuildingVisitExecutor; +import org.apache.cassandra.harry.gen.Generator; +import org.apache.cassandra.harry.gen.Generators; +import org.apache.cassandra.harry.gen.Generators.TrackingGenerator; +import org.apache.cassandra.harry.gen.SchemaGenerators; +import org.apache.cassandra.service.consensus.TransactionalMode; + +import static org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits; +import static org.apache.cassandra.distributed.shared.ClusterUtils.waitForCMSToQuiesce; +import static org.apache.cassandra.harry.checker.TestHelper.withRandom; + +public class AccordBootstrapTest extends FuzzTestBase +{ + private static final int WRITES = 10; + private static final int POPULATION = 1000; + @Test + public void bootstrapFuzzTest() throws Throwable + { + CassandraRelevantProperties.SYSTEM_TRACES_DEFAULT_RF.setInt(3); + IInvokableInstance forShutdown = null; + try (Cluster cluster = builder().withNodes(3) + .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(100)) + .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(100, "dc0", "rack0")) + .withConfig((config) -> config.with(Feature.NETWORK, Feature.GOSSIP) + .set("write_request_timeout", "2s") + .set("request_timeout", "5s") + .set("concurrent_accord_operations", 2) + .set("progress_barrier_min_consistency_level", "QUORUM") + .set("progress_barrier_default_consistency_level", "QUORUM") + .set("metadata_snapshot_frequency", 5)) + .start()) + { + IInvokableInstance cmsInstance = cluster.get(1); + forShutdown = cmsInstance; + waitForCMSToQuiesce(cluster, cmsInstance); + + HashSet downInstances = new HashSet<>(); + withRandom(rng -> { + Generator schemaGen = SchemaGenerators.trivialSchema(KEYSPACE, "bootstrap_fuzz", POPULATION, + SchemaSpec.optionsBuilder() + .addWriteTimestamps(false) + .withTransactionalMode(TransactionalMode.full) + ); + + SchemaSpec schema = schemaGen.generate(rng); + TrackingGenerator pkGen = Generators.tracking(Generators.int32(0, Math.min(schema.valueGenerators.pkPopulation(), POPULATION))); + Generator ckGen = Generators.int32(0, Math.min(schema.valueGenerators.ckPopulation(), POPULATION)); + HistoryBuilder history = new ReplayingHistoryBuilder(schema.valueGenerators, + hb -> InJvmDTestVisitExecutor.builder() + .consistencyLevel(ConsistencyLevel.QUORUM) + .wrapQueries(QueryBuildingVisitExecutor.WrapQueries.TRANSACTION) + .pageSizeSelector(p -> InJvmDTestVisitExecutor.PageSizeSelector.NO_PAGING) + .nodeSelector(lts -> { + while (true) + { + int pick = rng.nextInt(1, cluster.size() + 1); + if (!downInstances.contains(pick)) + return pick; + + } + }) + .build(schema, hb, cluster)); + + Runnable writeAndValidate = () -> { + for (int i = 0; i < WRITES; i++) + HistoryBuilderHelper.insertRandomData(schema, pkGen, ckGen, rng, history); + + for (int pk : pkGen.generated()) + history.selectPartition(pk); + }; + + history.customThrowing(() -> { + cluster.schemaChange(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 3};", KEYSPACE)); + cluster.schemaChange(schema.compile()); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Setup"); + Thread.sleep(1000); + writeAndValidate.run(); + + history.customThrowing(() -> { + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true); + cluster.bootstrap(config).startup(); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Start boostrap"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + downInstances.add(2); + ClusterUtils.stopUnchecked(cluster.get(2)); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now DOWN"); + }, "Shut down node 2"); + + history.customThrowing(() -> { + IInstanceConfig config = cluster.newInstanceConfig() + .set("auto_bootstrap", true) + .set(Constants.KEY_DTEST_FULL_STARTUP, true); + cluster.bootstrap(config).startup(); + waitForCMSToQuiesce(cluster, cmsInstance); + }, "Bootstrap one more"); + + writeAndValidate.run(); + + history.customThrowing(() -> { + cluster.get(2).startup(); + cluster.get(1).logs().watchFor("/127.0.0.2:.* is now UP"); + downInstances.remove(2); + }, "Start up node 2"); + + }); + } + catch (Throwable t) + { + if (forShutdown != null) + unpauseCommits(forShutdown); + throw t; + } + } +} diff --git a/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java index 84f5fbbe5e89..feb51f117cec 100644 --- a/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java +++ b/test/harry/main/org/apache/cassandra/harry/SchemaSpec.java @@ -376,6 +376,11 @@ private OptionsBuilder() { } + public Options build() + { + return this; + } + public OptionsBuilder withTransactionalMode(TransactionalMode mode) { this.transactionalMode = mode; diff --git a/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java index 1922f0011a7e..92a962768307 100644 --- a/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java +++ b/test/harry/main/org/apache/cassandra/harry/execution/CQLVisitExecutor.java @@ -90,9 +90,12 @@ public enum ResultDumpMode public static void replayAfterFailure(Visit visit, CQLVisitExecutor executor, Model.Replay replay) { QueryBuildingVisitExecutor queryBuilder = executor.queryBuilder; - logger.error("Caught an exception at {} while replaying {}\ncluster.schemaChange(\"{}\");\nOperations _for this partition_ up to this visit:", - visit, queryBuilder.compile(visit), - queryBuilder.schema.compile()); + if (!visit.hasCustom) + { + logger.error("Caught an exception at {} while replaying {}\ncluster.schemaChange(\"{}\");\nOperations _for this partition_ up to this visit:", + visit, queryBuilder.compile(visit), + queryBuilder.schema.compile()); + } // Configurable yet hardcoded for a person who is trying to generate repro ResultDumpMode mode = ResultDumpMode.PARTITION; diff --git a/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java index 32d6492e0288..ccc95afe0cc9 100644 --- a/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java +++ b/test/harry/main/org/apache/cassandra/harry/execution/InJvmDTestVisitExecutor.java @@ -292,9 +292,10 @@ public interface ConsistencyLevelSelector public interface RetryPolicy { RetryPolicy RETRY_ON_TIMEOUT = (t) -> { - return t.getMessage().contains("timed out") || - AssertionUtils.isInstanceof(RequestTimeoutException.class) - .matches(Throwables.getRootCause(t)); + return t.getMessage() != null && + (t.getMessage().contains("timed out") || + AssertionUtils.isInstanceof(RequestTimeoutException.class) + .matches(Throwables.getRootCause(t))); }; boolean retry(Throwable t); } diff --git a/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java b/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java index c0e0c6e5696b..a3433c5b56d1 100644 --- a/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java +++ b/test/harry/main/org/apache/cassandra/harry/gen/SchemaGenerators.java @@ -97,6 +97,11 @@ public ColumnSpec generate(EntropySource rng) } public static Generator trivialSchema(String ks, String table, int population) + { + return trivialSchema(ks, table, population, SchemaSpec.optionsBuilder().build()); + } + + public static Generator trivialSchema(String ks, String table, int population, SchemaSpec.Options options) { return (rng) -> { return new SchemaSpec(rng.next(), @@ -105,7 +110,8 @@ public static Generator trivialSchema(String ks, String table, int p Arrays.asList(ColumnSpec.pk("pk1", ColumnSpec.int64Type, Generators.int64())), Arrays.asList(ColumnSpec.ck("ck1", ColumnSpec.int64Type, Generators.int64(), false)), Arrays.asList(ColumnSpec.regularColumn("v1", ColumnSpec.int64Type)), - List.of(ColumnSpec.staticColumn("s1", ColumnSpec.int64Type))); + Arrays.asList(ColumnSpec.staticColumn("s1", ColumnSpec.int64Type)), + options); }; } } diff --git a/test/harry/main/org/apache/cassandra/harry/op/Visit.java b/test/harry/main/org/apache/cassandra/harry/op/Visit.java index 6c8c31390c89..7d77b7287159 100644 --- a/test/harry/main/org/apache/cassandra/harry/op/Visit.java +++ b/test/harry/main/org/apache/cassandra/harry/op/Visit.java @@ -32,6 +32,8 @@ public class Visit public final Set visitedPartitions; public final boolean selectOnly; + public final boolean hasCustom; + public Visit(long lts, Operation[] operations) { Assert.assertTrue(operations.length > 0); @@ -39,8 +41,11 @@ public Visit(long lts, Operation[] operations) this.operations = operations; this.visitedPartitions = new HashSet<>(); boolean selectOnly = true; + boolean hasCustom = false; for (Operation operation : operations) { + if (operation.kind() == Operations.Kind.CUSTOM) + hasCustom = true; if (selectOnly && !(operation instanceof Operations.SelectStatement)) selectOnly = false; @@ -49,6 +54,7 @@ public Visit(long lts, Operation[] operations) } this.selectOnly = selectOnly; + this.hasCustom = hasCustom; } public String toString() diff --git a/test/unit/org/apache/cassandra/net/MessageDeliveryTest.java b/test/unit/org/apache/cassandra/net/MessageDeliveryTest.java index 9d2fa5de21e2..0661304b03ca 100644 --- a/test/unit/org/apache/cassandra/net/MessageDeliveryTest.java +++ b/test/unit/org/apache/cassandra/net/MessageDeliveryTest.java @@ -47,14 +47,12 @@ import org.mockito.Mockito; import static accord.utils.Property.qt; +import static org.apache.cassandra.net.MessageDelivery.*; import static org.assertj.core.api.Assertions.assertThat; public class MessageDeliveryTest { private static final InetAddressAndPort ID1 = InetAddressAndPort.getByNameUnchecked("127.0.0.1"); - private static final MessageDelivery.RetryErrorMessage RETRY_ERROR_MESSAGE = (i1, i2, i3, i4) -> null; - private static final MessageDelivery.RetryPredicate ALWAYS_RETRY = (i1, i2, i3) -> true; - private static final MessageDelivery.RetryPredicate ALWAYS_REJECT = (i1, i2, i3) -> false; static { @@ -79,8 +77,8 @@ public void sendWithRetryFailsAfterMaxAttempts() scheduler::schedule, Verb.ECHO_REQ, NoPayload.noPayload, Iterators.cycle(ID1), - ALWAYS_RETRY, - RETRY_ERROR_MESSAGE); + RetryPredicate.ALWAYS_RETRY, + RetryErrorMessage.EMPTY); assertThat(result).isNotDone(); factory.processAll(); assertThat(result).isDone(); @@ -104,8 +102,8 @@ public void sendWithRetryFirstAttempt() scheduler::schedule, Verb.ECHO_REQ, NoPayload.noPayload, Iterators.cycle(ID1), - ALWAYS_RETRY, - RETRY_ERROR_MESSAGE); + RetryPredicate.ALWAYS_RETRY, + RetryErrorMessage.EMPTY); assertThat(result).isNotDone(); factory.processAll(); assertThat(result).isDone(); @@ -135,8 +133,8 @@ public void sendWithRetry() scheduler::schedule, Verb.ECHO_REQ, NoPayload.noPayload, Iterators.cycle(ID1), - ALWAYS_RETRY, - RETRY_ERROR_MESSAGE); + RetryPredicate.ALWAYS_RETRY, + RetryErrorMessage.EMPTY); assertThat(result).isNotDone(); factory.processAll(); assertThat(result).isDone(); @@ -163,8 +161,8 @@ public void sendWithRetryDontAllowRetry() scheduler::schedule, Verb.ECHO_REQ, NoPayload.noPayload, Iterators.cycle(ID1), - ALWAYS_REJECT, - RETRY_ERROR_MESSAGE); + RetryPredicate.ALWAYS_REJECT, + RetryErrorMessage.EMPTY); assertThat(result).isNotDone(); factory.processAll(); assertThat(result).isDone(); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordConfigurationServiceTest.java b/test/unit/org/apache/cassandra/service/accord/AccordConfigurationServiceTest.java index 7210d4a09716..67dfb4fbfc4d 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordConfigurationServiceTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordConfigurationServiceTest.java @@ -26,21 +26,6 @@ import java.util.UUID; import com.google.common.collect.Sets; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.dht.Range; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.locator.Replica; -import org.apache.cassandra.schema.DistributedSchema; -import org.apache.cassandra.schema.KeyspaceMetadata; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.schema.Tables; -import org.apache.cassandra.tcm.ValidatingClusterMetadataService; -import org.apache.cassandra.tcm.membership.Location; -import org.apache.cassandra.tcm.membership.NodeAddresses; -import org.apache.cassandra.tcm.membership.NodeId; -import org.apache.cassandra.tcm.membership.NodeVersion; -import org.apache.cassandra.tcm.ownership.DataPlacement; -import org.apache.cassandra.tcm.serialization.Version; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -56,16 +41,31 @@ import org.apache.cassandra.concurrent.ScheduledExecutors; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.locator.Replica; import org.apache.cassandra.net.ConnectionType; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.schema.DistributedSchema; +import org.apache.cassandra.schema.KeyspaceMetadata; import org.apache.cassandra.schema.KeyspaceParams; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.Tables; import org.apache.cassandra.service.accord.AccordKeyspace.EpochDiskState; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ValidatingClusterMetadataService; +import org.apache.cassandra.tcm.membership.Location; +import org.apache.cassandra.tcm.membership.NodeAddresses; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.ownership.DataPlacement; +import org.apache.cassandra.tcm.serialization.Version; import org.apache.cassandra.utils.MockFailureDetector; import org.apache.cassandra.utils.concurrent.Future; @@ -175,7 +175,8 @@ public void initialEpochTest() throws Throwable { ValidatingClusterMetadataService cms = ValidatingClusterMetadataService.createAndRegister(Version.MIN_ACCORD_VERSION); - AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + + AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, null); Assert.assertEquals(null, AccordKeyspace.loadEpochDiskState()); service.start(); Assert.assertEquals(null, AccordKeyspace.loadEpochDiskState()); @@ -183,16 +184,14 @@ public void initialEpochTest() throws Throwable Topology topology1 = createTopology(cms); service.reportTopology(topology1); - loadEpoch(1, cms.metadata(), (epoch, cm, topology, syncStatus, pendingSync, remoteSync, closed, redundant) -> { - Assert.assertEquals(topology1, topology); + loadEpoch(1, (epoch, syncStatus, pendingSync, remoteSync, closed, redundant) -> { Assert.assertTrue(remoteSync.isEmpty()); }); Assert.assertEquals(EpochDiskState.create(1), service.diskState()); service.receiveRemoteSyncComplete(ID1, 1); service.receiveRemoteSyncComplete(ID2, 1); - loadEpoch(1, cms.metadata(), (epoch, cm, topology, syncStatus, pendingSync, remoteSync, closed, redundant) -> { - Assert.assertEquals(topology1, topology); + loadEpoch(1, (epoch, syncStatus, pendingSync, remoteSync, closed, redundant) -> { Assert.assertEquals(Sets.newHashSet(ID1, ID2), remoteSync); }); } @@ -202,7 +201,8 @@ public void loadTest() { ValidatingClusterMetadataService cms = ValidatingClusterMetadataService.createAndRegister(Version.MIN_ACCORD_VERSION); - AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + // TODO + AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, null); service.start(); Topology topology1 = createTopology(cms); @@ -222,7 +222,7 @@ public void loadTest() service.reportTopology(topology3); service.acknowledgeEpoch(EpochReady.done(3), true); - AccordConfigurationService loaded = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + AccordConfigurationService loaded = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, null); // TODO loaded.updateMapping(mappingForEpoch(cms.metadata().epoch.getEpoch() + 1)); AbstractConfigurationServiceTest.TestListener listener = new AbstractConfigurationServiceTest.TestListener(loaded, true); loaded.registerListener(listener); @@ -243,7 +243,7 @@ public void truncateTest() { ValidatingClusterMetadataService cms = ValidatingClusterMetadataService.createAndRegister(Version.MIN_ACCORD_VERSION); - AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + AccordConfigurationService service = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, null); // TODO TestListener serviceListener = new TestListener(service, true); service.registerListener(serviceListener); service.start(); @@ -261,7 +261,7 @@ public void truncateTest() Assert.assertEquals(EpochDiskState.create(3), service.diskState()); serviceListener.assertTruncates(3L); - AccordConfigurationService loaded = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks); + AccordConfigurationService loaded = new AccordConfigurationService(ID1, new Messaging(), new MockFailureDetector(), AccordConfigurationService.SystemTableDiskStateManager.instance, ScheduledExecutors.scheduledTasks, null); // TODO loaded.updateMapping(mappingForEpoch(cms.metadata().epoch.getEpoch() + 1)); TestListener loadListener = new TestListener(loaded, true); loaded.registerListener(loadListener); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java index 7e10e8ed5c56..3f5b59f0dcf4 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordTestUtils.java @@ -43,7 +43,6 @@ import accord.api.Timeouts; import accord.impl.DefaultLocalListeners; import accord.impl.DefaultLocalListeners.NotifySink.NoOpNotifySink; -import accord.impl.InMemoryCommandStore; import accord.local.Command; import accord.local.CommandStore; import accord.local.CommandStores; @@ -356,33 +355,6 @@ private void set() } } - // TODO: why unused? - public static InMemoryCommandStore.Synchronized createInMemoryCommandStore(LongSupplier now, String keyspace, String table) - { - TableMetadata metadata = Schema.instance.getTableMetadata(keyspace, table); - TokenRange range = TokenRange.fullRange(metadata.id); - Node.Id node = new Id(1); - NodeCommandStoreService time = new NodeCommandStoreService() - { - private ToLongFunction elapsed = TimeService.elapsedWrapperFromNonMonotonicSource(TimeUnit.MICROSECONDS, this::now); - - @Override public Id id() { return node;} - @Override public Timeouts timeouts() { return null; } - @Override public DurableBefore durableBefore() { return DurableBefore.EMPTY; } - @Override public long epoch() {return 1; } - @Override public long now() {return now.getAsLong(); } - @Override public Timestamp uniqueNow() { return uniqueNow(Timestamp.NONE); } - @Override public Timestamp uniqueNow(Timestamp atLeast) { return Timestamp.fromValues(1, now.getAsLong(), node); } - @Override public long elapsed(TimeUnit timeUnit) { return elapsed.applyAsLong(timeUnit); } - }; - - SingleEpochRanges holder = new SingleEpochRanges(Ranges.of(range)); - InMemoryCommandStore.Synchronized result = new InMemoryCommandStore.Synchronized(0, time, new AccordAgent(), - null, null, cs -> null, holder); - holder.set(); - return result; - } - public static AccordCommandStore createAccordCommandStore( Node.Id node, LongSupplier now, Topology topology, ExecutorPlus loadExecutor, ExecutorPlus saveExecutor) { diff --git a/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java b/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java index 55e64e87a95e..30c05691ae5d 100644 --- a/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java +++ b/test/unit/org/apache/cassandra/service/accord/EpochSyncTest.java @@ -20,7 +20,6 @@ import java.net.UnknownHostException; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.EnumMap; @@ -50,6 +49,7 @@ import accord.api.ConfigurationService; import accord.api.ConfigurationService.EpochReady; +import accord.api.Journal; import accord.api.LocalConfig; import accord.api.Scheduler; import accord.impl.SizeOfIntersectionSorter; @@ -671,7 +671,8 @@ private class Instance // TODO (review): Should there be a real scheduler here? Is it possible to adapt the Scheduler interface to scheduler used in this test? this.topology = new TopologyManager(SizeOfIntersectionSorter.SUPPLIER, new TestAgent.RethrowAgent(), id, Scheduler.NEVER_RUN_SCHEDULED, TimeService.ofNonMonotonic(globalExecutor::currentTimeMillis, TimeUnit.MILLISECONDS), LocalConfig.DEFAULT); AccordConfigurationService.DiskStateManager instance = MockDiskStateManager.instance; - config = new AccordConfigurationService(node, messagingService, failureDetector, instance, scheduler); + Journal journal = null; // TODO + config = new AccordConfigurationService(node, messagingService, failureDetector, instance, scheduler, journal); config.registerListener(new ConfigurationService.Listener() { @Override @@ -699,9 +700,9 @@ public void onRemoteSyncComplete(Node.Id node, long epoch) } @Override - public void onRemoveNodes(long epoch, Collection removed) + public void onRemoveNode(long epoch, Node.Id removed) { - topology.onRemoveNodes(epoch, removed); + topology.onRemoveNode(epoch, removed); } @Override From 1bbd3667311342d8185a2f1b8e83f527b154773d Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Mon, 16 Dec 2024 16:27:36 +0100 Subject: [PATCH 10/10] Fix tests --- .../apache/cassandra/net/MessageDelivery.java | 40 ++--------- .../apache/cassandra/net/MessagingUtils.java | 67 +++++++++++++++++++ .../accord/AccordConfigurationService.java | 4 ++ .../service/accord/AccordService.java | 16 ++++- .../service/accord/FetchTopology.java | 7 +- .../test/accord/AccordBootstrapTest.java | 14 ++-- 6 files changed, 103 insertions(+), 45 deletions(-) create mode 100644 src/java/org/apache/cassandra/net/MessagingUtils.java diff --git a/src/java/org/apache/cassandra/net/MessageDelivery.java b/src/java/org/apache/cassandra/net/MessageDelivery.java index ccbabea4da27..dc08d9df5685 100644 --- a/src/java/org/apache/cassandra/net/MessageDelivery.java +++ b/src/java/org/apache/cassandra/net/MessageDelivery.java @@ -31,7 +31,6 @@ import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.exceptions.RequestFailureReason; import org.apache.cassandra.locator.InetAddressAndPort; -import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.utils.Backoff; import org.apache.cassandra.utils.Pair; import org.apache.cassandra.utils.concurrent.Accumulator; @@ -100,41 +99,12 @@ public default Future> sendWithRetries(Backoff backoff, return promise; } - public default Future> sendWithRetries(SharedContext sharedContext, Verb verb, REQ request, Collection peers) + public default Future> sendWithRetries(Verb verb, REQ request, + Iterator candidates, + RetryPredicate shouldRetry, + RetryErrorMessage errorMessage) { - // TODO: move somewhere else - Iterator candidates = new Iterator<>() - { - boolean firstRun = true; - Iterator iter = peers.iterator(); - - public boolean hasNext() - { - return !peers.isEmpty(); - } - - public InetAddressAndPort next() - { - // At first, try all alive nodes - if (firstRun) - { - while (iter.hasNext()) - { - InetAddressAndPort candidate = iter.next(); - if (sharedContext.failureDetector().isAlive(candidate)) - return candidate; - } - firstRun = false; - } - - // After that, cycle through all nodes - if (!iter.hasNext()) - iter = peers.iterator(); - - return iter.next(); - } - }; - return sendWithRetries(Backoff.None.INSTANCE, ImmediateRetryScheduler.instance, verb, request, candidates, RetryPredicate.ALWAYS_RETRY, RetryErrorMessage.EMPTY); + return sendWithRetries(Backoff.None.INSTANCE, ImmediateRetryScheduler.instance, verb, request, candidates, shouldRetry, errorMessage); } public default void sendWithRetries(Backoff backoff, RetryScheduler retryThreads, diff --git a/src/java/org/apache/cassandra/net/MessagingUtils.java b/src/java/org/apache/cassandra/net/MessagingUtils.java new file mode 100644 index 000000000000..2190eaf3a655 --- /dev/null +++ b/src/java/org/apache/cassandra/net/MessagingUtils.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.net; + +import java.util.Iterator; + +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.repair.SharedContext; + +public class MessagingUtils +{ + /** + * Candidate iterator that would try all endpoints known to be alive first, and then try all endpoints + * in a round-robin manner. + */ + public static Iterator tryAliveFirst(SharedContext context, Iterable peers) + { + return new Iterator<>() + { + boolean firstRun = true; + Iterator iter = peers.iterator(); + boolean isEmpty = !iter.hasNext(); + + public boolean hasNext() + { + return !isEmpty; + } + + public InetAddressAndPort next() + { + // At first, try all alive nodes + if (firstRun) + { + while (iter.hasNext()) + { + InetAddressAndPort candidate = iter.next(); + if (context.failureDetector().isAlive(candidate)) + return candidate; + } + firstRun = false; + } + + // After that, cycle through all nodes + if (!iter.hasNext()) + iter = peers.iterator(); + + return iter.next(); + } + }; + } +} diff --git a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java index 4678adfb0d52..429758b2bb1c 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java @@ -440,11 +440,15 @@ protected void fetchTopologyInternal(long epoch) } catch (InterruptedException e) { + if (currentEpoch() >= epoch) + return; Thread.currentThread().interrupt(); throw new UncheckedInterruptedException(e); } catch (ExecutionException e) { + if (currentEpoch() >= epoch) + return; throw new RuntimeException(e.getCause()); } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 746b5a7d9aac..a88efb575ed7 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -382,9 +382,19 @@ public synchronized void startup() { for (long epoch = minEpoch; epoch <= metadata.epoch.getEpoch(); epoch++) node.configService().fetchTopologyForEpoch(epoch); -// Invariants.checkState(node.configService().currentEpoch() >= metadata.epoch.getEpoch(), -// "Config service epoch is %s, but metadata epoch is reported to be %s", -// node.configService().currentEpoch(), metadata.epoch.getEpoch()); + + try + { + epochReady(metadata.epoch).get(DatabaseDescriptor.getTransactionTimeout(MILLISECONDS), MILLISECONDS); + } + catch (InterruptedException e) + { + throw new UncheckedInterruptedException(e); + } + catch (ExecutionException | TimeoutException e) + { + throw new RuntimeException(e); + } } fastPathCoordinator.start(); diff --git a/src/java/org/apache/cassandra/service/accord/FetchTopology.java b/src/java/org/apache/cassandra/service/accord/FetchTopology.java index bbe9a70f199b..a8cf491ebfa5 100644 --- a/src/java/org/apache/cassandra/service/accord/FetchTopology.java +++ b/src/java/org/apache/cassandra/service/accord/FetchTopology.java @@ -27,7 +27,9 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.MessageDelivery; import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.MessagingUtils; import org.apache.cassandra.net.Verb; import org.apache.cassandra.repair.SharedContext; import org.apache.cassandra.service.accord.serializers.TopologySerializers; @@ -108,7 +110,10 @@ public Response(long epoch, Topology topology) public static Future fetch(SharedContext context, Collection peers, long epoch) { FetchTopology req = new FetchTopology(epoch); - return context.messaging().sendWithRetries(SharedContext.Global.instance, Verb.ACCORD_FETCH_TOPOLOGY_REQ, req, peers) + return context.messaging().sendWithRetries(Verb.ACCORD_FETCH_TOPOLOGY_REQ, req, MessagingUtils.tryAliveFirst(SharedContext.Global.instance, peers), + // If the epoch is already discovered, no need to retry + (attempt, from, failure) -> AccordService.instance().currentEpoch() < epoch, + MessageDelivery.RetryErrorMessage.EMPTY) .map(m -> m.payload.topology); } } \ No newline at end of file diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java index 82371a84bb57..b94c05776a83 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java @@ -183,12 +183,17 @@ public void bootstrapTest() throws Throwable .with(NETWORK, GOSSIP)) .start()) { + cluster.schemaChange("CREATE KEYSPACE ks WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor':2}"); + cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key(k, c)) WITH transactional_mode='full'"); + long initialMax = maxEpoch(cluster); + for (IInvokableInstance node : cluster) { node.runOnInstance(() -> { Assert.assertEquals(initialMax, ClusterMetadata.current().epoch.getEpoch()); + System.out.println("Awaiting " + initialMax); awaitEpoch(initialMax); AccordConfigurationService configService = service().configurationService(); long minEpoch = configService.minEpoch(); @@ -211,9 +216,6 @@ public void bootstrapTest() throws Throwable node.runOnInstance(StreamListener::register); } - cluster.schemaChange("CREATE KEYSPACE ks WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor':2}"); - cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key(k, c)) WITH transactional_mode='full'"); - long schemaChangeMax = maxEpoch(cluster); for (IInvokableInstance node : cluster) { @@ -363,6 +365,9 @@ public void moveTest() throws Throwable .with(NETWORK, GOSSIP)) .start()) { + cluster.schemaChange("CREATE KEYSPACE ks WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor':2}"); + cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key(k, c)) WITH transactional_mode='full'"); + long initialMax = maxEpoch(cluster); long[] tokens = new long[3]; for (int i=0; i<3; i++) @@ -392,9 +397,6 @@ public void moveTest() throws Throwable }); } - cluster.schemaChange("CREATE KEYSPACE ks WITH REPLICATION={'class':'SimpleStrategy', 'replication_factor':2}"); - cluster.schemaChange("CREATE TABLE ks.tbl (k int, c int, v int, primary key(k, c)) WITH transactional_mode='full'"); - long schemaChangeMax = maxEpoch(cluster); for (IInvokableInstance node : cluster) {