From 09b52668978ffb9d185e5b17704fbd11c5d3f343 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Thu, 26 Jun 2025 17:55:11 +0200 Subject: [PATCH 1/6] Accord: Add Rebootstrap and unsafe Bootstrap To support recovering a node that has lost some of its local transaction log, introduce rebootstrap and unsafe bootstrap modes, where Accord ensures no responses are produced for transactions the node cannot be certain it had not previously answered. patch by Benedict and Alex Petrov for CASSANDRA-20908 --- .gitmodules | 4 +- modules/accord | 2 +- .../db/virtual/AccordDebugKeyspace.java | 26 +-- .../metrics/AccordSystemMetrics.java | 2 +- .../service/ActiveRepairService.java | 3 +- .../org/apache/cassandra/service/Rebuild.java | 3 +- .../cassandra/service/StorageService.java | 32 ++- .../service/accord/AccordCommandStores.java | 24 +-- .../accord/AccordConfigurationService.java | 4 +- .../service/accord/AccordDataStore.java | 28 ++- .../service/accord/AccordJournal.java | 3 +- .../service/accord/AccordService.java | 34 ++- .../service/accord/AccordTopology.java | 3 +- .../service/accord/IAccordService.java | 19 +- .../service/accord/api/AccordAgent.java | 9 +- .../accord/api/AccordViolationHandler.java | 10 +- .../interop/AccordInteropStableThenRead.java | 4 +- .../accord/serializers/AcceptSerializers.java | 4 +- .../accord/serializers/AwaitSerializers.java | 4 +- .../BeginInvalidationSerializers.java | 4 +- .../serializers/CheckStatusSerializers.java | 8 +- .../serializers/CommandStoreSerializers.java | 13 +- .../accord/serializers/CommitSerializers.java | 8 +- .../serializers/RecoverySerializers.java | 2 +- .../serializers/TxnRequestSerializer.java | 6 +- .../tcm/sequences/BootstrapAndJoin.java | 3 +- .../tcm/sequences/DropAccordTable.java | 3 +- .../apache/cassandra/tcm/sequences/Move.java | 3 +- .../distributed/shared/ClusterUtils.java | 3 +- .../distributed/test/TestBaseImpl.java | 9 +- .../test/accord/AccordBootstrapTest.java | 195 +++++++----------- .../accord/AccordIncrementalRepairTest.java | 12 +- .../test/accord/AccordIntegrationTest.java | 2 +- .../test/accord/AccordTestBase.java | 2 +- .../fuzz/topology/AccordBounceTest.java | 8 +- .../service/accord/AccordJournalBurnTest.java | 6 +- .../db/virtual/AccordDebugKeyspaceTest.java | 9 +- .../index/accord/RouteIndexTest.java | 6 +- .../service/accord/AccordCommandTest.java | 2 +- .../accord/AccordSyncPropagatorTest.java | 6 +- .../accord/SimulatedAccordCommandStore.java | 10 +- .../accord/SimulatedAccordTaskTest.java | 4 +- .../CommandsForKeySerializerTest.java | 6 +- .../cassandra/utils/AccordGenerators.java | 10 +- 44 files changed, 266 insertions(+), 292 deletions(-) diff --git a/.gitmodules b/.gitmodules index 616dacf610a7..510c7c19e633 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/belliottsmith/cassandra-accord.git + branch = rebootstrap-sq diff --git a/modules/accord b/modules/accord index 520818a004a8..5b35a301d160 160000 --- a/modules/accord +++ b/modules/accord @@ -1 +1 @@ -Subproject commit 520818a004a89217cf86efa6c8fa2968401968ec +Subproject commit 5b35a301d160ede05e69ac1907cd10a4df1799c1 diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java index 87029aa94fd2..87c69688eb72 100644 --- a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java @@ -47,6 +47,7 @@ import accord.coordinate.Coordinations; import accord.coordinate.PrepareRecovery; import accord.coordinate.tracking.AbstractTracker; +import accord.primitives.RoutingKeys; import accord.utils.SortedListMap; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.EmptyIterators; @@ -74,8 +75,6 @@ import accord.local.CommandStores.LatentStoreSelector; import accord.local.Commands; import accord.local.DurableBefore; -import accord.local.LoadKeys; -import accord.local.LoadKeysFor; import accord.local.MaxConflicts; import accord.local.Node; import accord.local.PreLoadContext; @@ -143,8 +142,9 @@ import static accord.local.RedundantStatus.Property.LOCALLY_REDUNDANT; import static accord.local.RedundantStatus.Property.LOCALLY_SYNCED; import static accord.local.RedundantStatus.Property.LOCALLY_WITNESSED; +import static accord.local.RedundantStatus.Property.LOG_UNAVAILABLE; import static accord.local.RedundantStatus.Property.QUORUM_APPLIED; -import static accord.local.RedundantStatus.Property.PRE_BOOTSTRAP; +import static accord.local.RedundantStatus.Property.UNREADY; import static accord.local.RedundantStatus.Property.SHARD_APPLIED; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.String.format; @@ -365,9 +365,8 @@ public Partition getPartition(DecoratedKey partitionKey) TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); List cfks = new CopyOnWriteArrayList<>(); - PreLoadContext context = PreLoadContext.contextFor(key, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, "commands_for_key table query"); CommandStores commandStores = AccordService.instance().node().commandStores(); - AccordService.getBlocking(commandStores.forEach(context, key, Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { + AccordService.getBlocking(commandStores.forEach("commands_for_key table query", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { SafeCommandsForKey safeCfk = safeStore.get(key); CommandsForKey cfk = safeCfk.current(); if (cfk == null) @@ -475,9 +474,8 @@ public Partition getPartition(DecoratedKey partitionKey) TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); List cfks = new CopyOnWriteArrayList<>(); - PreLoadContext context = PreLoadContext.contextFor(key, LoadKeys.SYNC, LoadKeysFor.READ_WRITE, "commands_for_key_unmanaged table query"); CommandStores commandStores = AccordService.instance().node().commandStores(); - AccordService.getBlocking(commandStores.forEach(context, key, Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { + AccordService.getBlocking(commandStores.forEach("commands_for_key_unmanaged table query", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { SafeCommandsForKey safeCfk = safeStore.get(key); CommandsForKey cfk = safeCfk.current(); if (cfk == null) @@ -888,8 +886,9 @@ private RedundantBeforeTable() " locally_redundant 'TxnIdUtf8Type',\n" + " locally_synced 'TxnIdUtf8Type',\n" + " locally_witnessed 'TxnIdUtf8Type',\n" + - " pre_bootstrap 'TxnIdUtf8Type',\n" + - " stale_until_at_least 'TxnIdUtf8Type',\n" + + " log_unavailable 'TxnIdUtf8Type',\n" + + " unready 'TxnIdUtf8Type',\n" + + " stale_until 'TxnIdUtf8Type',\n" + " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, token_start)" + ')', UTF8Type.instance)); } @@ -923,8 +922,9 @@ public DataSet data() .column("locally_redundant", entry.maxBound(LOCALLY_REDUNDANT).toString()) .column("locally_synced", entry.maxBound(LOCALLY_SYNCED).toString()) .column("locally_witnessed", entry.maxBound(LOCALLY_WITNESSED).toString()) - .column("pre_bootstrap", entry.maxBound(PRE_BOOTSTRAP).toString()) - .column("stale_until_at_least", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); + .column("log_unavailable", entry.maxBound(LOG_UNAVAILABLE).toString()) + .column("unready", entry.maxBound(UNREADY).toString()) + .column("stale_until", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); return ds; }, dataSet, @@ -1188,7 +1188,7 @@ public Partition getPartition(DecoratedKey partitionKey) TxnId txnId = TxnId.parse(txnIdStr); List commands = new CopyOnWriteArrayList<>(); - AccordService.instance().node().commandStores().forEachCommandStore(store -> { + AccordService.instance().node().commandStores().forAllUnsafe(store -> { Command command = ((AccordCommandStore)store).loadCommand(txnId); if (command != null) commands.add(new Entry(store.id(), command)); @@ -1293,7 +1293,7 @@ public Partition getPartition(DecoratedKey partitionKey) TxnId txnId = TxnId.parse(txnIdStr); List entries = new ArrayList<>(); - AccordService.instance().node().commandStores().forEachCommandStore(store -> { + AccordService.instance().node().commandStores().forAllUnsafe(store -> { for (AccordJournal.DebugEntry e : ((AccordCommandStore)store).debugCommand(txnId)) entries.add(new Entry(store.id(), e.segment, e.position, e.builder)); }); diff --git a/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java b/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java index a47fff2302b3..c9f813022537 100644 --- a/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java +++ b/src/java/org/apache/cassandra/metrics/AccordSystemMetrics.java @@ -175,7 +175,7 @@ private synchronized void refreshHistograms() int nowSeconds = (int) (Clock.Global.currentTimeMillis() / 1000); SnapshotBuilder builder = new SnapshotBuilder(); - service.node().commandStores().forEachCommandStore(commandStore -> { + service.node().commandStores().forAllUnsafe(commandStore -> { DefaultProgressLog.ImmutableView view = ((DefaultProgressLog)commandStore.unsafeProgressLog()).immutableView(); builder.progressLogActive += view.activeCount(); builder.progressLogSize.increment(view.size()); diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java index c297ab191297..27e50157a3b5 100644 --- a/src/java/org/apache/cassandra/service/ActiveRepairService.java +++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java @@ -582,8 +582,7 @@ else if (range.intersects(toRepair)) { throw new IllegalArgumentException(String.format("Requested range %s intersects a local range (%s) " + "but is not fully contained in one; this would lead to " + - "imprecise repair. keyspace: %s", toRepair.toString(), - range.toString(), keyspaceName)); + "imprecise repair. keyspace: %s", toRepair, range, keyspaceName)); } } if (rangeSuperSet == null || !replicaSets.containsKey(rangeSuperSet)) diff --git a/src/java/org/apache/cassandra/service/Rebuild.java b/src/java/org/apache/cassandra/service/Rebuild.java index c7d40f08bf5b..56ad1f47ba88 100644 --- a/src/java/org/apache/cassandra/service/Rebuild.java +++ b/src/java/org/apache/cassandra/service/Rebuild.java @@ -33,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.dht.Range; @@ -160,7 +161,7 @@ else if (tokens == null) StreamResultFuture streamResult = streamer.fetchAsync(); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(streamResult, accordReady); // wait for result diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 8d861618a58f..84a61f6b2646 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -71,9 +71,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Uninterruptibles; - -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,6 +131,7 @@ import org.apache.cassandra.io.sstable.IScrubber; import org.apache.cassandra.io.sstable.IVerifier; import org.apache.cassandra.io.sstable.SSTableLoader; +import org.apache.cassandra.io.sstable.format.SSTableReader; import org.apache.cassandra.io.sstable.format.Version; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileUtils; @@ -158,7 +156,9 @@ import org.apache.cassandra.metrics.StorageMetrics; import org.apache.cassandra.net.MessagingService; import org.apache.cassandra.repair.RepairCoordinator; +import org.apache.cassandra.repair.RepairParallelism; import org.apache.cassandra.repair.SharedContext; +import org.apache.cassandra.repair.autorepair.AutoRepair; import org.apache.cassandra.repair.messages.RepairOption; import org.apache.cassandra.schema.CompactionParams.TombstoneOption; import org.apache.cassandra.schema.KeyspaceMetadata; @@ -183,6 +183,7 @@ import org.apache.cassandra.service.paxos.cleanup.PaxosCleanupLocalCoordinator; import org.apache.cassandra.service.paxos.cleanup.PaxosRepairState; import org.apache.cassandra.service.snapshot.SnapshotManager; +import org.apache.cassandra.streaming.PreviewKind; import org.apache.cassandra.streaming.StreamManager; import org.apache.cassandra.streaming.StreamResultFuture; import org.apache.cassandra.streaming.StreamState; @@ -206,9 +207,9 @@ import org.apache.cassandra.tcm.sequences.BootstrapAndReplace; import org.apache.cassandra.tcm.sequences.InProgressSequences; import org.apache.cassandra.tcm.sequences.SingleNodeSequences; +import org.apache.cassandra.tcm.transformations.AlterTopology; import org.apache.cassandra.tcm.transformations.Assassinate; import org.apache.cassandra.tcm.transformations.CancelInProgressSequence; -import org.apache.cassandra.tcm.transformations.AlterTopology; import org.apache.cassandra.tcm.transformations.Register; import org.apache.cassandra.tcm.transformations.Startup; import org.apache.cassandra.tcm.transformations.Unregister; @@ -3145,6 +3146,29 @@ private FutureTask createRepairTask(final int cmd, final String keyspace return new FutureTask<>(task); } + public RepairCoordinator repairAccordKeyspace(String keyspace, Collection> ranges) + { + int cmd = nextRepairCommand.incrementAndGet(); + RepairOption options = new RepairOption(RepairParallelism.PARALLEL, // parallelism + false, // primaryRange + false, // incremental + false, // trace + 5, // jobThreads + ranges, // ranges + true, // pullRepair + true, // forceRepair + PreviewKind.NONE, // previewKind + false, // optimiseStreams + true, // ignoreUnreplicatedKeyspaces + true, // repairData + false, // repairPaxos + true, // dontPurgeTombstones + false // repairAccord + ); + + return new RepairCoordinator(this, cmd, options, keyspace); + } + private void tryRepairPaxosForTopologyChange(String reason) { try diff --git a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java index cbf1fe49dc64..68fcaa8e4aec 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java +++ b/src/java/org/apache/cassandra/service/accord/AccordCommandStores.java @@ -37,9 +37,9 @@ import org.apache.cassandra.cache.CacheSize; import org.apache.cassandra.config.AccordSpec.QueueShardModel; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.accord.AccordExecutor.AccordExecutorFactory; -import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeState; import static org.apache.cassandra.config.AccordSpec.QueueShardModel.THREAD_PER_SHARD; import static org.apache.cassandra.config.DatabaseDescriptor.getAccordQueueShardCount; @@ -112,15 +112,6 @@ static Factory factory() }; } - @Override - protected boolean shouldBootstrap(Node node, Topology previous, Topology updated, Range range) - { - if (!super.shouldBootstrap(node, previous, updated, range)) - return false; - // we see new ranges when a new keyspace is added, so avoid bootstrap in these cases - return contains(previous, ((TokenKey) range.start()).table()); - } - @Override public SequentialAsyncExecutor someSequentialExecutor() { @@ -128,17 +119,6 @@ public SequentialAsyncExecutor someSequentialExecutor() return executors[idx].newSequentialExecutor(); } - private static boolean contains(Topology previous, TableId searchTable) - { - for (Range range : previous.ranges()) - { - TableId table = ((TokenKey) range.start()).table(); - if (table.equals(searchTable)) - return true; - } - return false; - } - public synchronized void setCapacity(long bytes) { cacheSize = bytes; diff --git a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java index 5dc0d19d0969..80e3296f5695 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordConfigurationService.java @@ -113,7 +113,7 @@ AsyncResult acknowledged() @Nullable AsyncResult reads() { - return reads; + return ready == null ? null : ready.reads; } AsyncResult.Settable localSyncNotified() @@ -449,7 +449,7 @@ public void reportTopology(Topology topology, boolean isLoad, boolean startSync) } @Override - protected void localSyncComplete(Topology topology, boolean startSync) + protected void onReadyToCoordinate(Topology topology, boolean startSync) { long epoch = topology.epoch(); EpochState epochState = getOrCreateEpochState(epoch); diff --git a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java index 0234184a383b..7934aebf769f 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordDataStore.java +++ b/src/java/org/apache/cassandra/service/accord/AccordDataStore.java @@ -30,6 +30,7 @@ import accord.primitives.Range; import accord.primitives.Ranges; import accord.primitives.SyncPoint; +import accord.utils.UnhandledEnum; import org.apache.cassandra.db.ColumnFamilyStore; import org.apache.cassandra.db.memtable.Memtable; import org.apache.cassandra.schema.Schema; @@ -40,14 +41,6 @@ public class AccordDataStore implements DataStore private static final Logger logger = LoggerFactory.getLogger(AccordDataStore.class); enum FlushListenerKey { KEY } - @Override - public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback) - { - AccordFetchCoordinator coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); - coordinator.start(); - return coordinator.result(); - } - /** * Ensures data for the intersecting ranges is flushed to sstable before calling back with reportOnSuccess. * This is used to gate journal cleanup, since we skip the CommitLog for applying to the data table. @@ -95,4 +88,23 @@ public void ensureDurable(CommandStore commandStore, Ranges ranges, RedundantBef prev = cfs; } } + + @Override + public FetchResult fetch(Node node, SafeCommandStore safeStore, Ranges ranges, SyncPoint syncPoint, FetchRanges callback, FetchKind kind) + { + switch (kind) + { + default: throw new UnhandledEnum(kind); + case Image: + { + AccordFetchCoordinator coordinator = new AccordFetchCoordinator(node, ranges, syncPoint, callback, safeStore.commandStore()); + coordinator.start(); + return coordinator.result(); + } + case Sync: + { + throw new UnsupportedOperationException(); + } + } + } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index 045acea8dfab..cfa224ab4945 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -585,7 +585,7 @@ public void forEach(Consumer consumer) @SuppressWarnings("unchecked") @Override - public void replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores) { // TODO (expected): make the parallelisms configurable // Replay is performed in parallel, where at most X commands can be in flight, accross at most Y commands stores. @@ -716,6 +716,7 @@ public void close() ++cur; } + return true; } catch (Throwable t) { diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 058b9171ac16..34b7dd28c4b0 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -39,6 +39,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.metrics.AccordReplicaMetrics; import org.apache.cassandra.service.accord.api.AccordViolationHandler; import org.apache.cassandra.utils.Clock; @@ -327,7 +328,7 @@ public synchronized static AccordService startup(NodeId tcmId) } @VisibleForTesting - public static void replayJournal(AccordService as) + public static boolean replayJournal(AccordService as) { logger.info("Starting journal replay."); long before = Clock.Global.nanoTime(); @@ -337,12 +338,12 @@ public static void replayJournal(AccordService as) if (as.journalConfiguration().replayMode() == RESET) AccordKeyspace.truncateCommandsForKey(); - as.node.commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop()); + as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); as.journal().replay(as.node().commandStores()); logger.info("Waiting for command stores to quiesce."); ((AccordCommandStores)as.node.commandStores()).waitForQuiescense(); as.journal.unsafeSetStarted(); - as.node.commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start()); + as.node.commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start()); } finally { @@ -351,14 +352,7 @@ public static void replayJournal(AccordService as) long after = Clock.Global.nanoTime(); logger.info("Finished journal replay. {}ms elapsed", NANOSECONDS.toMillis(after - before)); - } - - public static void shutdownServiceAndWait(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException - { - IAccordService i = instance; - if (i == null) - return; - i.shutdownAndWait(timeout, unit); + return true; } @Override @@ -565,7 +559,7 @@ public AsyncChain sync(Timestamp minBound, Keys keys, DurabilityService.Sy if (keys.size() != 1) return syncInternal(minBound, keys, syncLocal, syncRemote); - return KeyBarriers.find(node, minBound, keys.get(0).toUnseekable(), syncLocal, syncRemote) + return KeyBarriers.find(node, minBound, keys.get(0).toUnseekable(), syncLocal, syncRemote).chain() .flatMap(found -> KeyBarriers.await(node, node.someSequentialExecutor(), found, syncLocal, syncRemote)) .flatMap(success -> { if (success) @@ -799,8 +793,8 @@ class Ready extends AsyncResults.CountingResult implements Runnable } Ready ready = new Ready(); AccordCommandStores commandStores = (AccordCommandStores) node.commandStores(); - getBlocking(commandStores.forEach((PreLoadContext.Empty)() -> "Flush Caches", safeStore -> { - AccordCommandStore commandStore = (AccordCommandStore)safeStore.commandStore(); + commandStores.forAllUnsafe(unsafeStore -> { + AccordCommandStore commandStore = (AccordCommandStore)unsafeStore; try (AccordCommandStore.ExclusiveCaches caches = commandStore.lockCaches()) { caches.commandsForKeys().forEach(entry -> { @@ -811,7 +805,7 @@ class Ready extends AsyncResults.CountingResult implements Runnable } }); } - })); + }); ready.decrement(); AsyncPromise result = new AsyncPromise<>(); ready.invoke((success, fail) -> { @@ -1037,18 +1031,18 @@ public AccordJournal journal() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { - return toFuture(configService.epochReady(epoch.getEpoch())); + return toFuture(configService.epochReady(epoch.getEpoch(), get)); } @Override - public Future epochReadyFor(ClusterMetadata metadata) + public Future epochReadyFor(ClusterMetadata metadata, Function> get) { if (!metadata.schema.hasAccordKeyspaces()) return EPOCH_READY; - return epochReady(metadata.epoch); + return epochReady(metadata.epoch, get); } @Override @@ -1116,7 +1110,7 @@ public AccordConfigurationService configService() public AccordCompactionInfos getCompactionInfo() { AccordCompactionInfos compactionInfos = new AccordCompactionInfos(node.durableBefore(), node.topology().minEpoch()); - node.commandStores().forEachCommandStore(commandStore -> { + node.commandStores().forAllUnsafe(commandStore -> { compactionInfos.put(commandStore.id(), ((AccordCommandStore)commandStore).getCompactionInfo()); }); return compactionInfos; diff --git a/src/java/org/apache/cassandra/service/accord/AccordTopology.java b/src/java/org/apache/cassandra/service/accord/AccordTopology.java index ca817e647d57..8c60ecae5e3a 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordTopology.java +++ b/src/java/org/apache/cassandra/service/accord/AccordTopology.java @@ -35,6 +35,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import accord.api.ConfigurationService.EpochReady; import accord.local.Node; import accord.local.Node.Id; import accord.primitives.Ranges; @@ -399,7 +400,7 @@ public static void awaitTopologyReadiness(Keyspaces.KeyspacesDiff keyspacesDiff, { ClusterMetadataService.instance().fetchLogFromCMS(epoch); IAccordService service = AccordService.instance(); - service.epochReady(epoch).get(service.agent().expireEpochWait(MILLISECONDS), MILLISECONDS); + service.epochReady(epoch, EpochReady::reads).get(service.agent().expireEpochWait(MILLISECONDS), MILLISECONDS); } catch (InterruptedException e) { diff --git a/src/java/org/apache/cassandra/service/accord/IAccordService.java b/src/java/org/apache/cassandra/service/accord/IAccordService.java index 34453d22c30b..422155449bd9 100644 --- a/src/java/org/apache/cassandra/service/accord/IAccordService.java +++ b/src/java/org/apache/cassandra/service/accord/IAccordService.java @@ -25,9 +25,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; +import java.util.function.Function; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import accord.api.ConfigurationService.EpochReady; import accord.utils.async.AsyncResult; import org.apache.cassandra.tcm.ClusterMetadata; import org.slf4j.Logger; @@ -127,9 +129,8 @@ interface IAccordResult * Return a future that will complete once the accord has completed it's local bootstrap process * for any ranges gained in the given epoch */ - Future epochReady(Epoch epoch); - - Future epochReadyFor(ClusterMetadata epoch); + Future epochReady(Epoch epoch, Function> f); + Future epochReadyFor(ClusterMetadata epoch, Function> f); void receive(Message message); @@ -308,13 +309,13 @@ public AccordScheduler scheduler() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { return BOOTSTRAP_SUCCESS; } @Override - public Future epochReadyFor(ClusterMetadata epoch) + public Future epochReadyFor(ClusterMetadata epoch, Function> get) { return BOOTSTRAP_SUCCESS; } @@ -515,15 +516,15 @@ public AccordScheduler scheduler() } @Override - public Future epochReady(Epoch epoch) + public Future epochReady(Epoch epoch, Function> get) { - return delegate.epochReady(epoch); + return delegate.epochReady(epoch, get); } @Override - public Future epochReadyFor(ClusterMetadata epoch) + public Future epochReadyFor(ClusterMetadata epoch, Function> get) { - return delegate.epochReadyFor(epoch); + return delegate.epochReadyFor(epoch, get); } @Override diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java index be7153fc2c0c..e09a565c4130 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordAgent.java @@ -30,6 +30,7 @@ import accord.api.Agent; import accord.api.CoordinatorEventListener; +import accord.api.OwnershipEventListener; import accord.api.ReplicaEventListener; import accord.api.ProgressLog.BlockedUntil; import accord.api.RoutingKey; @@ -93,7 +94,7 @@ import static org.apache.cassandra.utils.Clock.Global.nanoTime; // TODO (expected): merge with AccordService -public class AccordAgent implements Agent +public class AccordAgent implements Agent, OwnershipEventListener { private static final Logger logger = LoggerFactory.getLogger(AccordAgent.class); private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, MINUTES); @@ -125,6 +126,12 @@ public AccordTracing tracing() return tracing.trace(txnId, eventType); } + @Override + public OwnershipEventListener ownershipEvents() + { + return this; + } + public void setNodeId(Node.Id id) { self = id; diff --git a/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java b/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java index 67ec0111a8fe..c40278932942 100644 --- a/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java +++ b/src/java/org/apache/cassandra/service/accord/api/AccordViolationHandler.java @@ -18,6 +18,8 @@ package org.apache.cassandra.service.accord.api; +import javax.annotation.Nullable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,8 +31,6 @@ import accord.primitives.Timestamp; import accord.primitives.TxnId; -import static accord.utils.Invariants.illegalState; - public class AccordViolationHandler implements ViolationHandler { private static final Logger logger = LoggerFactory.getLogger(AccordViolationHandler.class); @@ -40,13 +40,11 @@ public static void setup() ViolationHandlerHolder.set(AccordViolationHandler::new); } - @Override - public void onTimestampViolation(SafeCommandStore safeStore, Command command, Participants otherParticipants, Route otherRoute, Timestamp otherExecuteAt) + public void onTimestampViolation(@Nullable SafeCommandStore safeStore, Command command, Participants otherParticipants, @Nullable Route otherRoute, Timestamp otherExecuteAt) { - throw illegalState(ViolationHandler.timestampViolationMessage(safeStore, command, otherParticipants, otherRoute, otherExecuteAt)); + logger.error(ViolationHandler.timestampViolationMessage(safeStore, command, otherParticipants, otherRoute, otherExecuteAt)); } - @Override public void onDependencyViolation(Participants participants, TxnId notWitnessed, Timestamp notWitnessedExecuteAt, TxnId by, Timestamp byExecuteAt) { logger.error(ViolationHandler.dependencyViolationMessage(participants, notWitnessed, notWitnessedExecuteAt, by, byExecuteAt)); diff --git a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java index 5dcbb4552fc7..468f5ff41373 100644 --- a/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java +++ b/src/java/org/apache/cassandra/service/accord/interop/AccordInteropStableThenRead.java @@ -144,13 +144,13 @@ public AccordInteropStableThenRead(TxnId txnId, Participants scope, Commit.Ki } @Override - public CommitOrReadNack apply(SafeCommandStore safeStore) + public CommitOrReadNack applyInternal(SafeCommandStore safeStore) { Route route = this.route == null ? (Route)scope : this.route; StoreParticipants participants = StoreParticipants.execute(safeStore, route, txnId, minEpoch(), executeAtEpoch); SafeCommand safeCommand = safeStore.get(txnId, participants); Commands.commit(safeStore, safeCommand, participants, kind.saveStatus, Ballot.ZERO, txnId, route, partialTxn, executeAt, partialDeps, kind); - return super.apply(safeStore, safeCommand, participants); + return super.applyInternal(safeStore, safeCommand, participants); } @Override diff --git a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java index cd967961424d..53391cd322c5 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/AcceptSerializers.java @@ -88,7 +88,7 @@ public void serialize(Accept.NotAccept invalidate, DataOutputPlus out) throws IO CommandSerializers.status.serialize(invalidate.status, out); CommandSerializers.ballot.serialize(invalidate.ballot, out); CommandSerializers.txnId.serialize(invalidate.txnId, out); - KeySerializers.participants.serialize(invalidate.participants, out); + KeySerializers.participants.serialize(invalidate.scope, out); } @Override @@ -106,7 +106,7 @@ public long serializedSize(Accept.NotAccept invalidate) return CommandSerializers.status.serializedSize(invalidate.status) + CommandSerializers.ballot.serializedSize(invalidate.ballot) + CommandSerializers.txnId.serializedSize(invalidate.txnId) - + KeySerializers.participants.serializedSize(invalidate.participants); + + KeySerializers.participants.serializedSize(invalidate.scope); } }; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java index af67c05b0482..0e017184fb53 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/AwaitSerializers.java @@ -122,7 +122,7 @@ public long serializedSize(A await) public void serialize(AsyncAwaitComplete ok, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(ok.txnId, out); - KeySerializers.route.serialize(ok.route, out); + KeySerializers.route.serialize(ok.scope, out); out.writeByte(ok.newStatus.ordinal()); out.writeUnsignedVInt32(ok.callbackId); } @@ -141,7 +141,7 @@ public AsyncAwaitComplete deserialize(DataInputPlus in) throws IOException public long serializedSize(AsyncAwaitComplete ok) { return CommandSerializers.txnId.serializedSize(ok.txnId) - + KeySerializers.route.serializedSize(ok.route) + + KeySerializers.route.serializedSize(ok.scope) + TypeSizes.BYTE_SIZE + VIntCoding.computeVIntSize(ok.callbackId); } diff --git a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java index 25a40a6a418c..231c93287836 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/BeginInvalidationSerializers.java @@ -39,7 +39,7 @@ public class BeginInvalidationSerializers public void serialize(BeginInvalidation begin, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(begin.txnId, out); - KeySerializers.participants.serialize(begin.participants, out); + KeySerializers.participants.serialize(begin.scope, out); CommandSerializers.ballot.serialize(begin.ballot, out); } @@ -55,7 +55,7 @@ public BeginInvalidation deserialize(DataInputPlus in) throws IOException public long serializedSize(BeginInvalidation begin) { return CommandSerializers.txnId.serializedSize(begin.txnId) - + KeySerializers.participants.serializedSize(begin.participants) + + KeySerializers.participants.serializedSize(begin.scope) + CommandSerializers.ballot.serializedSize(begin.ballot); } }; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java index d1f81512b975..9747d9f72f4c 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CheckStatusSerializers.java @@ -128,8 +128,8 @@ public long serializedSize(KnownMap knownMap) public void serialize(CheckStatus check, DataOutputPlus out) throws IOException { CommandSerializers.txnId.serialize(check.txnId, out); - KeySerializers.participants.serialize(check.query, out); - out.writeUnsignedVInt(check.sourceEpoch); + KeySerializers.participants.serialize(check.scope, out); + out.writeUnsignedVInt(check.waitForEpoch); out.writeByte(check.includeInfo.ordinal()); CommandSerializers.ballot.serialize(check.bumpBallot, out); } @@ -149,8 +149,8 @@ public CheckStatus deserialize(DataInputPlus in) throws IOException public long serializedSize(CheckStatus check) { return CommandSerializers.txnId.serializedSize(check.txnId) - + KeySerializers.participants.serializedSize(check.query) - + TypeSizes.sizeofUnsignedVInt(check.sourceEpoch) + + KeySerializers.participants.serializedSize(check.scope) + + TypeSizes.sizeofUnsignedVInt(check.waitForEpoch) + TypeSizes.BYTE_SIZE + CommandSerializers.ballot.serializedSize(check.bumpBallot); } 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 a1cb244b13c6..a30ea5927056 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommandStoreSerializers.java @@ -155,11 +155,18 @@ public void serialize(RedundantBefore.Bounds b, DataOutputPlus out) throws IOExc } for (int i = 0 ; i < b.bounds.length ; ++i) { - out.writeShort(b.status(i * 2)); - out.writeShort(b.status(i * 2 + 1)); + out.writeShort(cast(b.status(i * 2))); + out.writeShort(cast(b.status(i * 2 + 1))); } } + private short cast(long v) + { + if ((v & ~0xFFFF) != 0) + throw new IllegalStateException("Cannot serialize RedundantStatus larger than 0xFFFF. Requires serialization version bump."); + return (short)v; + } + @Override public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException { @@ -174,7 +181,7 @@ public RedundantBefore.Bounds deserialize(DataInputPlus in) throws IOException TxnId[] bounds = new TxnId[count]; for (int i = 0 ; i < bounds.length ; ++i) bounds[i] = CommandSerializers.txnId.deserialize(in); - short[] statuses = new short[count * 2]; + int[] statuses = new int[count * 2]; for (int i = 0 ; i < statuses.length ; ++i) statuses[i] = in.readShort(); diff --git a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java index 31d879ce7c87..2d2592eee820 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/CommitSerializers.java @@ -52,9 +52,9 @@ public void serializeBody(Commit msg, DataOutputPlus out, Version version) throw kind.serialize(msg.kind, out); CommandSerializers.ballot.serialize(msg.ballot, out); ExecuteAtSerializer.serialize(msg.txnId, msg.executeAt, out); - CommandSerializers.nullablePartialTxn.serialize(msg.partialTxn, out, version); + CommandSerializers.nullablePartialTxn.serialize(msg.partialTxn(), out, version); if (msg.kind.withDeps == Commit.WithDeps.HasDeps) - DepsSerializers.partialDeps.serialize(msg.partialDeps, out); + DepsSerializers.partialDeps.serialize(msg.partialDeps(), out); serializeNullable(msg.route, out, KeySerializers.fullRoute); } @@ -78,10 +78,10 @@ public long serializedBodySize(Commit msg, Version version) long size = kind.serializedSize(msg.kind) + CommandSerializers.ballot.serializedSize(msg.ballot) + ExecuteAtSerializer.serializedSize(msg.txnId, msg.executeAt) - + CommandSerializers.nullablePartialTxn.serializedSize(msg.partialTxn, version); + + CommandSerializers.nullablePartialTxn.serializedSize(msg.partialTxn(), version); if (msg.kind.withDeps == Commit.WithDeps.HasDeps) - size += DepsSerializers.partialDeps.serializedSize(msg.partialDeps); + size += DepsSerializers.partialDeps.serializedSize(msg.partialDeps()); size += serializedNullableSize(msg.route, KeySerializers.fullRoute); return size; diff --git a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java index 3eb2fa498454..b3f81bec281c 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/RecoverySerializers.java @@ -56,7 +56,7 @@ public class RecoverySerializers static final int HAS_EXECUTE_AT_EPOCH = 0x2; static final int IS_FAST_PATH_DECIDED = 0x4; static final int SIZE_OF_FLAGS = VIntCoding.computeUnsignedVIntSize(HAS_ROUTE | HAS_EXECUTE_AT_EPOCH | IS_FAST_PATH_DECIDED); - public static final IVersionedSerializer request = new WithUnsyncedSerializer() + public static final IVersionedSerializer request = new WithUnsyncedSerializer<>() { @Override public void serializeBody(BeginRecovery recover, DataOutputPlus out, Version version) throws IOException diff --git a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java index fe2cbe26136b..37e5efdb8f13 100644 --- a/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java +++ b/src/java/org/apache/cassandra/service/accord/serializers/TxnRequestSerializer.java @@ -20,14 +20,14 @@ import java.io.IOException; -import accord.messages.TxnRequest; +import accord.messages.RouteRequest; import accord.primitives.Route; import accord.primitives.TxnId; import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; -public abstract class TxnRequestSerializer> implements IVersionedSerializer +public abstract class TxnRequestSerializer> implements IVersionedSerializer { void serializeHeader(T msg, DataOutputPlus out, Version version) throws IOException { @@ -72,7 +72,7 @@ public final long serializedSize(T msg, Version version) return serializedHeaderSize(msg, version) + serializedBodySize(msg, version); } - public static abstract class WithUnsyncedSerializer> extends TxnRequestSerializer + public static abstract class WithUnsyncedSerializer> extends TxnRequestSerializer { @Override void serializeHeader(T msg, DataOutputPlus out, Version version) throws IOException diff --git a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java index 341eff845bcc..a40a42dded9c 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java +++ b/src/java/org/apache/cassandra/tcm/sequences/BootstrapAndJoin.java @@ -28,6 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import com.googlecode.concurrenttrees.common.Iterables; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.db.ColumnFamilyStore; @@ -362,7 +363,7 @@ public static boolean bootstrap(final Collection tokens, StorageService.instance.repairPaxosForTopologyChange("bootstrap"); Future bootstrapStream = StorageService.instance.startBootstrap(metadata, beingReplaced, movements, strictMovements); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(bootstrapStream, accordReady); try diff --git a/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java b/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java index 68fc2471ac59..8a121fd2c2a2 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java +++ b/src/java/org/apache/cassandra/tcm/sequences/DropAccordTable.java @@ -26,6 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.io.util.DataInputPlus; import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.Keyspaces; @@ -168,7 +169,7 @@ private SequenceState awaitSafeFromAccord() throws ExecutionException, Interrupt return error(new IllegalStateException(String.format("Table %s is in an invalid state to be dropped", table))); long startNanos = nanoTime(); - AccordService.instance().epochReady(metadata.epoch).get(); + AccordService.instance().epochReady(metadata.epoch, EpochReady::reads).get(); long epochEndNanos = nanoTime(); // As of this writing this logic is based off ExclusiveSyncPoints which is a bit heavy weight for what is needed, this could cause timeouts for clusters that have a lot of data. diff --git a/src/java/org/apache/cassandra/tcm/sequences/Move.java b/src/java/org/apache/cassandra/tcm/sequences/Move.java index b54b796749fa..25058e20f2ae 100644 --- a/src/java/org/apache/cassandra/tcm/sequences/Move.java +++ b/src/java/org/apache/cassandra/tcm/sequences/Move.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import accord.api.ConfigurationService.EpochReady; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.SystemKeyspace; import org.apache.cassandra.db.TypeSizes; @@ -258,7 +259,7 @@ else if (destination.isSelf()) StreamResultFuture streamResult = streamPlan.execute(); - Future accordReady = AccordService.instance().epochReadyFor(metadata); + Future accordReady = AccordService.instance().epochReadyFor(metadata, EpochReady::reads); Future ready = FutureCombiner.allOf(streamResult, accordReady); ready.get(); StorageService.instance.repairPaxosForTopologyChange("move"); diff --git a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java index 51a551a19abd..a2d0429c3f66 100644 --- a/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java +++ b/test/distributed/org/apache/cassandra/distributed/shared/ClusterUtils.java @@ -49,6 +49,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; +import accord.api.ConfigurationService.EpochReady; import org.agrona.collections.IntArrayList; import org.apache.cassandra.tcm.compatibility.TokenRingUtils; import org.apache.cassandra.utils.FBUtilities; @@ -1698,7 +1699,7 @@ public static void awaitAccordEpochReady(Cluster cluster, long epoch) i.runOnInstance(() -> { try { - AccordService.instance().epochReady(Epoch.create(epoch)).get(); + AccordService.instance().epochReady(Epoch.create(epoch), EpochReady::reads).get(); } catch (InterruptedException | ExecutionException e) { diff --git a/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java b/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java index 7d4320aa01a0..6d91a592379a 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java +++ b/test/distributed/org/apache/cassandra/distributed/test/TestBaseImpl.java @@ -44,7 +44,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import accord.messages.AbstractRequest; +import accord.messages.NoWaitRequest; import net.openhft.chronicle.core.util.SerializablePredicate; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.cql3.CQLTester; @@ -109,8 +109,8 @@ protected static class MessageCountingSink implements IMessageSink // This isn't perfect at excluding messages so make sure it excludes the ones you care about in your test public static final SerializablePredicate> EXCLUDE_SYNC_POINT_MESSAGES = message -> { - if (message.payload instanceof AbstractRequest) - return !((AbstractRequest)message.payload).txnId.isSyncPoint(); + if (message.payload instanceof NoWaitRequest) + return !((NoWaitRequest)message.payload).txnId.isSyncPoint(); return true; }; @@ -251,7 +251,7 @@ public static String batch(boolean logged, String... queries) return sb.toString(); } - protected void bootstrapAndJoinNode(Cluster cluster) + protected IInvokableInstance bootstrapAndJoinNode(Cluster cluster) { IInstanceConfig config = cluster.newInstanceConfig(); config.set("auto_bootstrap", true); @@ -261,6 +261,7 @@ protected void bootstrapAndJoinNode(Cluster cluster) () -> newInstance.startup(cluster)); newInstance.nodetoolResult("join").asserts().success(); newInstance.nodetoolResult("cms", "describe").asserts().success(); // just make sure we're joined, remove later + return newInstance; } @SuppressWarnings("unchecked") 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 d30c3d887a39..49632ed1535d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordBootstrapTest.java @@ -24,13 +24,16 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import java.util.function.Function; import org.junit.Assert; import org.junit.Test; -import accord.local.PreLoadContext; +import accord.api.ConfigurationService.EpochReady; +import accord.primitives.RoutingKeys; import accord.primitives.Timestamp; import accord.topology.TopologyManager; +import accord.utils.async.AsyncResult; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.cql3.QueryProcessor; import org.apache.cassandra.cql3.UntypedResultSet; @@ -41,14 +44,15 @@ import org.apache.cassandra.distributed.Cluster; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableFunction; import org.apache.cassandra.distributed.api.TokenSupplier; import org.apache.cassandra.distributed.shared.NetworkTopology; import org.apache.cassandra.distributed.test.TestBaseImpl; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordConfigurationService; -import org.apache.cassandra.service.accord.AccordConfigurationService.EpochSnapshot; import org.apache.cassandra.service.accord.AccordSafeCommandStore; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.api.PartitionKey; @@ -66,6 +70,8 @@ import static org.apache.cassandra.distributed.api.Feature.GOSSIP; import static org.apache.cassandra.distributed.api.Feature.NETWORK; import static org.apache.cassandra.service.accord.AccordService.getBlocking; +import static org.apache.cassandra.service.accord.AccordConfigurationService.EpochSnapshot.ResultStatus.SUCCESS; +import static org.apache.cassandra.service.accord.AccordConfigurationService.SyncStatus.COMPLETED; public class AccordBootstrapTest extends TestBaseImpl { @@ -81,7 +87,7 @@ private static PartitionKey pk(int key, String keyspace, String table) return new PartitionKey(tid, dk(key)); } - protected void bootstrapAndJoinNode(Cluster cluster) + protected IInvokableInstance bootstrapAndJoinNode(Cluster cluster) { IInstanceConfig config = cluster.newInstanceConfig(); config.set("auto_bootstrap", true); @@ -94,6 +100,7 @@ protected void bootstrapAndJoinNode(Cluster cluster) // () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster))); // newInstance.nodetoolResult("join").asserts().success(); newInstance.nodetoolResult("cms", "describe").asserts().success(); // just make sure we're joined, remove later + return newInstance; } private static AccordService service() @@ -101,11 +108,11 @@ private static AccordService service() return (AccordService) AccordService.instance(); } - private static void awaitEpoch(long epoch) + private static void awaitEpoch(long epoch, Function> await) { try { - boolean completed = service().epochReady(Epoch.create(epoch)).await(60, TimeUnit.SECONDS); + boolean completed = service().epochReady(Epoch.create(epoch), await).await(60, TimeUnit.SECONDS); Assertions.assertThat(completed) .describedAs("Epoch %s did not become ready within timeout on %s -> %s", epoch, FBUtilities.getBroadcastAddressAndPort(), @@ -168,6 +175,14 @@ public synchronized void forSession(Consumer consumer) @Test public void bootstrapTest() throws Throwable + { + bootstrapTest(Function.identity(), cluster -> { + bootstrapAndJoinNode(cluster); + awaitMaxEpochReadyToRead(cluster); + }); + } + + public void bootstrapTest(Function setup, Consumer bootstrapAndJoinNode) throws Throwable { int originalNodeCount = 2; int expandedNodeCount = originalNodeCount + 1; @@ -188,49 +203,10 @@ public void bootstrapTest() 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 initialMax = maxEpoch(cluster); - + awaitMaxEpochReadyToRead(cluster); for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - Assert.assertEquals(initialMax, ClusterMetadata.current().epoch.getEpoch()); - awaitEpoch(initialMax); - AccordConfigurationService configService = service().configService(); - long minEpoch = configService.minEpoch(); - - Assert.assertEquals(initialMax, configService.maxEpoch()); - - for (long epoch = minEpoch; epoch < initialMax; epoch++) - { - awaitEpoch(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - - awaitLocalSyncNotification(initialMax); - Assert.assertEquals(EpochSnapshot.completed(initialMax), configService.getEpochSnapshot(initialMax)); - }); - } - - for (IInvokableInstance node : cluster) - { node.runOnInstance(StreamListener::register); - } - - long schemaChangeMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(schemaChangeMax)); - awaitEpoch(schemaChangeMax); - AccordConfigurationService configService = service().configService(); - - for (long epoch = initialMax + 1; epoch <= schemaChangeMax; epoch++) - { - awaitLocalSyncNotification(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - }); - } + awaitMaxEpochReadyToRead(cluster); for (int key = 0; key < 100; key++) { @@ -251,21 +227,7 @@ public void bootstrapTest() throws Throwable }); } - bootstrapAndJoinNode(cluster); - long bootstrapMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(bootstrapMax)); - Assert.assertEquals(bootstrapMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(bootstrapMax); - AccordConfigurationService configService = service.configService(); - - awaitLocalSyncNotification(bootstrapMax); - Assert.assertEquals(EpochSnapshot.completed(bootstrapMax), configService.getEpochSnapshot(bootstrapMax)); - }); - } + bootstrapAndJoinNode.accept(cluster); InetAddress node3Addr = cluster.get(3).broadcastAddress().getAddress(); for (IInvokableInstance node : cluster.get(1, 2)) @@ -278,15 +240,11 @@ public void bootstrapTest() throws Throwable Assert.assertTrue(session.getNumKeyspaceTransfers() > 0); }); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", safeStore -> { - AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; - Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.bootstrapBeganAt().keySet())); - Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.safeToReadAt().keySet())); -// -// Assert.assertTrue(commandStore.maxBootstrapEpoch() > 0); -// Assert.assertTrue(commandStore.bootstrapBeganAt().isEmpty()); -// Assert.assertTrue(commandStore.safeToRead().isEmpty()); - })); + service().node().commandStores().forAllUnsafe(unsafeStore -> { + AccordCommandStore ss = (AccordCommandStore) unsafeStore; + Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.unsafeGetBootstrapBeganAt().keySet())); + Assert.assertEquals(Timestamp.NONE, getOnlyElement(ss.unsafeGetSafeToRead().keySet())); + }); }); } @@ -321,7 +279,7 @@ public void bootstrapTest() throws Throwable Assert.assertEquals(key, row.getInt("c")); Assert.assertEquals(key, row.getInt("v")); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", safeStore -> { + getBlocking(service().node().commandStores().forEach("Test", RoutingKeys.of(partitionKey.toUnseekable()), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { if (safeStore.ranges().currentRanges().contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; @@ -375,44 +333,7 @@ public void moveTest() throws Throwable tokens[i] = cluster.get(i+1).callOnInstance(() -> Long.valueOf(getOnlyElement(StorageService.instance.getTokens()))); } - for (IInvokableInstance node : cluster) - { - - node.runOnInstance(() -> { - Assert.assertEquals(initialMax, ClusterMetadata.current().epoch.getEpoch()); - awaitEpoch(initialMax); - AccordConfigurationService configService = service().configService(); - long minEpoch = configService.minEpoch(); - - Assert.assertEquals(initialMax, configService.maxEpoch()); - - for (long epoch = minEpoch; epoch < initialMax; epoch++) - { - awaitEpoch(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - - awaitLocalSyncNotification(initialMax); - Assert.assertEquals(EpochSnapshot.completed(initialMax), configService.getEpochSnapshot(initialMax)); - }); - } - - long schemaChangeMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - Assert.assertEquals(schemaChangeMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(schemaChangeMax); - AccordConfigurationService configService = service.configService(); - - for (long epoch = initialMax + 1; epoch <= schemaChangeMax; epoch++) - { - awaitLocalSyncNotification(epoch); - Assert.assertEquals(EpochSnapshot.completed(epoch), configService.getEpochSnapshot(epoch)); - } - }); - } + awaitMaxEpochReadyToRead(cluster); for (int key = 0; key < 100; key++) { @@ -431,20 +352,7 @@ public void moveTest() throws Throwable cluster.get(1).runOnInstance(() -> StorageService.instance.move(Long.toString(token))); - long moveMax = maxEpoch(cluster); - for (IInvokableInstance node : cluster) - { - node.runOnInstance(() -> { - ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(moveMax)); - Assert.assertEquals(moveMax, ClusterMetadata.current().epoch.getEpoch()); - AccordService service = (AccordService) AccordService.instance(); - awaitEpoch(moveMax); - AccordConfigurationService configService = service.configService(); - - awaitLocalSyncNotification(moveMax); - Assert.assertEquals(EpochSnapshot.completed(moveMax), configService.getEpochSnapshot(moveMax)); - }); - } + long moveMax = awaitMaxEpochReadyToRead(cluster); for (IInvokableInstance node : cluster) { @@ -464,9 +372,7 @@ public void moveTest() throws Throwable PartitionKey partitionKey = new PartitionKey(tableId, dk); - getBlocking(service().node().commandStores().forEach((PreLoadContext.Empty)()->"Test", - partitionKey.toUnseekable(), moveMax, moveMax, - safeStore -> { + getBlocking(service().node().commandStores().forEach("Test", RoutingKeys.of(partitionKey.toUnseekable()), moveMax, moveMax, safeStore -> { if (!safeStore.ranges().allAt(preMove).contains(partitionKey)) { AccordSafeCommandStore ss = (AccordSafeCommandStore) safeStore; @@ -493,4 +399,41 @@ public void moveTest() throws Throwable } } } + + private static long awaitMaxEpochReadyToRead(Cluster cluster) + { + return awaitMaxEpoch(cluster, EpochReady::reads, true); + } + + private static long awaitMaxEpochMetadataReady(Cluster cluster) + { + return awaitMaxEpoch(cluster, EpochReady::metadata, false); + } + + private static long awaitMaxEpoch(Cluster cluster, SerializableFunction> await, boolean expectReadyToRead) + { + long maxEpoch = maxEpoch(cluster); + for (IInvokableInstance node : cluster) + { + node.acceptOnInstance(aw -> { + ClusterMetadataService.instance().fetchLogFromCMS(Epoch.create(maxEpoch)); + Assert.assertEquals(maxEpoch, ClusterMetadata.current().epoch.getEpoch()); + AccordService service = (AccordService) AccordService.instance(); + awaitEpoch(maxEpoch, aw); + AccordConfigurationService configService = service.configService(); + + awaitLocalSyncNotification(maxEpoch); + for (long epoch = configService.minEpoch(); epoch <= maxEpoch; epoch++) + { + Assert.assertEquals(COMPLETED, configService.getEpochSnapshot(maxEpoch).syncStatus); + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).acknowledged); + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).received); + if (expectReadyToRead) + Assert.assertEquals(SUCCESS, configService.getEpochSnapshot(maxEpoch).reads); + } + }, node.transfer(await)); + } + return maxEpoch; + } + } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java index a5a38edadf74..bbb8a7f4cb4d 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIncrementalRepairTest.java @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import accord.local.Node; -import accord.local.PreLoadContext; import accord.local.SafeCommand; import accord.local.StoreParticipants; import accord.local.cfk.CommandsForKey; @@ -32,6 +31,7 @@ import accord.local.durability.DurabilityService; import accord.primitives.Keys; import accord.primitives.Ranges; +import accord.primitives.RoutingKeys; import accord.primitives.Status; import accord.primitives.Timestamp; import accord.primitives.TxnId; @@ -68,8 +68,6 @@ import org.slf4j.LoggerFactory; -import static accord.local.LoadKeys.SYNC; -import static accord.local.LoadKeysFor.READ_WRITE; import static java.lang.String.format; import static org.apache.cassandra.distributed.test.accord.AccordTestBase.executeWithRetry; import static org.apache.cassandra.service.accord.AccordService.getBlocking; @@ -158,7 +156,7 @@ private static void withCluster(ModelChecker.ThrowingConsumer run) thro { cluster.filters().reset(); for (IInvokableInstance instance : cluster) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start())); } } } @@ -207,7 +205,7 @@ private static TxnId awaitLocalApplyOnKey(TokenKey key) { Node node = accordService().node(); AtomicReference waitFor = new AtomicReference<>(null); - getBlocking(node.commandStores().ifLocal(PreLoadContext.contextFor(key, SYNC, READ_WRITE, "Test"), key.toUnseekable(), 0, Long.MAX_VALUE, safeStore -> { + getBlocking(node.commandStores().forEach("Test", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { AccordSafeCommandStore store = (AccordSafeCommandStore) safeStore; SafeCommandsForKey safeCfk = store.ifLoadedAndInitialised(key); if (safeCfk == null) @@ -229,7 +227,7 @@ private static TxnId awaitLocalApplyOnKey(TokenKey key) long now = Clock.Global.currentTimeMillis(); if (now - start > TimeUnit.MINUTES.toMillis(1)) throw new AssertionError("Timeout"); - AccordService.getBlocking(node.commandStores().ifLocal(PreLoadContext.contextFor(txnId, "Test"), key.toUnseekable(), 0, Long.MAX_VALUE, safeStore -> { + getBlocking(node.commandStores().forEach("Test", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { SafeCommand command = safeStore.get(txnId, StoreParticipants.empty(txnId)); Assert.assertNotNull(command.current()); if (command.current().status().hasBeen(Status.Applied)) @@ -291,7 +289,7 @@ public void txnRepairTest(Cluster cluster) throws Throwable // heal partition and wait for node 1 to see node 3 again for (IInvokableInstance instance : cluster) instance.runOnInstance(() -> { - AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop()); + AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop()); Assert.assertFalse(barrierRecordingService().executedBarriers); }); cluster.filters().reset(); diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java index c2b8f1e48618..a7354e17b7cf 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordIntegrationTest.java @@ -125,6 +125,6 @@ public void testLostCommitReadTriggersFallbackRead() throws Exception private void pauseSimpleProgressLog() { for (IInvokableInstance instance : SHARED_CLUSTER) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().stop())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().stop())); } } diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java index b4c53b6159b9..cb91e9a160e1 100644 --- a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java @@ -166,7 +166,7 @@ public void tearDown() throws Exception { SHARED_CLUSTER.filters().reset(); for (IInvokableInstance instance : SHARED_CLUSTER) - instance.runOnInstance(() -> AccordService.instance().node().commandStores().forEachCommandStore(cs -> cs.unsafeProgressLog().start())); + instance.runOnInstance(() -> AccordService.instance().node().commandStores().forAllUnsafe(cs -> cs.unsafeProgressLog().start())); truncateSystemTables(); diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java index 1e311bf8a541..b6aeb6231d56 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/AccordBounceTest.java @@ -150,9 +150,9 @@ public String get() // Command Stores should not be lost on bounce Map> before = cluster.get(1).callOnInstance(() -> { Map> m = new HashMap<>(); - AccordService.instance().node().commandStores().forEach((store, ranges) -> { + AccordService.instance().node().commandStores().forAllUnsafe((store) -> { Set set = new HashSet<>(); - for (Range range : ranges.all()) + for (Range range : store.unsafeGetRangesForEpoch().all()) set.add(range.toString()); m.put(store.id(), set); }); @@ -169,9 +169,9 @@ public String get() Map> after = cluster.get(1).callOnInstance(() -> { Map> m = new HashMap<>(); - AccordService.instance().node().commandStores().forEach((store, ranges) -> { + AccordService.instance().node().commandStores().forAllUnsafe(store -> { Set set = new HashSet<>(); - for (Range range : ranges.all()) + for (Range range : store.unsafeGetRangesForEpoch().all()) set.add(range.toString()); m.put(store.id(), set); }); diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java index 25b7b952c888..42db56fb9494 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java @@ -354,11 +354,11 @@ private TreeMap read(CommandStores commandStores) } @Override - public void replay(CommandStores commandStores) + public boolean replay(CommandStores commandStores) { // Make sure to replay _only_ static segments this.closeCurrentSegmentForTestingIfNonEmpty(); - super.replay(commandStores); + return super.replay(commandStores); } @Override @@ -388,7 +388,7 @@ public PersistentField.Persister durableBeforePers public static IAccordService.AccordCompactionInfos getCompactionInfo(Node node, TableId tableId) { IAccordService.AccordCompactionInfos compactionInfos = new IAccordService.AccordCompactionInfos(node.durableBefore(), node.topology().minEpoch()); - node.commandStores().forEachCommandStore(commandStore -> { + node.commandStores().forAllUnsafe(commandStore -> { RedundantBefore redundantBefore = commandStore.unsafeGetRedundantBefore(); if (redundantBefore == null) redundantBefore = RedundantBefore.EMPTY; diff --git a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java index 7f7e44fd9f64..024e343ed7f7 100644 --- a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java @@ -34,8 +34,7 @@ import org.slf4j.LoggerFactory; import accord.api.ProtocolModifiers; -import accord.local.PreLoadContext; -import accord.messages.TxnRequest; +import accord.messages.NoWaitRequest; import accord.primitives.Ranges; import accord.primitives.Routable; import accord.primitives.SaveStatus; @@ -212,7 +211,7 @@ public void redundantBefore() throws ExecutionException, InterruptedException TxnId syncId2 = new TxnId(101, 300, Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, accord.nodeId()); Ranges ranges1 = Ranges.of(TokenRange.create(new TokenKey(tableId, new LongToken(1)), new TokenKey(tableId, new LongToken(100)))); Ranges ranges2 = Ranges.of(TokenRange.create(new TokenKey(tableId, new LongToken(100)), new TokenKey(tableId, new LongToken(200)))); - getBlocking(accord.node().commandStores().forEach((PreLoadContext.Empty)() -> "Test", safeStore -> { + getBlocking(accord.node().commandStores().forAll("Test", safeStore -> { safeStore.commandStore().markShardDurable(safeStore, syncId1, ranges1, HasOutcome.Universal); safeStore.commandStore().markShardDurable(safeStore, syncId2, ranges2, HasOutcome.Quorum); })); @@ -436,9 +435,9 @@ public boolean test(Message msg, InetAddressAndPort to) if (!msg.verb().name().startsWith("ACCORD_")) return true; TxnId txnId = null; - if (msg.payload instanceof TxnRequest) + if (msg.payload instanceof NoWaitRequest) { - txnId = ((TxnRequest) msg.payload).txnId; + txnId = ((NoWaitRequest) msg.payload).txnId; if (applyTo != null && !applyTo.contains(txnId)) return true; } diff --git a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java index d0c2d87f1eba..b467174c42be 100644 --- a/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java +++ b/test/unit/org/apache/cassandra/index/accord/RouteIndexTest.java @@ -30,6 +30,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import javax.annotation.Nullable; + +import accord.api.ConfigurationService.EpochReady; import accord.api.Journal; import accord.api.RoutingKey; import accord.local.CommandStores; @@ -238,7 +240,7 @@ private InsertTxn(int storeId, TxnId txnId, SaveStatus saveStatus, Route rout this.storeId = storeId; this.txnId = txnId; this.saveStatus = saveStatus; - this.participants = StoreParticipants.all(route); + this.participants = StoreParticipants.all(route, saveStatus); } @Override @@ -503,7 +505,7 @@ public State(RandomSource rs) storeRangesForEpochs.put(i, new RangesForEpoch(1, Ranges.of(TokenRange.fullRange(tableId, getPartitioner())))); accordService = startAccord(); - accordService.epochReady(ClusterMetadata.current().epoch).awaitUninterruptibly(); + accordService.epochReady(ClusterMetadata.current().epoch, EpochReady::reads).awaitUninterruptibly(); minDecidedIdNull = rs.nextFloat(); txnWriteFrequency = rs.pickInt(1, // every txn is a Write diff --git a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java index b714b833f1a8..ac322403d15d 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordCommandTest.java @@ -176,7 +176,7 @@ public void basicCycleTest() throws Throwable Command before = safeStore.ifInitialised(txnId).current(); Assert.assertEquals(commit.executeAt, before.executeAt()); Assert.assertTrue(before.hasBeen(Status.Committed)); - Assert.assertEquals(commit.partialDeps, before.partialDeps()); + Assert.assertEquals(commit.partialDeps(), before.partialDeps()); CommandsForKey cfk = safeStore.get(key(1).toUnseekable()).current(); Assert.assertTrue(cfk.indexOf(txnId) >= 0); diff --git a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java b/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java index adaff55578d4..081aaeec473e 100644 --- a/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java +++ b/test/unit/org/apache/cassandra/service/accord/AccordSyncPropagatorTest.java @@ -41,7 +41,7 @@ import org.junit.Test; import accord.api.Agent; -import accord.impl.AbstractConfigurationService; +import accord.impl.AbstractTestConfigurationService; import accord.impl.TestAgent; import accord.impl.basic.Pending; import accord.impl.basic.PendingQueue; @@ -410,7 +410,7 @@ public void unregisterFailureDetectionEventListener(IFailureDetectionEventListen } } - private class ConfigService extends AbstractConfigurationService.Minimal implements AccordSyncPropagator.Listener + private class ConfigService extends AbstractTestConfigurationService implements AccordSyncPropagator.Listener { private final Map> syncCompletes = new HashMap<>(); private final Map> endpointAcks = new HashMap<>(); @@ -436,7 +436,7 @@ public void fetchTopologyForEpoch(long epoch) } @Override - protected void localSyncComplete(Topology topology, boolean startSync) + protected void onReadyToCoordinate(Topology topology, boolean startSync) { Set notify = topology.nodes().stream().filter(i -> !localId.equals(i)).collect(Collectors.toSet()); instances.get(localId).propagator.reportSyncComplete(topology.epoch(), notify, localId); diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java index b9602dec07fc..d73763ba5aac 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordCommandStore.java @@ -59,7 +59,7 @@ import accord.messages.BeginRecovery; import accord.messages.PreAccept; import accord.messages.Reply; -import accord.messages.TxnRequest; +import accord.messages.RouteRequest; import accord.primitives.AbstractUnseekableKeys; import accord.primitives.Ballot; import accord.primitives.EpochSupplier; @@ -421,9 +421,9 @@ public void checkFailures() throw error; } - public T process(TxnRequest request) throws ExecutionException, InterruptedException + public T process(RouteRequest request) throws ExecutionException, InterruptedException { - return process(request, request::apply); + return process(request, request); } public T process(PreLoadContext loadCtx, Function function) throws ExecutionException, InterruptedException @@ -433,9 +433,9 @@ public T process(PreLoadContext loadCtx, Function AsyncResult processAsync(TxnRequest request) + public AsyncResult processAsync(RouteRequest request) { - return processAsync(request, request::apply); + return processAsync(request, request); } public AsyncResult processAsync(PreLoadContext loadCtx, Function function) diff --git a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java index d54058b72703..f18065adc96f 100644 --- a/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java +++ b/test/unit/org/apache/cassandra/service/accord/SimulatedAccordTaskTest.java @@ -120,9 +120,9 @@ private static void test(RandomSource rs, int numSamples, TableMetadata tbl, Gen FullRoute route = txnWithRoute.right; PreAccept preAccept = new PreAccept(nodeId, instance.topologies, txnId, txn, null, false, route) { @Override - public PreAcceptReply apply(SafeCommandStore safeStore) + public PreAcceptReply applyInternal(SafeCommandStore safeStore) { - PreAcceptReply result = super.apply(safeStore); + PreAcceptReply result = super.applyInternal(safeStore); if (action == Action.FAILURE) throw new SimulatedFault("PreAccept failed for keys " + keys()); return result; diff --git a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java index 884638da98dc..edca7e1c6b0c 100644 --- a/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java +++ b/test/unit/org/apache/cassandra/service/accord/serializers/CommandsForKeySerializerTest.java @@ -28,6 +28,7 @@ import java.util.Random; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; import java.util.function.Consumer; @@ -49,6 +50,7 @@ import accord.api.DataStore; import accord.api.Journal; import accord.api.Key; +import accord.api.OwnershipEventListener; import accord.api.ProgressLog; import accord.api.RoutingKey; import accord.api.Timeouts; @@ -657,8 +659,8 @@ protected TestCommandStore() @Override public Agent agent() { return this; } @Override public void execute(Runnable run) {} @Override public void shutdown() { } - @Override public void onFailedBootstrap(int attempts, String phase, Ranges ranges, Runnable retry, Throwable failure) { throw new UnsupportedOperationException(); } - @Override public void onStale(Timestamp staleSince, Ranges ranges) { throw new UnsupportedOperationException(); } + @Override public AsyncChain chain(Callable call) { throw new UnsupportedOperationException(); } + @Override public OwnershipEventListener ownershipEvents() { return null; } @Override public void onUncaughtException(Throwable t) { throw new UnsupportedOperationException(); } @Override public void onCaughtException(Throwable t, String context) { throw new UnsupportedOperationException(); } @Override public boolean rejectPreAccept(TimeService time, TxnId txnId) { throw new UnsupportedOperationException(); } diff --git a/test/unit/org/apache/cassandra/utils/AccordGenerators.java b/test/unit/org/apache/cassandra/utils/AccordGenerators.java index bd994fc27a10..2c8f292b6814 100644 --- a/test/unit/org/apache/cassandra/utils/AccordGenerators.java +++ b/test/unit/org/apache/cassandra/utils/AccordGenerators.java @@ -90,7 +90,7 @@ import static accord.local.CommandStores.RangesForEpoch; import static accord.local.RedundantStatus.Property.GC_BEFORE; -import static accord.local.RedundantStatus.Property.PRE_BOOTSTRAP; +import static accord.local.RedundantStatus.Property.UNREADY; import static accord.local.RedundantStatus.SomeStatus.LOCALLY_APPLIED_ONLY; import static accord.local.RedundantStatus.SomeStatus.LOCALLY_WITNESSED_ONLY; import static accord.local.RedundantStatus.SomeStatus.SHARD_APPLIED_ONLY; @@ -276,7 +276,7 @@ private ICommand attributes(SaveStatus saveStatus) if (saveStatus.known.deps().hasPreAcceptedOrProposedOrDecidedDeps()) builder.partialDeps(partialDeps); - builder.setParticipants(StoreParticipants.all(route)); + builder.setParticipants(StoreParticipants.all(route, saveStatus)); builder.durability(NotDurable); if (saveStatus.compareTo(SaveStatus.PreAccepted) >= 0) builder.executeAt(executeAt); @@ -601,9 +601,9 @@ public static Gen redundantBeforeEntry(Gen emptyGen, Gen if (rs.nextBoolean()) bounds.add(Bounds.create(range, txnIdGen.next(rs).addFlag(SHARD_BOUND), oneSlow(GC_BEFORE), null )); if (rs.nextBoolean()) - bounds.add(Bounds.create(range, txnIdGen.next(rs), oneSlow(PRE_BOOTSTRAP), null )); + bounds.add(Bounds.create(range, txnIdGen.next(rs), oneSlow(UNREADY), null )); if (rs.nextBoolean()) - bounds.add(new Bounds(range, Long.MIN_VALUE, Long.MAX_VALUE, new TxnId[0], new short[0], txnIdGen.next(rs))); + bounds.add(new Bounds(range, Long.MIN_VALUE, Long.MAX_VALUE, new TxnId[0], new int[0], txnIdGen.next(rs))); Collections.shuffle(bounds); long endEpoch = emptyGen.next(rs) ? Long.MAX_VALUE : rs.nextLong(0, Long.MAX_VALUE); @@ -618,7 +618,7 @@ public static Gen redundantBeforeEntry(Gen emptyGen, Gen } long startEpoch = rs.nextLong(Math.min(minEpoch, endEpoch)); - Bounds epochBounds = new Bounds(range, startEpoch, endEpoch, new TxnId[0], new short[0], null); + Bounds epochBounds = new Bounds(range, startEpoch, endEpoch, new TxnId[0], new int[0], null); if (result == null) return epochBounds; return Bounds.reduce(result, epochBounds); From add004bbea271d6da6e1443858fde22ea9057159 Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Mon, 8 Sep 2025 14:42:32 +0100 Subject: [PATCH 2/6] Lazy Virtual Tables Also Improve: - Searchable system_accord_debug.txn patch by Benedict; reviewed by Aleksey Yeschenko for CASSANDRA-20899 --- .../restrictions/StatementRestrictions.java | 12 +- .../cql3/statements/SelectStatement.java | 4 +- .../db/PartitionRangeReadCommand.java | 2 +- .../db/SinglePartitionReadCommand.java | 2 +- .../db/virtual/AbstractLazyVirtualTable.java | 653 ++++++++++++++++++ .../db/virtual/AbstractVirtualTable.java | 5 +- .../db/virtual/AccordDebugKeyspace.java | 521 +++++++------- .../CollectionVirtualTableAdapter.java | 5 +- .../db/virtual/PartitionKeyStatsTable.java | 5 +- .../cassandra/db/virtual/VirtualTable.java | 22 +- .../cassandra/journal/InMemoryIndex.java | 11 + .../org/apache/cassandra/journal/Journal.java | 87 ++- .../org/apache/cassandra/journal/Segment.java | 3 +- .../cassandra/journal/StaticSegment.java | 6 + .../service/accord/AccordExecutor.java | 5 +- .../service/accord/AccordJournal.java | 14 +- .../service/accord/AccordJournalTable.java | 11 +- .../tools/StandaloneJournalUtil.java | 2 +- .../fuzz/topology/JournalGCTest.java | 4 +- .../service/accord/AccordJournalBurnTest.java | 2 +- .../db/virtual/AccordDebugKeyspaceTest.java | 46 ++ 21 files changed, 1100 insertions(+), 322 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java index 3b40d7e4123a..8373f3eb8877 100644 --- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java +++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java @@ -156,13 +156,13 @@ public static StatementRestrictions empty(StatementType type, TableMetadata tabl return new StatementRestrictions(type, table, IndexHints.NONE, false); } - private StatementRestrictions(StatementType type, TableMetadata table, IndexHints indexHints, boolean allowFiltering) + private StatementRestrictions(StatementType type, TableMetadata table, IndexHints indexHints, boolean allowFilteringOfPrimaryKeys) { this.type = type; this.table = table; this.indexHints = indexHints; this.partitionKeyRestrictions = new PartitionKeyRestrictions(table.partitionKeyAsClusteringComparator()); - this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFiltering); + this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFilteringOfPrimaryKeys); this.nonPrimaryKeyRestrictions = RestrictionSet.empty(); this.notNullColumns = new HashSet<>(); } @@ -370,7 +370,7 @@ else if (operator.requiresIndexing()) } else { - if (!allowFiltering && requiresAllowFilteringIfNotSpecified(table)) + if (!allowFiltering && requiresAllowFilteringIfNotSpecified(table, false)) throw invalidRequest(allowFilteringMessage(state)); } @@ -381,14 +381,14 @@ else if (operator.requiresIndexing()) validateSecondaryIndexSelections(); } - public static boolean requiresAllowFilteringIfNotSpecified(TableMetadata metadata) + public static boolean requiresAllowFilteringIfNotSpecified(TableMetadata metadata, boolean isPrimaryKey) { if (!metadata.isVirtual()) return true; VirtualTable tableNullable = VirtualKeyspaceRegistry.instance.getTableNullable(metadata.id); assert tableNullable != null; - return !tableNullable.allowFilteringImplicitly(); + return isPrimaryKey ? !tableNullable.allowFilteringPrimaryKeysImplicitly() : !tableNullable.allowFilteringImplicitly(); } private void addRestriction(Restriction restriction, IndexRegistry indexRegistry, IndexHints indexHints) @@ -593,7 +593,7 @@ private void processPartitionKeyRestrictions(ClientState state, boolean hasQueri // components must have a EQ. Only the last partition key component can be in IN relation. if (partitionKeyRestrictions.needFiltering()) { - if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified(table)) + if (!allowFiltering && !forView && !hasQueriableIndex && requiresAllowFilteringIfNotSpecified(table, true)) throw new InvalidRequestException(allowFilteringMessage(state)); isKeyRange = true; diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index ecdce1437b1f..aa5500356d02 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1474,7 +1474,7 @@ private StatementRestrictions prepareRestrictions(ClientState state, boundNames, orderings, selectsOnlyStaticColumns, - parameters.allowFiltering || !requiresAllowFilteringIfNotSpecified(metadata), + parameters.allowFiltering || !requiresAllowFilteringIfNotSpecified(metadata, true), forView); } @@ -1700,7 +1700,7 @@ private void checkNeedsFiltering(TableMetadata table, StatementRestrictions rest { // We will potentially filter data if the row filter is not the identity and there isn't any index group // supporting all the expressions in the filter. - if (requiresAllowFilteringIfNotSpecified(table)) + if (requiresAllowFilteringIfNotSpecified(table, true)) checkFalse(restrictions.needFiltering(table), StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE); } } diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java index 25986314a278..f448bcf8724d 100644 --- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java +++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java @@ -647,7 +647,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter(), rowFilter()); + UnfilteredPartitionIterator resultIterator = view.select(dataRange, columnFilter(), rowFilter(), limits()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index 7c4864e6db2d..780a8c083f59 100644 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@ -1516,7 +1516,7 @@ public PartitionIterator execute(ConsistencyLevel consistency, ClientState state public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController) { VirtualTable view = VirtualKeyspaceRegistry.instance.getTableNullable(metadata().id); - UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter(), rowFilter()); + UnfilteredPartitionIterator resultIterator = view.select(partitionKey, clusteringIndexFilter, columnFilter(), rowFilter(), limits()); return limits().filter(rowFilter().filter(resultIterator, nowInSec()), nowInSec(), selectsFullPartition()); } diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java new file mode 100644 index 000000000000..9f98cd52d73b --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java @@ -0,0 +1,653 @@ +/* + * 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.db.virtual; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.UnaryOperator; + +import javax.annotation.Nullable; + +import accord.utils.Invariants; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.Clusterable; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.BufferCell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Bounds; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.ReadTimeoutException; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.BulkIterator; +import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.btree.UpdateFunction; + +import static org.apache.cassandra.db.ClusteringPrefix.Kind.STATIC_CLUSTERING; +import static org.apache.cassandra.utils.Clock.Global.nanoTime; + +/** + * An abstract virtual table implementation that builds the resultset on demand. + */ +public abstract class AbstractLazyVirtualTable implements VirtualTable +{ + // in the special case where we know we have enough rows in the collector, throw this exception to terminate early + static class InternalDoneException extends RuntimeException {} + + public static class FilterRange + { + final V min, max; + public FilterRange(V min, V max) + { + this.min = min; + this.max = max; + } + } + + public interface PartitionsCollector + { + DataRange dataRange(); + RowFilter rowFilter(); + RowCollector row(Object... primaryKeys); + PartitionCollector partition(Object... partitionKeys); + UnfilteredPartitionIterator finish(); + + default @Nullable DecoratedKey singleKey() + { + AbstractBounds bounds = dataRange().keyRange(); + if (!bounds.isStartInclusive() || !bounds.isEndInclusive() || !bounds.left.equals(bounds.right) || !(bounds.left instanceof DecoratedKey)) + return null; + + return (DecoratedKey) bounds.left; + } + + FilterRange filters(String column, Function translate, UnaryOperator increment, UnaryOperator decrement); + } + + public interface PartitionCollector + { + RowCollector row(Object... clusteringKeys); + } + + public interface RowCollector + { + default PartitionsCollector lazyAdd(Consumer addToIfNeeded) { return eagerAdd(addToIfNeeded); } + PartitionsCollector eagerAdd(Consumer addToNow); + } + + public interface ColumnsCollector + { + ColumnsCollector add(String columnName, V value, Function transform); + default ColumnsCollector add(String columnName, Object value) { return add(columnName, value, Function.identity()); } + } + + public static class SimplePartitionsCollector implements PartitionsCollector + { + final TableMetadata metadata; + final boolean isSorted; + + final Map columnLookup = new HashMap<>(); + final NavigableMap partitions; + + final DataRange dataRange; + final ColumnFilter columnFilter; + final RowFilter rowFilter; + final DataLimits limits; + + final long startedAt = Clock.Global.nanoTime(); + final long timeoutAt; + + final long nowInSeconds = Clock.Global.nowInSeconds(); + final long timestamp; + + int totalRowCount; + int lastFilteredTotalRowCount; + + @Override public DataRange dataRange() { return dataRange; } + @Override public RowFilter rowFilter() { return rowFilter; } + public ColumnFilter columnFilter() { return columnFilter; } + + public SimplePartitionsCollector(TableMetadata metadata, boolean isSorted, DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + this.metadata = metadata; + this.isSorted = isSorted; + this.dataRange = dataRange; + this.columnFilter = columnFilter; + this.rowFilter = rowFilter; + this.limits = limits; + this.timestamp = FBUtilities.timestampMicros(); + this.timeoutAt = startedAt + DatabaseDescriptor.getReadRpcTimeout(TimeUnit.NANOSECONDS); + this.partitions = new TreeMap<>(dataRange.isReversed() ? DecoratedKey.comparator.reversed() : DecoratedKey.comparator); + for (ColumnMetadata cm : metadata.columns()) + columnLookup.put(cm.name.toString(), cm); + } + + @Override + public PartitionCollector partition(Object ... partitionKeys) + { + int pkSize = metadata.partitionKeyColumns().size(); + if (pkSize != partitionKeys.length) + throw new IllegalArgumentException(); + + DecoratedKey partitionKey = makeDecoratedKey(partitionKeys); + if (!dataRange.contains(partitionKey)) + return dropCks -> dropRow -> this; + + return partitions.computeIfAbsent(partitionKey, SimplePartition::new); + } + + @Override + public UnfilteredPartitionIterator finish() + { + final Iterator partitions = this.partitions.values().iterator(); + return new UnfilteredPartitionIterator() + { + @Override public TableMetadata metadata() { return metadata; } + @Override public void close() {} + + @Override + public boolean hasNext() + { + return partitions.hasNext(); + } + + @Override + public UnfilteredRowIterator next() + { + SimplePartition partition = partitions.next(); + Iterator rows = partition.rows(); + + return new UnfilteredRowIterator() + { + @Override public TableMetadata metadata() { return metadata; } + @Override public boolean isReverseOrder() { return dataRange.isReversed(); } + @Override public RegularAndStaticColumns columns() { return columnFilter.fetchedColumns(); } + @Override public DecoratedKey partitionKey() { return partition.key; } + + @Override public Row staticRow() { return partition.staticRow(); } + @Override public boolean hasNext() { return rows.hasNext(); } + @Override public Unfiltered next() { return rows.next(); } + + @Override public void close() {} + @Override public DeletionTime partitionLevelDeletion() { return DeletionTime.LIVE; } + @Override public EncodingStats stats() { return EncodingStats.NO_STATS; } + }; + } + }; + } + + @Override + @Nullable + public FilterRange filters(String columnName, Function translate, UnaryOperator increment, UnaryOperator decrement) + { + ColumnMetadata column = columnLookup.get(columnName); + O min = null, max = null; + for (RowFilter.Expression expression : rowFilter().getExpressions()) + { + if (!expression.column().equals(column)) + continue; + + if (expression.isCustom()) + continue; + + O bound = translate.apply((I)column.type.compose(expression.getIndexValue())); + switch (expression.operator()) + { + default: continue; + case EQ: min = max = bound; break; + case LTE: max = bound; break; + case LT: max = decrement.apply(bound); break; + case GTE: min = bound; break; + case GT: min = increment.apply(bound); break; + } + } + + return new FilterRange<>(min, max); + } + + @Override + public RowCollector row(Object... primaryKeys) + { + int pkSize = metadata.partitionKeyColumns().size(); + int ckSize = metadata.clusteringColumns().size(); + if (pkSize + ckSize != primaryKeys.length) + throw new IllegalArgumentException(); + + Object[] partitionKeyValues = new Object[pkSize]; + Object[] clusteringValues = new Object[ckSize]; + + System.arraycopy(primaryKeys, 0, partitionKeyValues, 0, pkSize); + System.arraycopy(primaryKeys, pkSize, clusteringValues, 0, ckSize); + + DecoratedKey partitionKey = makeDecoratedKey(partitionKeyValues); + Clustering clustering = makeClustering(clusteringValues); + + if (!dataRange.contains(partitionKey) || !dataRange.clusteringIndexFilter(partitionKey).selects(clustering)) + return drop -> this; + + return partitions.computeIfAbsent(partitionKey, SimplePartition::new).row(); + } + + private DecoratedKey makeDecoratedKey(Object... partitionKeyValues) + { + ByteBuffer partitionKey = partitionKeyValues.length == 1 + ? decompose(metadata.partitionKeyType, partitionKeyValues[0]) + : ((CompositeType) metadata.partitionKeyType).decompose(partitionKeyValues); + return metadata.partitioner.decorateKey(partitionKey); + } + + private Clustering makeClustering(Object... clusteringValues) + { + if (clusteringValues.length == 0) + return Clustering.EMPTY; + + ByteBuffer[] clusteringByteBuffers = new ByteBuffer[clusteringValues.length]; + for (int i = 0; i < clusteringValues.length; i++) + clusteringByteBuffers[i] = decompose(metadata.clusteringColumns().get(i).type, clusteringValues[i]); + return Clustering.make(clusteringByteBuffers); + } + + private final class SimplePartition implements PartitionCollector + { + private final DecoratedKey key; + // we assume no duplicate rows, and impose the condition lazily + private SimpleRow[] rows; + private int rowCount; + private SimpleRow staticRow; + private boolean dropRows; + + private SimplePartition(DecoratedKey key) + { + this.key = key; + this.rows = new SimpleRow[1]; + } + + @Override + public RowCollector row(Object... clusteringKeys) + { + int ckSize = metadata.clusteringColumns().size(); + if (ckSize != clusteringKeys.length) + throw new IllegalArgumentException(); + + return row(makeClustering(clusteringKeys)); + } + + RowCollector row(Clustering clustering) + { + if (nanoTime() > timeoutAt) + throw new ReadTimeoutException(ConsistencyLevel.ONE, 0, 1, false); + + if (dropRows || !dataRange.clusteringIndexFilter(key).selects(clustering)) + return drop -> SimplePartitionsCollector.this; + + if (totalRowCount >= limits.count()) + { + boolean filter; + if (!isSorted) filter = totalRowCount / 2 >= Math.max(1024, limits.count()); + else + { + int rowsAddedSinceLastFiltered = totalRowCount - lastFilteredTotalRowCount; + int threshold = Math.max(32, Math.min(1024, lastFilteredTotalRowCount / 2)); + filter = lastFilteredTotalRowCount == 0 || rowsAddedSinceLastFiltered >= threshold; + } + + if (filter) + { + // first filter within each partition + for (SimplePartition partition : partitions.values()) + { + int curCount = partition.rowCount; + int newCount = Math.min(curCount, limits.perPartitionCount()); + newCount = partition.filterAndSortAndTruncate(newCount); + totalRowCount -= curCount - newCount; + } + + // then drop any partitions that completely fall outside our limit + Iterator iter = partitions.descendingMap().values().iterator(); + SimplePartition last; + while (true) + { + SimplePartition next = last = iter.next(); + if (totalRowCount - next.rowCount < limits.count()) + break; + + iter.remove(); + totalRowCount -= next.rowCount; + if (next == this) + dropRows = true; + } + + // possibly truncate the last partition if it partially falls outside the limit + int overflow = Math.max(0, totalRowCount - limits.count()); + int curCount = last.rowCount; + int newCount = curCount - overflow; + newCount = last.filterAndSortAndTruncate(newCount); + totalRowCount -= curCount - newCount; + lastFilteredTotalRowCount = totalRowCount; + + if (isSorted && totalRowCount >= limits.count()) + throw new InternalDoneException(); + + if (dropRows) + return drop -> SimplePartitionsCollector.this; + } + } + + SimpleRow result = new SimpleRow(clustering); + if (clustering.kind() == STATIC_CLUSTERING) + { + Invariants.require(staticRow == null); + staticRow = result; + } + else + { + totalRowCount++; + if (rowCount == rows.length) + rows = Arrays.copyOf(rows, Math.max(8, rowCount * 2)); + rows[rowCount++] = result; + } + return result; + } + + void filterAndSort() + { + int newCount = 0; + for (int i = 0 ; i < rowCount; ++i) + { + if (rows[i].rowFilterIncludes()) + { + if (newCount != i) + rows[newCount] = rows[i]; + newCount++; + } + } + totalRowCount -= (rowCount - newCount); + Arrays.fill(rows, newCount, rowCount, null); + rowCount = newCount; + Arrays.sort(rows, 0, newCount, rowComparator()); + } + + int filterAndSortAndTruncate(int newCount) + { + Invariants.requireArgument(newCount <= rowCount); + filterAndSort(); + if (rowCount < newCount) + return rowCount; + + Arrays.fill(rows, newCount, rowCount, null); + rowCount = newCount; + return newCount; + } + + private Comparator rowComparator() + { + Comparator cmp = dataRange.isReversed() ? metadata.comparator.reversed() : metadata.comparator; + return (a, b) -> cmp.compare(a.clustering, b.clustering); + } + + Row staticRow() + { + if (staticRow == null) + return null; + + return staticRow.materialiseAndFilter(); + } + + Iterator rows() + { + filterAndSort(); + return Arrays.stream(rows, 0, rowCount).map(SimpleRow::materialiseAndFilter).iterator(); + } + + private final class SimpleRow implements RowCollector + { + final Clustering clustering; + SomeColumns state; + + private SimpleRow(Clustering clustering) + { + this.clustering = clustering; + } + + @Override + public PartitionsCollector lazyAdd(Consumer addToIfNeeded) + { + Invariants.require(state == null); + state = new LazyColumnsCollector(addToIfNeeded); + return SimplePartitionsCollector.this; + } + + @Override + public PartitionsCollector eagerAdd(Consumer addToNow) + { + Invariants.require(state == null); + state = new EagerColumnsCollector(addToNow); + return SimplePartitionsCollector.this; + } + + boolean rowFilterIncludes() + { + return null != materialiseAndFilter(); + } + + Row materialiseAndFilter() + { + if (state == null) + return null; + + FilteredRow filtered = state.materialiseAndFilter(this); + state = filtered; + return filtered == null ? null : filtered.row; + } + + DecoratedKey partitionKey() + { + return SimplePartition.this.key; + } + + SimplePartitionsCollector collector() + { + return SimplePartitionsCollector.this; + } + } + } + + static abstract class SomeColumns + { + abstract FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent); + } + + static class LazyColumnsCollector extends SomeColumns + { + final Consumer lazy; + LazyColumnsCollector(Consumer lazy) + { + this.lazy = lazy; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + return parent.collector().new EagerColumnsCollector(lazy).materialiseAndFilter(parent); + } + } + + class EagerColumnsCollector extends SomeColumns implements ColumnsCollector + { + Object[] columns = new Object[4]; + int columnCount; + + public EagerColumnsCollector(Consumer add) + { + add.accept(this); + } + + @Override + public ColumnsCollector add(String name, V input, Function f) + { + ColumnMetadata cm = columnLookup.get(name); + if (!columnFilter.fetches(cm)) + return this; + + Object value = f.apply(input); + if (value == null) + return this; + + if (columnCount * 2 == columns.length) + columns = Arrays.copyOf(columns, columnCount * 4); + + columns[columnCount * 2] = cm; + columns[columnCount * 2 + 1] = value; + ++columnCount; + return this; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + for (int i = 0 ; i < columnCount ; i++) + { + ColumnMetadata cm = (ColumnMetadata) columns[i * 2]; + columns[i] = BufferCell.live(cm, timestamp, decompose(cm.type, columns[i * 2 + 1])); + } + Arrays.sort(columns, 0, columnCount, (a, b) -> ColumnData.comparator.compare((BufferCell)a, (BufferCell)b)); + Object[] btree = BTree.build(BulkIterator.of(columns), columnCount, UpdateFunction.noOp); + BTreeRow row = BTreeRow.create(parent.clustering, LivenessInfo.EMPTY, Row.Deletion.LIVE, btree); + if (!rowFilter.isSatisfiedBy(metadata, parent.partitionKey(), row, nowInSeconds)) + return null; + return new FilteredRow(row); + } + } + + static class FilteredRow extends SomeColumns + { + final Row row; + FilteredRow(Row row) + { + this.row = row; + } + + @Override + FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) + { + return this; + } + } + } + + // TODO (expected): add e.g. BOTH_ASC_DESC when some vtable supports it + public enum Sorted { UNSORTED, ASC, DESC } + + protected final TableMetadata metadata; + private final Sorted sorted; + + protected AbstractLazyVirtualTable(TableMetadata metadata, Sorted sorted) + { + if (!metadata.isVirtual()) + throw new IllegalArgumentException("Cannot instantiate a non-virtual table"); + + this.metadata = metadata; + this.sorted = sorted; + } + + @Override + public TableMetadata metadata() + { + return metadata; + } + + protected PartitionsCollector collector(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + boolean isSorted = sorted == (dataRange.isReversed() ? Sorted.DESC : Sorted.ASC); + return new SimplePartitionsCollector(metadata, isSorted, dataRange, columnFilter, rowFilter, limits); + } + + protected abstract void collect(PartitionsCollector collector); + + @Override + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + return select(new DataRange(new Bounds<>(partitionKey, partitionKey), clusteringIndexFilter), columnFilter, rowFilter, limits); + } + + @Override + public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + { + PartitionsCollector collector = collector(dataRange, columnFilter, rowFilter, limits); + try + { + collect(collector); + } + catch (InternalDoneException ignore) {} + return collector.finish(); + } + + @Override + public void apply(PartitionUpdate update) + { + throw new InvalidRequestException("Modification is not supported by table " + metadata); + } + + @Override + public void truncate() + { + throw new InvalidRequestException("Truncation is not supported by table " + metadata); + } + + @Override + public String toString() + { + return metadata().toString(); + } + + private static ByteBuffer decompose(AbstractType type, Object value) + { + return type.decomposeUntyped(value); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java index a32ea67ab69f..2c4a04e5a0f3 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractVirtualTable.java @@ -29,6 +29,7 @@ import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator; import org.apache.cassandra.db.partitions.PartitionUpdate; @@ -76,7 +77,7 @@ public DataSet data(DecoratedKey partitionKey) } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { Partition partition = data(partitionKey).getPartition(partitionKey); @@ -89,7 +90,7 @@ public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringI } @Override - public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { DataSet data = data(); diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java index 87c69688eb72..84a0129ca679 100644 --- a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java @@ -34,6 +34,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -51,9 +52,12 @@ import accord.utils.SortedListMap; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.db.EmptyIterators; +import org.apache.cassandra.db.PartitionPosition; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.CompositeType; import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; import org.apache.cassandra.db.rows.UnfilteredRowIterator; @@ -126,6 +130,8 @@ import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.AccordTracing; import org.apache.cassandra.service.accord.CommandStoreTxnBlockedGraph; +import org.apache.cassandra.service.accord.IAccordService; +import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.api.AccordAgent; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.consensus.migration.ConsensusMigrationState; @@ -202,7 +208,7 @@ private AccordDebugKeyspace() } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class ExecutorsTable extends AbstractVirtualTable + public static final class ExecutorsTable extends AbstractLazyVirtualTable { private ExecutorsTable() { @@ -221,43 +227,43 @@ private ExecutorsTable() " keys_loading text,\n" + " keys_loading_for text,\n" + " PRIMARY KEY (executor_id, status, position, unique_position)" + - ')', UTF8Type.instance)); + ')', Int32Type.instance), Sorted.ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { AccordCommandStores commandStores = (AccordCommandStores) AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); - + // TODO (desired): we can easily also support sorted collection for DESC queries for (AccordExecutor executor : commandStores.executors()) { int uniquePos = 0; int executorId = executor.executorId(); AccordExecutor.TaskInfo prev = null; + PartitionCollector partition = collector.partition(executorId); for (AccordExecutor.TaskInfo info : executor.taskSnapshot()) { if (prev != null && info.status() == prev.status() && info.position() == prev.position()) ++uniquePos; else uniquePos = 0; prev = info; PreLoadContext preLoadContext = info.preLoadContext(); - ds.row(executorId, Objects.toString(info.status()), info.position(), uniquePos) - .column("description", info.describe()) - .column("command_store_id", info.commandStoreId()) - .column("txn_id", preLoadContext == null ? null : toStringOrNull(preLoadContext.primaryTxnId())) - .column("txn_id_additional", preLoadContext == null ? null : toStringOrNull(preLoadContext.additionalTxnId())) - .column("keys", preLoadContext == null ? null : toStringOrNull(preLoadContext.keys())) - .column("keys_loading", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeys())) - .column("keys_loading_for", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeysFor())) - ; + partition.row(info.status().name(), info.position(), uniquePos) + .lazyAdd(columns -> { + columns.add("description", info.describe()) + .add("command_store_id", info.commandStoreId()) + .add("txn_id", preLoadContext == null ? null : toStringOrNull(preLoadContext.primaryTxnId())) + .add("txn_id_additional", preLoadContext == null ? null : toStringOrNull(preLoadContext.additionalTxnId())) + .add("keys", preLoadContext == null ? null : toStringOrNull(preLoadContext.keys())) + .add("keys_loading", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeys())) + .add("keys_loading_for", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeysFor())); + }); } } - return ds; } } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class CoordinationsTable extends AbstractVirtualTable + public static final class CoordinationsTable extends AbstractLazyVirtualTable { private CoordinationsTable() { @@ -275,26 +281,26 @@ private CoordinationsTable() " replies text,\n" + " tracker text,\n" + " PRIMARY KEY (txn_id, kind, coordination_id)" + - ')', UTF8Type.instance)); + ')', UTF8Type.instance), Sorted.UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { Coordinations coordinations = AccordService.instance().node().coordinations(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (Coordination c : coordinations) { - ds.row(toStringOrNull(c.txnId()), c.kind().toString(), c.coordinationId()) - .column("nodes", toStringOrNull(c.nodes())) - .column("nodes_inflight", toStringOrNull(c.inflight())) - .column("nodes_contacted", toStringOrNull(c.contacted())) - .column("description", c.describe()) - .column("participants", toStringOrNull(c.scope())) - .column("replies", summarise(c.replies())) - .column("tracker", summarise(c.tracker())); + collector.row(toStringOrNull(c.txnId()), c.kind().toString(), c.coordinationId()) + .lazyAdd(columns -> { + columns.add("nodes", toStringOrNull(c.nodes())) + .add("nodes_inflight", toStringOrNull(c.inflight())) + .add("nodes_contacted", toStringOrNull(c.contacted())) + .add("description", c.describe()) + .add("participants", toStringOrNull(c.scope())) + .add("replies", summarise(c.replies())) + .add("tracker", summarise(c.tracker())); + }); } - return ds; } private static String summarise(@Nullable SortedListMap replies) @@ -313,8 +319,7 @@ private static String summarise(@Nullable AbstractTracker tracker) } - // TODO (desired): don't report null as "null" - public static final class CommandsForKeyTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + public static final class CommandsForKeyTable extends AbstractLazyVirtualTable { static class Entry { @@ -343,24 +348,16 @@ private CommandsForKeyTable() " status text,\n" + " status_overrides text,\n" + " PRIMARY KEY (key, command_store_id, txn_id)" + - ')', UTF8Type.instance)); + ')', UTF8Type.instance), Sorted.ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { - return this; - } + DecoratedKey partitionKey = collector.singleKey(); + if (partitionKey == null) + throw new InvalidRequestException("This table currently only supports querying single partitions"); - @Override - public boolean isEmpty() - { - return false; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) - { String keyStr = UTF8Type.instance.compose(partitionKey.getKey()); TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); @@ -376,33 +373,27 @@ public Partition getPartition(DecoratedKey partitionKey) })); if (cfks.isEmpty()) - return null; + return; - SimpleDataSet ds = new SimpleDataSet(metadata); for (Entry e : cfks) { CommandsForKey cfk = e.cfk; + PartitionCollector partition = collector.partition(keyStr); for (int i = 0 ; i < cfk.size() ; ++i) { CommandsForKey.TxnInfo txn = cfk.get(i); - ds.row(keyStr, e.commandStoreId, toStringOrNull(txn.plainTxnId())) - .column("ballot", toStringOrNull(txn.ballot())) - .column("deps_known_before", toStringOrNull(txn.depsKnownUntilExecuteAt())) - .column("flags", flags(txn)) - .column("execute_at", toStringOrNull(txn.plainExecuteAt())) - .column("missing", Arrays.toString(txn.missing())) - .column("status", toStringOrNull(txn.status())) - .column("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); + partition.row(e.commandStoreId, txn.plainTxnId().toString()) + .lazyAdd(columns -> { + columns.add("ballot", toStringOrNull(txn.ballot())) + .add("deps_known_before", toStringOrNull(txn.depsKnownUntilExecuteAt())) + .add("flags", flags(txn)) + .add("execute_at", toStringOrNull(txn.plainExecuteAt())) + .add("missing", Arrays.toString(txn.missing())) + .add("status", toStringOrNull(txn.status())) + .add("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); + }); } } - - return ds.getPartition(partitionKey); - } - - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); } private static String flags(CommandsForKey.TxnInfo txn) @@ -426,8 +417,6 @@ private static String flags(CommandsForKey.TxnInfo txn) } } - - // TODO (expected): test this table public static final class CommandsForKeyUnmanagedTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet { static class Entry @@ -510,16 +499,14 @@ public Iterator getPartitions(DataRange range) } } - - public static final class DurabilityServiceTable extends AbstractVirtualTable + public static final class DurabilityServiceTable extends AbstractLazyVirtualTable { private DurabilityServiceTable() { super(parse(VIRTUAL_ACCORD_DEBUG, DURABILITY_SERVICE, "Accord per-Range Durability Service State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " table_id text,\n" + " token_start 'TokenUtf8Type',\n" + " token_end 'TokenUtf8Type',\n" + " last_started_at bigint,\n" + @@ -538,78 +525,73 @@ private DurabilityServiceTable() " current_splits int,\n" + " stopping boolean,\n" + " stopped boolean,\n" + - " PRIMARY KEY (keyspace_name, table_name, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (table_id, token_start)" + + ')', UTF8Type.instance), Sorted.UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { ShardDurability.ImmutableView view = ((AccordService) AccordService.instance()).shardDurability(); - SimpleDataSet ds = new SimpleDataSet(metadata()); while (view.advance()) { TableId tableId = (TableId) view.shard().range.start().prefix(); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), printToken(view.shard().range.start())) - .column("token_end", printToken(view.shard().range.end())) - .column("last_started_at", approxTime.translate().toMillisSinceEpoch(view.lastStartedAtMicros() * 1000)) - .column("cycle_started_at", approxTime.translate().toMillisSinceEpoch(view.cycleStartedAtMicros() * 1000)) - .column("retries", view.retries()) - .column("min", Objects.toString(view.min())) - .column("requested_by", Objects.toString(view.requestedBy())) - .column("active", Objects.toString(view.active())) - .column("waiting", Objects.toString(view.waiting())) - .column("node_offset", view.nodeOffset()) - .column("cycle_offset", view.cycleOffset()) - .column("active_index", view.activeIndex()) - .column("next_index", view.nextIndex()) - .column("next_to_index", view.toIndex()) - .column("end_index", view.cycleLength()) - .column("current_splits", view.currentSplits()) - .column("stopping", view.stopping()) - .column("stopped", view.stopped()) - ; + collector.row(tableId.toString(), printToken(view.shard().range.start())) + .eagerAdd(columns -> { + columns.add("token_end", printToken(view.shard().range.end())) + .add("last_started_at", approxTime.translate().toMillisSinceEpoch(view.lastStartedAtMicros() * 1000)) + .add("cycle_started_at", approxTime.translate().toMillisSinceEpoch(view.cycleStartedAtMicros() * 1000)) + .add("retries", view.retries()) + .add("min", Objects.toString(view.min())) + .add("requested_by", Objects.toString(view.requestedBy())) + .add("active", Objects.toString(view.active())) + .add("waiting", Objects.toString(view.waiting())) + .add("node_offset", view.nodeOffset()) + .add("cycle_offset", view.cycleOffset()) + .add("active_index", view.activeIndex()) + .add("next_index", view.nextIndex()) + .add("next_to_index", view.toIndex()) + .add("end_index", view.cycleLength()) + .add("current_splits", view.currentSplits()) + .add("stopping", view.stopping()) + .add("stopped", view.stopped()); + }); } - return ds; } } - public static final class DurableBeforeTable extends AbstractVirtualTable + public static final class DurableBeforeTable extends AbstractLazyVirtualTable { private DurableBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, DURABLE_BEFORE, "Accord Node's DurableBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " table_id text,\n" + " token_start 'TokenUtf8Type',\n" + " token_end 'TokenUtf8Type',\n" + " quorum 'TxnIdUtf8Type',\n" + " universal 'TxnIdUtf8Type',\n" + - " PRIMARY KEY (keyspace_name, table_name, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (table_id, token_start)" + + ')', UTF8Type.instance), Sorted.UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { DurableBefore durableBefore = AccordService.instance().node().durableBefore(); - return durableBefore.foldlWithBounds( - (entry, ds, start, end) -> { + durableBefore.foldlWithBounds( + (entry, ignore, start, end) -> { TableId tableId = (TableId) start.prefix(); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), printToken(start)) - .column("token_end", printToken(end)) - .column("quorum", entry.quorumBefore.toString()) - .column("universal", entry.universalBefore.toString()); - return ds; - }, - new SimpleDataSet(metadata()), - ignore -> false - ); + collector.row(tableId.toString(), printToken(start)) + .lazyAdd(columns -> { + columns.add("token_end", printToken(end)) + .add("quorum", entry.quorumBefore.toString()) + .add("universal", entry.universalBefore.toString()); + }); + return null; + }, null, ignore -> false); } } @@ -655,48 +637,47 @@ private static void addRow(SimpleDataSet ds, int executorId, String scope, Accor } - public static final class MaxConflictsTable extends AbstractVirtualTable + public static final class MaxConflictsTable extends AbstractLazyVirtualTable { private MaxConflictsTable() { super(parse(VIRTUAL_ACCORD_DEBUG, MAX_CONFLICTS, "Accord per-CommandStore MaxConflicts State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + " command_store_id bigint,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + " timestamp text,\n" + - " PRIMARY KEY (keyspace_name, table_name, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, token_start)" + + ')', Int32Type.instance), Sorted.ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { int commandStoreId = commandStore.id(); MaxConflicts maxConflicts = commandStore.unsafeGetMaxConflicts(); TableId tableId = ((AccordCommandStore) commandStore).tableId(); - TableMetadata tableMetadata = tableMetadata(tableId); + String tableIdStr = tableId.toString(); + PartitionCollector partition = collector.partition(commandStoreId); maxConflicts.foldlWithBounds( - (timestamp, ds, start, end) -> { - return ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), commandStoreId, printToken(start)) - .column("token_end", printToken(end)) - .column("timestamp", timestamp.toString()) - ; - }, - dataSet, - ignore -> false + (timestamp, p, start, end) -> { + p.row(printToken(start)) + .lazyAdd(columns -> { + columns.add("token_end", printToken(end)) + .add("table_id", tableIdStr) + .add("timestamp", timestamp.toString()); + }); + return p; + }, partition, ignore -> false ); } - return dataSet; } } @@ -789,18 +770,16 @@ private SimpleDataSet data(Collection tableStates) } // TODO (desired): human readable packed key tracker (but requires loading Txn, so might be preferable to only do conditionally) - public static final class ProgressLogTable extends AbstractVirtualTable + public static final class ProgressLogTable extends AbstractLazyVirtualTable { private ProgressLogTable() { super(parse(VIRTUAL_ACCORD_DEBUG, PROGRESS_LOG, "Accord per-CommandStore ProgressLog State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + " command_store_id int,\n" + " txn_id 'TxnIdUtf8Type',\n" + + " table_id text,\n" + // Timer + BaseTxnState " contact_everyone boolean,\n" + // WaitingState @@ -816,43 +795,44 @@ private ProgressLogTable() " home_progress text,\n" + " home_retry_counter int,\n" + " home_scheduled_at timestamp,\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, txn_id)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, txn_id)" + + ')', Int32Type.instance), Sorted.ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { DefaultProgressLog.ImmutableView view = ((DefaultProgressLog) commandStore.unsafeProgressLog()).immutableView(); TableId tableId = ((AccordCommandStore)commandStore).tableId(); String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); + PartitionCollector partition = collector.partition(commandStore.id()); while (view.advance()) { - ds.row(keyspace(tableMetadata), table(tableId, tableMetadata), tableIdStr, view.commandStoreId(), view.txnId().toString()) - .column("contact_everyone", view.contactEveryone()) - .column("waiting_is_uninitialised", view.isWaitingUninitialised()) - .column("waiting_blocked_until", view.waitingIsBlockedUntil().name()) - .column("waiting_home_satisfies", view.waitingHomeSatisfies().name()) - .column("waiting_progress", view.waitingProgress().name()) - .column("waiting_retry_counter", view.waitingRetryCounter()) - .column("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) - .column("waiting_scheduled_at", toTimestamp(view.timerScheduledAt(TxnStateKind.Waiting))) - .column("home_phase", view.homePhase().name()) - .column("home_progress", view.homeProgress().name()) - .column("home_retry_counter", view.homeRetryCounter()) - .column("home_scheduled_at", toTimestamp(view.timerScheduledAt(TxnStateKind.Home))) - ; + // TODO (required): view should return an immutable per-row view so that we can call lazyAdd + partition.row(view.txnId().toString()) + .eagerAdd(columns -> { + columns.add("table_id", tableIdStr) + .add("contact_everyone", view.contactEveryone()) + .add("waiting_is_uninitialised", view.isWaitingUninitialised()) + .add("waiting_blocked_until", view.waitingIsBlockedUntil().name()) + .add("waiting_home_satisfies", view.waitingHomeSatisfies().name()) + .add("waiting_progress", view.waitingProgress().name()) + .add("waiting_retry_counter", view.waitingRetryCounter()) + .add("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) + .add("waiting_scheduled_at", view.timerScheduledAt(TxnStateKind.Waiting), ProgressLogTable::toTimestamp) + .add("home_phase", view.homePhase().name()) + .add("home_progress", view.homeProgress().name()) + .add("home_retry_counter", view.homeRetryCounter()) + .add("home_scheduled_at", view.timerScheduledAt(TxnStateKind.Home), ProgressLogTable::toTimestamp); + }); } } - return ds; } - private Date toTimestamp(Long deadline) + private static Date toTimestamp(Long deadline) { if (deadline == null) return null; @@ -862,19 +842,17 @@ private Date toTimestamp(Long deadline) } } - public static final class RedundantBeforeTable extends AbstractVirtualTable + public static final class RedundantBeforeTable extends AbstractLazyVirtualTable { private RedundantBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, REDUNDANT_BEFORE, "Accord per-CommandStore RedundantBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + + " command_store_id int,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + - " command_store_id int,\n" + " start_epoch bigint,\n" + " end_epoch bigint,\n" + " gc_before 'TxnIdUtf8Type',\n" + @@ -888,50 +866,46 @@ private RedundantBeforeTable() " locally_witnessed 'TxnIdUtf8Type',\n" + " log_unavailable 'TxnIdUtf8Type',\n" + " unready 'TxnIdUtf8Type',\n" + - " stale_until 'TxnIdUtf8Type',\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " stale_until_at_least 'TxnIdUtf8Type',\n" + + " PRIMARY KEY (command_store_id, token_start)" + + ')', Int32Type.instance), Sorted.ASC); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { int commandStoreId = commandStore.id(); + PartitionCollector partition = collector.partition(commandStoreId); TableId tableId = ((AccordCommandStore)commandStore).tableId(); String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); - String keyspace = keyspace(tableMetadata); - String table = table(tableId, tableMetadata); commandStore.unsafeGetRedundantBefore().foldl( - (entry, ds) -> { - ds.row(keyspace, table, tableIdStr, commandStoreId, printToken(entry.range.start())) - .column("token_end", printToken(entry.range.end())) - .column("start_epoch", entry.startEpoch) - .column("end_epoch", entry.endEpoch) - .column("gc_before", entry.maxBound(GC_BEFORE).toString()) - .column("shard_applied", entry.maxBound(SHARD_APPLIED).toString()) - .column("quorum_applied", entry.maxBound(QUORUM_APPLIED).toString()) - .column("locally_applied", entry.maxBound(LOCALLY_APPLIED).toString()) - .column("locally_durable_to_command_store", entry.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE).toString()) - .column("locally_durable_to_data_store", entry.maxBound(LOCALLY_DURABLE_TO_DATA_STORE).toString()) - .column("locally_redundant", entry.maxBound(LOCALLY_REDUNDANT).toString()) - .column("locally_synced", entry.maxBound(LOCALLY_SYNCED).toString()) - .column("locally_witnessed", entry.maxBound(LOCALLY_WITNESSED).toString()) - .column("log_unavailable", entry.maxBound(LOG_UNAVAILABLE).toString()) - .column("unready", entry.maxBound(UNREADY).toString()) - .column("stale_until", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); - return ds; - }, - dataSet, - ignore -> false + (entry, p) -> { + p.row(printToken(entry.range.start())).lazyAdd(columns -> { + columns.add("table_id", tableIdStr) + .add("token_end", printToken(entry.range.end())) + .add("start_epoch", entry.startEpoch) + .add("end_epoch", entry.endEpoch) + .add("gc_before", entry.maxBound(GC_BEFORE).toString()) + .add("shard_applied", entry.maxBound(SHARD_APPLIED).toString()) + .add("quorum_applied", entry.maxBound(QUORUM_APPLIED).toString()) + .add("locally_applied", entry.maxBound(LOCALLY_APPLIED).toString()) + .add("locally_durable_to_command_store", entry.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE).toString()) + .add("locally_durable_to_data_store", entry.maxBound(LOCALLY_DURABLE_TO_DATA_STORE).toString()) + .add("locally_redundant", entry.maxBound(LOCALLY_REDUNDANT).toString()) + .add("locally_synced", entry.maxBound(LOCALLY_SYNCED).toString()) + .add("locally_witnessed", entry.maxBound(LOCALLY_WITNESSED).toString()) + .add("unready", entry.maxBound(UNREADY).toString()) + .add("log_unavailable", entry.maxBound(LOG_UNAVAILABLE).toString()) + .add("stale_until_at_least", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); + }); + return p; + }, partition, ignore -> false ); } - return dataSet; } } @@ -1118,11 +1092,20 @@ public DataSet data() SimpleDataSet dataSet = new SimpleDataSet(metadata()); tracing().forEach(id -> true, (txnId, eventType, permits, events) -> { events.forEach(e -> { - e.messages().forEach(m -> { - dataSet.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) - .column("command_store_id", m.commandStoreId) - .column("message", m.message); - }); + if (e.messages().isEmpty()) + { + dataSet.row(txnId.toString(), eventType.name(), e.idMicros, 0L) + .column("message", ""); + + } + else + { + e.messages().forEach(m -> { + dataSet.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) + .column("command_store_id", m.commandStoreId) + .column("message", m.message); + }); + } }); }); return dataSet; @@ -1130,19 +1113,10 @@ public DataSet data() } // TODO (desired): don't report null as "null" - public static final class TxnTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + public static final class TxnTable extends AbstractLazyVirtualTable { - static class Entry - { - final int commandStoreId; - final Command command; + private static final CompositeType PK = CompositeType.getInstance(Int32Type.instance, UTF8Type.instance); - Entry(int commandStoreId, Command command) - { - this.commandStoreId = commandStoreId; - this.command = command; - } - } private TxnTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN, @@ -1165,67 +1139,116 @@ private TxnTable() " participants_has_touched text,\n" + " participants_executes text,\n" + " participants_waits_on text,\n" + - " PRIMARY KEY (txn_id, command_store_id)" + - ')', UTF8Type.instance)); + " PRIMARY KEY ((command_store_id, txn_id))" + + ')', PK), Sorted.ASC); } @Override - public DataSet data() + public boolean allowFilteringImplicitly() { - return this; + return false; } @Override - public boolean isEmpty() + public boolean allowFilteringPrimaryKeysImplicitly() { - return false; + return true; } @Override - public Partition getPartition(DecoratedKey partitionKey) + public void collect(PartitionsCollector collector) { - String txnIdStr = UTF8Type.instance.compose(partitionKey.getKey()); - TxnId txnId = TxnId.parse(txnIdStr); + AccordService accord; + { + IAccordService iaccord = AccordService.instance(); + if (!iaccord.isEnabled()) + return; - List commands = new CopyOnWriteArrayList<>(); - AccordService.instance().node().commandStores().forAllUnsafe(store -> { - Command command = ((AccordCommandStore)store).loadCommand(txnId); - if (command != null) - commands.add(new Entry(store.id(), command)); - }); + accord = (AccordService) iaccord; + } - if (commands.isEmpty()) - return null; + DataRange dataRange = collector.dataRange(); + JournalKey min = toJournalKey(dataRange.startKey()), + max = toJournalKey(dataRange.stopKey()); - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : commands) + if (min == null && max == null) { - Command command = e.command; - ds.row(txnIdStr, e.commandStoreId) - .column("save_status", toStringOrNull(command.saveStatus())) - .column("route", toStringOrNull(command.route())) - .column("participants_owns", toStr(command, StoreParticipants::owns, StoreParticipants::stillOwns)) - .column("participants_touches", toStr(command, StoreParticipants::touches, StoreParticipants::stillTouches)) - .column("participants_has_touched", toStringOrNull(command.participants().hasTouched())) - .column("participants_executes", toStr(command, StoreParticipants::executes, StoreParticipants::stillExecutes)) - .column("participants_waits_on", toStr(command, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) - .column("durability", toStringOrNull(command.durability())) - .column("execute_at", toStringOrNull(command.executeAt())) - .column("executes_at_least", toStringOrNull(command.executesAtLeast())) - .column("txn", toStringOrNull(command.partialTxn())) - .column("deps", toStringOrNull(command.partialDeps())) - .column("waiting_on", toStringOrNull(command.waitingOn())) - .column("writes", toStringOrNull(command.writes())) - .column("result", toStringOrNull(command.result())); + FilterRange filterTxnId = collector.filters("txn_id", TxnId::parse, UnaryOperator.identity(), UnaryOperator.identity()); + FilterRange filterCommandStoreId = collector.filters("command_store_id", UnaryOperator.identity(), i -> i + 1, i -> i - 1); + + int minCommandStoreId = filterCommandStoreId.min == null ? 0 : filterCommandStoreId.min; + int maxCommandStoreId = filterCommandStoreId.max == null ? Integer.MAX_VALUE : filterCommandStoreId.max; + + if (filterTxnId.min != null && filterTxnId.max != null && filterTxnId.min.equals(filterTxnId.max)) + { + TxnId txnId = filterTxnId.min; + accord.node().commandStores().forAllUnsafe(commandStore -> { + if (commandStore.id() < minCommandStoreId || commandStore.id() > maxCommandStoreId) + return; + + Command command = ((AccordCommandStore)commandStore).loadCommand(txnId); + if (command == null) + return; + + collector.row(commandStore.id(), txnId.toString()) + .lazyAdd(columns -> addColumns(command, columns)); + }); + + return; + } + + if (filterTxnId.min != null || minCommandStoreId > 0) + min = new JournalKey(filterTxnId.min == null ? TxnId.NONE : filterTxnId.min, JournalKey.Type.COMMAND_DIFF, minCommandStoreId); + if (filterTxnId.max != null || maxCommandStoreId < Integer.MAX_VALUE) + max = new JournalKey(filterTxnId.max == null ? TxnId.MAX.withoutNonIdentityFlags() : filterTxnId.max, JournalKey.Type.COMMAND_DIFF, maxCommandStoreId); } - return ds.getPartition(partitionKey); + accord.journal().forEach(key -> { + if (key.type != JournalKey.Type.COMMAND_DIFF) + return; + + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); + if (commandStore == null) + return; + + Command command = commandStore.loadCommand(key.id); + if (command == null) + return; + + collector.row(key.commandStoreId, key.id.toString()) + .lazyAdd(columns -> addColumns(command, columns)); + }, min, max, true); } - @Override - public Iterator getPartitions(DataRange range) + private static void addColumns(Command command, ColumnsCollector columns) { - throw new UnsupportedOperationException(); + columns.add("save_status", toStringOrNull(command.saveStatus())) + .add("route", toStringOrNull(command.route())) + .add("participants_owns", toStr(command, StoreParticipants::owns, StoreParticipants::stillOwns)) + .add("participants_touches", toStr(command, StoreParticipants::touches, StoreParticipants::stillTouches)) + .add("participants_has_touched", toStringOrNull(command.participants().hasTouched())) + .add("participants_executes", toStr(command, StoreParticipants::executes, StoreParticipants::stillExecutes)) + .add("participants_waits_on", toStr(command, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) + .add("durability", toStringOrNull(command.durability())) + .add("execute_at", toStringOrNull(command.executeAt())) + .add("executes_at_least", toStringOrNull(command.executesAtLeast())) + .add("txn", toStringOrNull(command.partialTxn())) + .add("deps", toStringOrNull(command.partialDeps())) + .add("waiting_on", toStringOrNull(command.waitingOn())) + .add("writes", toStringOrNull(command.writes())) + .add("result", toStringOrNull(command.result())); + } + + private static JournalKey toJournalKey(PartitionPosition position) + { + if (position.isMinimum()) + return null; + + if (!(position instanceof DecoratedKey)) + throw new InvalidRequestException("Cannot filter this table by partial partition key"); + + ByteBuffer[] keys = PK.split(((DecoratedKey) position).getKey()); + return new JournalKey(TxnId.parse(UTF8Type.instance.compose(keys[1])), JournalKey.Type.COMMAND_DIFF, Int32Type.instance.compose(keys[0])); } } @@ -1545,7 +1568,7 @@ protected TxnBlockedByTable() } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { Partition partition = data(partitionKey, rowFilter).getPartition(partitionKey); @@ -1710,6 +1733,14 @@ private static String toStringOrNull(Object o) { if (o == null) return null; - return Objects.toString(o); + + try + { + return Objects.toString(o); + } + catch (Throwable t) + { + return "'; + } } } diff --git a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java index cc311e1a2653..5bff0fe3be43 100644 --- a/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java +++ b/src/java/org/apache/cassandra/db/virtual/CollectionVirtualTableAdapter.java @@ -50,6 +50,7 @@ import org.apache.cassandra.db.EmptyIterators; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.BooleanType; @@ -308,7 +309,7 @@ else if (partitionKeyTypes.size() > 1) public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringFilter, ColumnFilter columnFilter, - RowFilter rowFilter) + RowFilter rowFilter, DataLimits limits) { if (!data.iterator().hasNext()) return EmptyIterators.unfilteredPartition(metadata); @@ -349,7 +350,7 @@ public UnfilteredPartitionIterator select(DecoratedKey partitionKey, } @Override - public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { return createPartitionIterator(metadata, new AbstractIterator<>() { diff --git a/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java index 550743c6a734..d63ae194b484 100644 --- a/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java +++ b/src/java/org/apache/cassandra/db/virtual/PartitionKeyStatsTable.java @@ -42,6 +42,7 @@ import org.apache.cassandra.db.context.CounterContext; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.marshal.CompositeType; import org.apache.cassandra.db.marshal.CounterColumnType; @@ -146,7 +147,7 @@ public PartitionKeyStatsTable(String keyspace) } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { if (clusteringIndexFilter.isReversed()) throw new InvalidRequestException(REVERSED_QUERY_ERROR); @@ -345,7 +346,7 @@ public TableMetadata metadata() } @Override - public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter) + public UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { throw new InvalidRequestException(UNSUPPORTED_RANGE_QUERY_ERROR); } diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java index 770cb139830f..51aa5140de03 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java @@ -21,6 +21,7 @@ import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.filter.ClusteringIndexFilter; import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.partitions.PartitionUpdate; import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; @@ -57,23 +58,25 @@ default String name() /** * Selects the rows from a single partition. * - * @param partitionKey the partition key + * @param partitionKey the partition key * @param clusteringIndexFilter the clustering columns to selected - * @param columnFilter the selected columns - * @param rowFilter filter on which rows a given query should include or exclude + * @param columnFilter the selected columns + * @param rowFilter filter on which rows a given query should include or exclude + * @param limits * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter); + UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits); /** * Selects the rows from a range of partitions. * - * @param dataRange the range of data to retrieve + * @param dataRange the range of data to retrieve * @param columnFilter the selected columns - * @param rowFilter filter on which rows a given query should include or exclude + * @param rowFilter filter on which rows a given query should include or exclude + * @param limits * @return the rows corresponding to the requested data. */ - UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter); + UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits); /** * Truncates data from the underlying source, if supported. @@ -90,4 +93,9 @@ default boolean allowFilteringImplicitly() { return true; } + + default boolean allowFilteringPrimaryKeysImplicitly() + { + return allowFilteringImplicitly(); + } } diff --git a/src/java/org/apache/cassandra/journal/InMemoryIndex.java b/src/java/org/apache/cassandra/journal/InMemoryIndex.java index 49fe4d136714..974767590a37 100644 --- a/src/java/org/apache/cassandra/journal/InMemoryIndex.java +++ b/src/java/org/apache/cassandra/journal/InMemoryIndex.java @@ -18,6 +18,7 @@ package org.apache.cassandra.journal; import java.io.IOException; +import java.util.Iterator; import java.util.NavigableMap; import java.util.TreeMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -126,6 +127,16 @@ long[] lookUpAll(K id) return lookUp(id); } + public Iterator keyIterator(@Nullable K min, @Nullable K max) + { + NavigableMap m; + if (min == null && max == null) m = index; + else if (min == null) m = index.headMap(max, true); + else if (max == null) m = index.tailMap(min, true); + else m = index.subMap(min, true, max, true); + return m.keySet().iterator(); + } + public void persist(Descriptor descriptor) { File tmpFile = descriptor.tmpFileFor(Component.INDEX); diff --git a/src/java/org/apache/cassandra/journal/Journal.java b/src/java/org/apache/cassandra/journal/Journal.java index d0793d39a7b8..2a63ab7cc4b0 100644 --- a/src/java/org/apache/cassandra/journal/Journal.java +++ b/src/java/org/apache/cassandra/journal/Journal.java @@ -934,11 +934,11 @@ public interface Writer } /** - * Static segment iterator iterates all keys in _static_ segments in order. + * segment iterator iterates all keys in order. */ - public StaticSegmentKeyIterator staticSegmentKeyIterator(K min, K max) + public SegmentKeyIterator segmentKeyIterator(K min, K max, Predicate> include) { - return new StaticSegmentKeyIterator(min, max); + return new SegmentKeyIterator(min, max, include); } /** @@ -1000,53 +1000,36 @@ public String toString() } } - public class StaticSegmentKeyIterator implements CloseableIterator> + public class SegmentKeyIterator implements CloseableIterator> { private final ReferencedSegments segments; private final MergeIterator> iterator; - public StaticSegmentKeyIterator(K min, K max) + public SegmentKeyIterator(K min, K max, Predicate> include) { - this.segments = selectAndReference(s -> s.isStatic() - && s.asStatic().index().entryCount() > 0 + this.segments = selectAndReference(s -> include.test(s) && !s.isEmpty() && (min == null || keySupport.compare(s.index().lastId(), min) >= 0) && (max == null || keySupport.compare(s.index().firstId(), max) <= 0)); List> iterators = new ArrayList<>(segments.count()); for (Segment segment : segments.allSorted(true)) { - final StaticSegment staticSegment = (StaticSegment) segment; - final OnDiskIndex.IndexReader iter = staticSegment.index().reader(); - if (min != null) iter.seek(min); - if (max != null) iter.seekEnd(max); - if (!iter.hasNext()) - continue; - - iterators.add(new AbstractIterator<>() + if (segment.isStatic()) { - final Head head = new Head(staticSegment.descriptor.timestamp); - - @Override - protected Head computeNext() - { - if (!iter.hasNext()) - return endOfData(); - - K next = iter.next(); - while (next.equals(head.key)) - { - if (!iter.hasNext()) - return endOfData(); - - next = iter.next(); - } - - Invariants.require(!next.equals(head.key), - "%s == %s", next, head.key); - head.key = next; - return head; - } - }); + final StaticSegment staticSegment = (StaticSegment) segment; + final OnDiskIndex.IndexReader iter = staticSegment.index().reader(); + if (min != null) iter.seek(min); + if (max != null) iter.seekEnd(max); + if (iter.hasNext()) + iterators.add(keyIterator(segment.descriptor.timestamp, iter)); + } + else + { + final ActiveSegment activeSegment = (ActiveSegment) segment; + final Iterator iter = activeSegment.index().keyIterator(min, max); + if (iter.hasNext()) + iterators.add(keyIterator(segment.descriptor.timestamp, iter)); + } } this.iterator = MergeIterator.get(iterators, @@ -1077,6 +1060,34 @@ protected void onKeyChange() }); } + private Iterator keyIterator(long segment, Iterator iter) + { + final Head head = new Head(segment); + return new AbstractIterator<>() + { + @Override + protected Head computeNext() + { + if (!iter.hasNext()) + return endOfData(); + + K next = iter.next(); + while (next.equals(head.key)) + { + if (!iter.hasNext()) + return endOfData(); + + next = iter.next(); + } + + Invariants.require(!next.equals(head.key), + "%s == %s", next, head.key); + head.key = next; + return head; + } + }; + } + @Override public void close() { diff --git a/src/java/org/apache/cassandra/journal/Segment.java b/src/java/org/apache/cassandra/journal/Segment.java index 3854f0ee27b5..1fda2f57c982 100644 --- a/src/java/org/apache/cassandra/journal/Segment.java +++ b/src/java/org/apache/cassandra/journal/Segment.java @@ -66,7 +66,8 @@ public final void tidy() abstract boolean isActive(); abstract boolean isFlushed(long position); - boolean isStatic() { return !isActive(); } + public boolean isStatic() { return !isActive(); } + abstract boolean isEmpty(); abstract ActiveSegment asActive(); abstract StaticSegment asStatic(); diff --git a/src/java/org/apache/cassandra/journal/StaticSegment.java b/src/java/org/apache/cassandra/journal/StaticSegment.java index 35c987c8a480..bc425dda7e2f 100644 --- a/src/java/org/apache/cassandra/journal/StaticSegment.java +++ b/src/java/org/apache/cassandra/journal/StaticSegment.java @@ -246,6 +246,12 @@ public int entryCount() return index.entryCount(); } + @Override + boolean isEmpty() + { + return entryCount() == 0; + } + @Override boolean isActive() { diff --git a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java index 637b158177fd..2fa8675563c4 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordExecutor.java +++ b/src/java/org/apache/cassandra/service/accord/AccordExecutor.java @@ -1655,7 +1655,8 @@ public DebuggableTask debuggable() public static class TaskInfo implements Comparable { - public enum Status { WAITING_TO_LOAD, SCANNING_RANGES, LOADING, WAITING_TO_RUN, RUNNING } + // sorted in name order for reporting to virtual tables + public enum Status { LOADING, RUNNING, SCANNING_RANGES, WAITING_TO_LOAD, WAITING_TO_RUN } final Status status; final int commandStoreId; @@ -1706,7 +1707,7 @@ public int position() public int compareTo(TaskInfo that) { int c = this.status.compareTo(that.status); - if (c == 0) c = this.position() - that.position(); + if (c == 0) c = Integer.compare(this.position(), that.position()); return c; } } diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournal.java b/src/java/org/apache/cassandra/service/accord/AccordJournal.java index cfa224ab4945..54f952212472 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournal.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournal.java @@ -376,7 +376,8 @@ public List replayTopologies() try (CloseableIterator iter = new CloseableIterator<>() { final CloseableIterator> iter = journalTable.keyIterator(topologyUpdateKey(0L), - topologyUpdateKey(Timestamp.MAX_EPOCH)); + topologyUpdateKey(Timestamp.MAX_EPOCH), + true); TopologyImage prev = null; @Override @@ -571,9 +572,14 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) journalTable.forceCompaction(); } - public void forEach(Consumer consumer) + public void forEach(Consumer consumer, boolean includeActive) { - try (CloseableIterator> iter = journalTable.keyIterator(null, null)) + forEach(consumer, null, null, includeActive); + } + + public void forEach(Consumer consumer, @Nullable JournalKey min, @Nullable JournalKey max, boolean includeActive) + { + try (CloseableIterator> iter = journalTable.keyIterator(min, max, includeActive)) { while (iter.hasNext()) { @@ -610,7 +616,7 @@ public ReplayStream(CommandStore commandStore) this.commandStore = commandStore; this.replayer = commandStore.replayer(); // Keys in the index are sorted by command store id, so index iteration will be sequential - this.iter = journalTable.keyIterator(new JournalKey(TxnId.NONE, COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id())); + this.iter = journalTable.keyIterator(new JournalKey(TxnId.NONE, COMMAND_DIFF, commandStore.id()), new JournalKey(TxnId.MAX.withoutNonIdentityFlags(), COMMAND_DIFF, commandStore.id()), false); } boolean replay() diff --git a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java b/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java index 984ab7ed6210..17aff49f86a0 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java +++ b/src/java/org/apache/cassandra/service/accord/AccordJournalTable.java @@ -75,6 +75,7 @@ import org.apache.cassandra.journal.Journal; import org.apache.cassandra.journal.KeySupport; import org.apache.cassandra.journal.RecordConsumer; +import org.apache.cassandra.journal.Segment; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.service.RetryStrategy; import org.apache.cassandra.service.accord.AccordKeyspace.JournalColumns; @@ -457,9 +458,9 @@ private TableKeyIterator readAllFromTable(K key) } @SuppressWarnings("resource") // Auto-closeable iterator will release related resources - public CloseableIterator> keyIterator(@Nullable K min, @Nullable K max) + public CloseableIterator> keyIterator(@Nullable K min, @Nullable K max, boolean includeActive) { - return new JournalAndTableKeyIterator(min, max); + return new JournalAndTableKeyIterator(min, max, includeActive); } private class TableIterator extends AbstractIterator implements CloseableIterator @@ -515,12 +516,12 @@ public void close() private class JournalAndTableKeyIterator extends AbstractIterator> implements CloseableIterator> { final TableIterator tableIterator; - final Journal.StaticSegmentKeyIterator journalIterator; + final Journal.SegmentKeyIterator journalIterator; - private JournalAndTableKeyIterator(K min, K max) + private JournalAndTableKeyIterator(K min, K max, boolean includeActive) { this.tableIterator = new TableIterator(min, max); - this.journalIterator = journal.staticSegmentKeyIterator(min, max); + this.journalIterator = journal.segmentKeyIterator(min, max, includeActive ? ignore -> true : Segment::isStatic); } K prevFromTable = null; diff --git a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java index 93132c5467de..a14770472c51 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java +++ b/src/java/org/apache/cassandra/tools/StandaloneJournalUtil.java @@ -274,7 +274,7 @@ public void run() Map cache = new HashMap<>(); journal.start(null); - journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes)); + journal.forEach(key -> processKey(cache, journal, key, txnId, sinceTimestamp, untilTimestamp, skipAllErrors, skipExceptionTypes), false); } private void processKey(Map redundantBeforeCache, AccordJournal journal, JournalKey key, Timestamp txnId, Timestamp minTimestamp, Timestamp maxTimestamp, boolean skipAllErrors, Set skipExceptionTypes) diff --git a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java index 5ba2a7c5d308..e4c497657d2b 100644 --- a/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java +++ b/test/distributed/org/apache/cassandra/fuzz/topology/JournalGCTest.java @@ -113,7 +113,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && (a.get() == null || v.id.compareTo(a.get()) > 0)) a.set(v.id); - }); + }, false); return a.get() == null ? "" : a.get().toString(); }); @@ -123,7 +123,7 @@ public void journalGCTest() throws Throwable ((AccordService) AccordService.instance()).journal().forEach((v) -> { if (v.type == JournalKey.Type.COMMAND_DIFF && v.id.compareTo(maxId) <= 0) a.incrementAndGet(); - }); + }, false); return a.get(); }, maximumId); diff --git a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java index 42db56fb9494..e074ed950e3e 100644 --- a/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java +++ b/test/distributed/org/apache/cassandra/service/accord/AccordJournalBurnTest.java @@ -332,7 +332,7 @@ public void purge(CommandStores commandStores, EpochSupplier minEpoch) private TreeMap read(CommandStores commandStores) { TreeMap result = new TreeMap<>(JournalKey.SUPPORT::compare); - try (CloseableIterator> iter = journalTable.keyIterator(null, null)) + try (CloseableIterator> iter = journalTable.keyIterator(null, null, false)) { JournalKey prev = null; while (iter.hasNext()) diff --git a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java index 024e343ed7f7..c072e6403c06 100644 --- a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java @@ -19,6 +19,8 @@ package org.apache.cassandra.db.virtual; import java.util.Collections; +import java.util.ArrayList; +import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -35,6 +37,7 @@ import accord.api.ProtocolModifiers; import accord.messages.NoWaitRequest; +import accord.api.RoutingKey; import accord.primitives.Ranges; import accord.primitives.Routable; import accord.primitives.SaveStatus; @@ -47,6 +50,7 @@ import org.apache.cassandra.config.YamlConfigurationLoader; import org.apache.cassandra.cql3.CQLTester; import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; @@ -56,11 +60,15 @@ import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.service.CassandraDaemon; +import org.apache.cassandra.service.accord.AccordCommandStore; import org.apache.cassandra.service.accord.AccordService; +import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.TokenRange; import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.transport.Dispatcher; import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.concurrent.Condition; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; @@ -87,6 +95,12 @@ public class AccordDebugKeyspaceTest extends CQLTester private static final String QUERY_TXN = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_TXNS = + String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + + private static final String QUERY_TXNS_SEARCH = + String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? AND txn_id > ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_JOURNAL = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.JOURNAL); @@ -240,6 +254,36 @@ public void completedTxn() throws ExecutionException, InterruptedException assertRows(execute(QUERY_COMMANDS_FOR_KEY, keyStr), row(id.toString(), "APPLIED_DURABLE")); } + @Test + public void manyTxns() throws ExecutionException, InterruptedException + { + String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); + AccordService accord = accord(); + List await = new ArrayList<>(); + Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); + for (int i = 0 ; i < 100; ++i) + await.add(accord.coordinateAsync(0, 0, txn, ConsistencyLevel.QUORUM, new Dispatcher.RequestTime(Clock.Global.nanoTime()))); + + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().unsafeForKey((RoutingKey) txn.keys().get(0).toUnseekable()); + await.forEach(IAccordService.IAccordResult::awaitAndGet); + + assertRows(execute(QUERY_TXNS, commandStore.id()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + + assertRows(execute(QUERY_TXNS_SEARCH, commandStore.id(), TxnId.NONE.toString()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + } + @Test public void inflight() throws ExecutionException, InterruptedException { @@ -463,4 +507,6 @@ public boolean test(Message msg, InetAddressAndPort to) return !dropVerbs.contains(msg.verb()); } } + + } \ No newline at end of file From 1ccec4887b16055bc88f4e0b5fb9e7c86b7bae16 Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Sat, 13 Sep 2025 10:45:08 +0100 Subject: [PATCH 3/6] - Support mutable lazy vtables and port remaining accord tables - Integrate txn_blocked_by deadline and depth filter with execution logic, to ensure we terminate promptly and get a best effort reply --- .../cassandra/db/marshal/TxnIdUtf8Type.java | 6 + .../db/virtual/AbstractLazyVirtualTable.java | 268 +++-- .../AbstractMutableLazyVirtualTable.java | 137 +++ .../db/virtual/AccordDebugKeyspace.java | 960 +++++++----------- .../cassandra/db/virtual/VirtualTable.java | 4 +- .../service/accord/AccordService.java | 149 +-- .../accord/CommandStoreTxnBlockedGraph.java | 135 --- .../service/accord/DebugBlockedTxns.java | 249 +++++ .../service/accord/IAccordService.java | 15 - .../org/apache/cassandra/cql3/CQLTester.java | 2 +- .../db/virtual/AccordDebugKeyspaceTest.java | 29 +- 11 files changed, 982 insertions(+), 972 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java delete mode 100644 src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java create mode 100644 src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java diff --git a/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java b/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java index 784969ead703..17c62be40ad8 100644 --- a/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java +++ b/src/java/org/apache/cassandra/db/marshal/TxnIdUtf8Type.java @@ -48,6 +48,12 @@ public void validate(V value, ValueAccessor accessor) throws MarshalExcep String describe() { return "TxnId"; } + @Override + public boolean isEmptyValueMeaningless() + { + return true; + } + @Override public TypeSerializer getSerializer() { diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java index 9f98cd52d73b..da7f37e1ed29 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractLazyVirtualTable.java @@ -36,7 +36,7 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Clusterable; import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ConsistencyLevel; +import org.apache.cassandra.db.ClusteringPrefix; import org.apache.cassandra.db.DataRange; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.DeletionTime; @@ -63,6 +63,7 @@ import org.apache.cassandra.exceptions.ReadTimeoutException; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ClientWarn; import org.apache.cassandra.utils.BulkIterator; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.FBUtilities; @@ -70,6 +71,7 @@ import org.apache.cassandra.utils.btree.UpdateFunction; import static org.apache.cassandra.db.ClusteringPrefix.Kind.STATIC_CLUSTERING; +import static org.apache.cassandra.db.ConsistencyLevel.ONE; import static org.apache.cassandra.utils.Clock.Global.nanoTime; /** @@ -77,8 +79,12 @@ */ public abstract class AbstractLazyVirtualTable implements VirtualTable { + public enum OnTimeout { BEST_EFFORT, FAIL } + // in the special case where we know we have enough rows in the collector, throw this exception to terminate early - static class InternalDoneException extends RuntimeException {} + public static class InternalDoneException extends RuntimeException {} + // in the special case where we have timed out, throw this exception to terminate early + public static class InternalTimeoutException extends RuntimeException {} public static class FilterRange { @@ -94,43 +100,64 @@ public interface PartitionsCollector { DataRange dataRange(); RowFilter rowFilter(); + ColumnFilter columnFilter(); + DataLimits limits(); + long nowInSeconds(); + long timestampMicros(); + long deadlineNanos(); + boolean isEmpty(); + RowCollector row(Object... primaryKeys); PartitionCollector partition(Object... partitionKeys); UnfilteredPartitionIterator finish(); - default @Nullable DecoratedKey singleKey() - { - AbstractBounds bounds = dataRange().keyRange(); - if (!bounds.isStartInclusive() || !bounds.isEndInclusive() || !bounds.left.equals(bounds.right) || !(bounds.left instanceof DecoratedKey)) - return null; - - return (DecoratedKey) bounds.left; - } - - FilterRange filters(String column, Function translate, UnaryOperator increment, UnaryOperator decrement); + @Nullable Object[] singlePartitionKey(); + FilterRange filters(String column, Function translate, UnaryOperator exclusiveStart, UnaryOperator exclusiveEnd); } public interface PartitionCollector { - RowCollector row(Object... clusteringKeys); + void collect(Consumer addTo); + } + + public interface RowsCollector + { + RowCollector add(Object... clusteringKeys); } public interface RowCollector { - default PartitionsCollector lazyAdd(Consumer addToIfNeeded) { return eagerAdd(addToIfNeeded); } - PartitionsCollector eagerAdd(Consumer addToNow); + default void lazyCollect(Consumer addToIfNeeded) { eagerCollect(addToIfNeeded); } + void eagerCollect(Consumer addToNow); } public interface ColumnsCollector { - ColumnsCollector add(String columnName, V value, Function transform); - default ColumnsCollector add(String columnName, Object value) { return add(columnName, value, Function.identity()); } + /** + * equivalent to + * {@code + * if (value == null) add(columnName, null); + * else if (f1.apply(value) == null) add(columnName, f1.apply(value)); + * else add(columnName, f2.apply(f1.apply(value))); + * } + */ + ColumnsCollector add(String columnName, V1 value, Function f1, Function f2); + + default ColumnsCollector add(String columnName, V value, Function transform) + { + return add(columnName, value, Function.identity(), transform); + } + default ColumnsCollector add(String columnName, Object value) + { + return add(columnName, value, Function.identity()); + } } public static class SimplePartitionsCollector implements PartitionsCollector { final TableMetadata metadata; final boolean isSorted; + final boolean isSortedByPartitionKey; final Map columnLookup = new HashMap<>(); final NavigableMap partitions; @@ -140,34 +167,50 @@ public static class SimplePartitionsCollector implements PartitionsCollector final RowFilter rowFilter; final DataLimits limits; - final long startedAt = Clock.Global.nanoTime(); - final long timeoutAt; + final long startedAtNanos = Clock.Global.nanoTime(); + final long deadlineNanos; final long nowInSeconds = Clock.Global.nowInSeconds(); - final long timestamp; + final long timestampMicros; int totalRowCount; - int lastFilteredTotalRowCount; + int lastFilteredTotalRowCount, lastFilteredPartitionCount; @Override public DataRange dataRange() { return dataRange; } @Override public RowFilter rowFilter() { return rowFilter; } - public ColumnFilter columnFilter() { return columnFilter; } - - public SimplePartitionsCollector(TableMetadata metadata, boolean isSorted, DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + @Override public ColumnFilter columnFilter() { return columnFilter; } + @Override public DataLimits limits() { return limits; } + @Override public long nowInSeconds() { return nowInSeconds; } + @Override public long timestampMicros() { return timestampMicros; } + @Override public long deadlineNanos() { return deadlineNanos; } + @Override public boolean isEmpty() { return totalRowCount == 0; } + + public SimplePartitionsCollector(TableMetadata metadata, boolean isSorted, boolean isSortedByPartitionKey, + DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { this.metadata = metadata; this.isSorted = isSorted; + this.isSortedByPartitionKey = isSortedByPartitionKey; this.dataRange = dataRange; this.columnFilter = columnFilter; this.rowFilter = rowFilter; this.limits = limits; - this.timestamp = FBUtilities.timestampMicros(); - this.timeoutAt = startedAt + DatabaseDescriptor.getReadRpcTimeout(TimeUnit.NANOSECONDS); + this.timestampMicros = FBUtilities.timestampMicros(); + this.deadlineNanos = startedAtNanos + DatabaseDescriptor.getReadRpcTimeout(TimeUnit.NANOSECONDS); this.partitions = new TreeMap<>(dataRange.isReversed() ? DecoratedKey.comparator.reversed() : DecoratedKey.comparator); for (ColumnMetadata cm : metadata.columns()) columnLookup.put(cm.name.toString(), cm); } + public Object[] singlePartitionKey() + { + AbstractBounds bounds = dataRange().keyRange(); + if (!bounds.isStartInclusive() || !bounds.isEndInclusive() || !bounds.left.equals(bounds.right) || !(bounds.left instanceof DecoratedKey)) + return null; + + return composePartitionKeys((DecoratedKey) bounds.left, metadata); + } + @Override public PartitionCollector partition(Object ... partitionKeys) { @@ -175,9 +218,9 @@ public PartitionCollector partition(Object ... partitionKeys) if (pkSize != partitionKeys.length) throw new IllegalArgumentException(); - DecoratedKey partitionKey = makeDecoratedKey(partitionKeys); + DecoratedKey partitionKey = decomposePartitionKeys(metadata, partitionKeys); if (!dataRange.contains(partitionKey)) - return dropCks -> dropRow -> this; + return dropCks -> {}; return partitions.computeIfAbsent(partitionKey, SimplePartition::new); } @@ -224,7 +267,7 @@ public UnfilteredRowIterator next() @Override @Nullable - public FilterRange filters(String columnName, Function translate, UnaryOperator increment, UnaryOperator decrement) + public FilterRange filters(String columnName, Function translate, UnaryOperator exclusiveStart, UnaryOperator exclusiveEnd) { ColumnMetadata column = columnLookup.get(columnName); O min = null, max = null; @@ -233,18 +276,15 @@ public FilterRange filters(String columnName, Function translate if (!expression.column().equals(column)) continue; - if (expression.isCustom()) - continue; - O bound = translate.apply((I)column.type.compose(expression.getIndexValue())); switch (expression.operator()) { - default: continue; - case EQ: min = max = bound; break; + default: throw new InvalidRequestException("Operator " + expression.operator() + " not supported for txn_id"); + case EQ: min = max = bound; break; case LTE: max = bound; break; - case LT: max = decrement.apply(bound); break; + case LT: max = exclusiveEnd.apply(bound); break; case GTE: min = bound; break; - case GT: min = increment.apply(bound); break; + case GT: min = exclusiveStart.apply(bound); break; } } @@ -265,35 +305,16 @@ public RowCollector row(Object... primaryKeys) System.arraycopy(primaryKeys, 0, partitionKeyValues, 0, pkSize); System.arraycopy(primaryKeys, pkSize, clusteringValues, 0, ckSize); - DecoratedKey partitionKey = makeDecoratedKey(partitionKeyValues); - Clustering clustering = makeClustering(clusteringValues); + DecoratedKey partitionKey = decomposePartitionKeys(metadata, partitionKeyValues); + Clustering clustering = decomposeClusterings(metadata, clusteringValues); if (!dataRange.contains(partitionKey) || !dataRange.clusteringIndexFilter(partitionKey).selects(clustering)) - return drop -> this; - - return partitions.computeIfAbsent(partitionKey, SimplePartition::new).row(); - } + return drop -> {}; - private DecoratedKey makeDecoratedKey(Object... partitionKeyValues) - { - ByteBuffer partitionKey = partitionKeyValues.length == 1 - ? decompose(metadata.partitionKeyType, partitionKeyValues[0]) - : ((CompositeType) metadata.partitionKeyType).decompose(partitionKeyValues); - return metadata.partitioner.decorateKey(partitionKey); + return partitions.computeIfAbsent(partitionKey, SimplePartition::new).row(clustering); } - private Clustering makeClustering(Object... clusteringValues) - { - if (clusteringValues.length == 0) - return Clustering.EMPTY; - - ByteBuffer[] clusteringByteBuffers = new ByteBuffer[clusteringValues.length]; - for (int i = 0; i < clusteringValues.length; i++) - clusteringByteBuffers[i] = decompose(metadata.clusteringColumns().get(i).type, clusteringValues[i]); - return Clustering.make(clusteringByteBuffers); - } - - private final class SimplePartition implements PartitionCollector + private final class SimplePartition implements PartitionCollector, RowsCollector { private final DecoratedKey key; // we assume no duplicate rows, and impose the condition lazily @@ -309,27 +330,36 @@ private SimplePartition(DecoratedKey key) } @Override - public RowCollector row(Object... clusteringKeys) + public void collect(Consumer addTo) + { + addTo.accept(this); + } + + @Override + public RowCollector add(Object... clusteringKeys) { int ckSize = metadata.clusteringColumns().size(); if (ckSize != clusteringKeys.length) throw new IllegalArgumentException(); - return row(makeClustering(clusteringKeys)); + return row(decomposeClusterings(metadata, clusteringKeys)); } RowCollector row(Clustering clustering) { - if (nanoTime() > timeoutAt) - throw new ReadTimeoutException(ConsistencyLevel.ONE, 0, 1, false); + if (nanoTime() > deadlineNanos) + throw new InternalTimeoutException(); if (dropRows || !dataRange.clusteringIndexFilter(key).selects(clustering)) - return drop -> SimplePartitionsCollector.this; + return drop -> {}; if (totalRowCount >= limits.count()) { boolean filter; - if (!isSorted) filter = totalRowCount / 2 >= Math.max(1024, limits.count()); + if (!isSorted || !isSortedByPartitionKey || lastFilteredPartitionCount == partitions.size()) + { + filter = totalRowCount / 2 >= Math.max(1024, limits.count()); + } else { int rowsAddedSinceLastFiltered = totalRowCount - lastFilteredTotalRowCount; @@ -370,12 +400,16 @@ RowCollector row(Clustering clustering) newCount = last.filterAndSortAndTruncate(newCount); totalRowCount -= curCount - newCount; lastFilteredTotalRowCount = totalRowCount; + lastFilteredPartitionCount = partitions.size(); + + if (isSortedByPartitionKey && totalRowCount - newCount >= limits.count()) + throw new InternalDoneException(); if (isSorted && totalRowCount >= limits.count()) throw new InternalDoneException(); if (dropRows) - return drop -> SimplePartitionsCollector.this; + return drop -> {}; } } @@ -456,19 +490,17 @@ private SimpleRow(Clustering clustering) } @Override - public PartitionsCollector lazyAdd(Consumer addToIfNeeded) + public void lazyCollect(Consumer addToIfNeeded) { Invariants.require(state == null); state = new LazyColumnsCollector(addToIfNeeded); - return SimplePartitionsCollector.this; } @Override - public PartitionsCollector eagerAdd(Consumer addToNow) + public void eagerCollect(Consumer addToNow) { Invariants.require(state == null); state = new EagerColumnsCollector(addToNow); - return SimplePartitionsCollector.this; } boolean rowFilterIncludes() @@ -529,21 +561,25 @@ public EagerColumnsCollector(Consumer add) } @Override - public ColumnsCollector add(String name, V input, Function f) + public ColumnsCollector add(String name, V1 v1, Function f1, Function f2) { ColumnMetadata cm = columnLookup.get(name); if (!columnFilter.fetches(cm)) return this; - Object value = f.apply(input); - if (value == null) + V2 v2 = f1.apply(v1); + if (v2 == null) + return this; + + Object result = f2.apply(v2); + if (result == null) return this; if (columnCount * 2 == columns.length) columns = Arrays.copyOf(columns, columnCount * 4); columns[columnCount * 2] = cm; - columns[columnCount * 2 + 1] = value; + columns[columnCount * 2 + 1] = result; ++columnCount; return this; } @@ -554,11 +590,13 @@ FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) for (int i = 0 ; i < columnCount ; i++) { ColumnMetadata cm = (ColumnMetadata) columns[i * 2]; - columns[i] = BufferCell.live(cm, timestamp, decompose(cm.type, columns[i * 2 + 1])); + Object value = columns[i * 2 + 1]; + ByteBuffer bb = value instanceof ByteBuffer ? (ByteBuffer)value : decompose(cm.type, value); + columns[i] = BufferCell.live(cm, timestampMicros, bb); } Arrays.sort(columns, 0, columnCount, (a, b) -> ColumnData.comparator.compare((BufferCell)a, (BufferCell)b)); Object[] btree = BTree.build(BulkIterator.of(columns), columnCount, UpdateFunction.noOp); - BTreeRow row = BTreeRow.create(parent.clustering, LivenessInfo.EMPTY, Row.Deletion.LIVE, btree); + BTreeRow row = BTreeRow.create(parent.clustering, LivenessInfo.create(timestampMicros, nowInSeconds), Row.Deletion.LIVE, btree); if (!rowFilter.isSatisfiedBy(metadata, parent.partitionKey(), row, nowInSeconds)) return null; return new FilteredRow(row); @@ -581,19 +619,24 @@ FilteredRow materialiseAndFilter(SimplePartition.SimpleRow parent) } } - // TODO (expected): add e.g. BOTH_ASC_DESC when some vtable supports it - public enum Sorted { UNSORTED, ASC, DESC } - protected final TableMetadata metadata; - private final Sorted sorted; + private final OnTimeout onTimeout; + private final Sorted sorted, sortedByPartitionKey; - protected AbstractLazyVirtualTable(TableMetadata metadata, Sorted sorted) + protected AbstractLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted) + { + this(metadata, onTimeout, sorted, sorted); + } + + protected AbstractLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted, Sorted sortedByPartitionKey) { if (!metadata.isVirtual()) throw new IllegalArgumentException("Cannot instantiate a non-virtual table"); this.metadata = metadata; + this.onTimeout = onTimeout; this.sorted = sorted; + this.sortedByPartitionKey = sortedByPartitionKey; } @Override @@ -602,10 +645,19 @@ public TableMetadata metadata() return metadata; } + public OnTimeout onTimeout() { return onTimeout; } + protected PartitionsCollector collector(DataRange dataRange, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) { - boolean isSorted = sorted == (dataRange.isReversed() ? Sorted.DESC : Sorted.ASC); - return new SimplePartitionsCollector(metadata, isSorted, dataRange, columnFilter, rowFilter, limits); + boolean isSorted = isSorted(sorted, !dataRange.isReversed()); + boolean isSortedByPartitionKey = isSorted || isSorted(sortedByPartitionKey, !dataRange.isReversed()); + return new SimplePartitionsCollector(metadata, isSorted, isSortedByPartitionKey, dataRange, columnFilter, rowFilter, limits); + } + + + private static boolean isSorted(Sorted sorted, boolean asc) + { + return sorted == Sorted.SORTED || sorted == (asc ? Sorted.ASC : Sorted.DESC); } protected abstract void collect(PartitionsCollector collector); @@ -625,6 +677,12 @@ public final UnfilteredPartitionIterator select(DataRange dataRange, ColumnFilte collect(collector); } catch (InternalDoneException ignore) {} + catch (InternalTimeoutException ignore) + { + if (onTimeout != OnTimeout.BEST_EFFORT || collector.isEmpty()) + throw new ReadTimeoutException(ONE, 0, 1, false); + ClientWarn.instance.warn("Ran out of time. Returning best effort."); + } return collector.finish(); } @@ -646,8 +704,50 @@ public String toString() return metadata().toString(); } + static Object[] composePartitionKeys(DecoratedKey decoratedKey, TableMetadata metadata) + { + if (metadata.partitionKeyColumns().size() == 1) + return new Object[] { metadata.partitionKeyType.compose(decoratedKey.getKey()) }; + + ByteBuffer[] split = ((CompositeType)metadata.partitionKeyType).split(decoratedKey.getKey()); + Object[] result = new Object[split.length]; + for (int i = 0 ; i < split.length ; ++i) + result[i] = metadata.partitionKeyColumns().get(i).type.compose(split[i]); + return result; + } + + static Object[] composeClusterings(ClusteringPrefix clustering, TableMetadata metadata) + { + Object[] result = new Object[clustering.size()]; + for (int i = 0 ; i < result.length ; ++i) + result[i] = metadata.clusteringColumns().get(i).type.compose(clustering.get(i), clustering.accessor()); + return result; + } + private static ByteBuffer decompose(AbstractType type, Object value) { return type.decomposeUntyped(value); } + + static DecoratedKey decomposePartitionKeys(TableMetadata metadata, Object... partitionKeys) + { + ByteBuffer partitionKey = partitionKeys.length == 1 + ? decompose(metadata.partitionKeyType, partitionKeys[0]) + : ((CompositeType) metadata.partitionKeyType).decompose(partitionKeys); + return metadata.partitioner.decorateKey(partitionKey); + } + + static Clustering decomposeClusterings(TableMetadata metadata, Object... clusteringKeys) + { + if (clusteringKeys.length == 0) + return Clustering.EMPTY; + + ByteBuffer[] clusteringByteBuffers = new ByteBuffer[clusteringKeys.length]; + for (int i = 0; i < clusteringKeys.length; i++) + { + if (clusteringKeys[i] instanceof ByteBuffer) clusteringByteBuffers[i] = (ByteBuffer) clusteringKeys[i]; + else clusteringByteBuffers[i] = decompose(metadata.clusteringColumns().get(i).type, clusteringKeys[i]); + } + return Clustering.make(clusteringByteBuffers); + } } diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java new file mode 100644 index 000000000000..cc90fa4a4692 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java @@ -0,0 +1,137 @@ +/* + * 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.db.virtual; + +import java.util.Iterator; + +import javax.annotation.Nullable; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; + +import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.db.ClusteringPrefix.Kind.STATIC_CLUSTERING; + +/** + * An abstract virtual table implementation that builds the resultset on demand and allows fine-grained source + * modification via INSERT/UPDATE, DELETE and TRUNCATE operations. + * + * Virtual table implementation need to be thread-safe has they can be called from different threads. + */ +public abstract class AbstractMutableLazyVirtualTable extends AbstractLazyVirtualTable +{ + protected AbstractMutableLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted) + { + super(metadata, onTimeout, sorted); + } + + protected AbstractMutableLazyVirtualTable(TableMetadata metadata, OnTimeout onTimeout, Sorted sorted, Sorted sortedByPartitionKey) + { + super(metadata, onTimeout, sorted, sortedByPartitionKey); + } + + protected void applyPartitionDeletion(Object[] partitionKeys) + { + throw invalidRequest("Partition deletion is not supported by table %s", metadata()); + } + + protected void applyRangeTombstone(Object[] partitionKey, Object[] start, boolean startInclusive, Object[] end, boolean endInclusive) + { + throw invalidRequest("Range deletion is not supported by table %s", metadata()); + } + + protected void applyRowDeletion(Object[] partitionKey, @Nullable Object[] clusteringKeys) + { + throw invalidRequest("Row deletion is not supported by table %s", metadata()); + } + + protected void applyRowUpdate(Object[] partitionKey, @Nullable Object[] clusteringColumns, ColumnMetadata[] columns, Object[] values) + { + throw invalidRequest("Column modification is not supported by table %s", metadata()); + } + + private void applyRangeTombstone(Object[] pks, RangeTombstone rt) + { + Slice slice = rt.deletedSlice(); + Object[] starts = composeClusterings(slice.start(), metadata()); + Object[] ends = composeClusterings(slice.end(), metadata()); + applyRangeTombstone(pks, starts, slice.start().isInclusive(), ends, slice.end().isInclusive()); + } + + private void applyRow(Object[] pks, Row row) + { + Object[] cks = row.clustering().kind() == STATIC_CLUSTERING ? null : composeClusterings(row.clustering(), metadata()); + if (!row.deletion().isLive()) + { + applyRowDeletion(pks, cks); + } + else + { + ColumnMetadata[] columns = new ColumnMetadata[row.columnCount()]; + Object[] values = new Object[row.columnCount()]; + int i = 0; + for (ColumnData cd : row) + { + ColumnMetadata cm = cd.column(); + if (cm.isComplex()) + throw new InvalidRequestException(metadata() + " does not support complex column updates"); + Cell cell = (Cell)cd; + columns[i] = cm; + if (!cell.isTombstone()) + values[i] = cm.type.compose(cell.value(), cell.accessor()); + ++i; + } + applyRowUpdate(pks, cks, columns, values); + } + } + + public void apply(PartitionUpdate update) + { + TableMetadata metadata = metadata(); + Object[] pks = composePartitionKeys(update.partitionKey(), metadata); + + DeletionInfo deletionInfo = update.deletionInfo(); + if (!deletionInfo.getPartitionDeletion().isLive()) + { + applyPartitionDeletion(pks); + } + else if (deletionInfo.hasRanges()) + { + Iterator iter = deletionInfo.rangeIterator(false); + while (iter.hasNext()) + applyRangeTombstone(pks, iter.next()); + } + else + { + for (Row row : update) + applyRow(pks, row); + if (!update.staticRow().isEmpty()) + applyRow(pks, update.staticRow()); + } + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java index 84a0129ca679..1f796bdd151e 100644 --- a/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugKeyspace.java @@ -18,20 +18,18 @@ package org.apache.cassandra.db.virtual; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.Date; -import java.util.HashSet; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; -import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.UnaryOperator; @@ -39,28 +37,17 @@ import javax.annotation.Nullable; -import com.google.common.collect.BoundType; -import com.google.common.collect.Range; -import com.google.common.collect.Sets; - import accord.coordinate.AbstractCoordination; import accord.coordinate.Coordination; import accord.coordinate.Coordinations; import accord.coordinate.PrepareRecovery; import accord.coordinate.tracking.AbstractTracker; +import accord.local.cfk.CommandsForKey.TxnInfo; import accord.primitives.RoutingKeys; import accord.utils.SortedListMap; -import org.apache.cassandra.cql3.Operator; -import org.apache.cassandra.db.EmptyIterators; import org.apache.cassandra.db.PartitionPosition; -import org.apache.cassandra.db.filter.ClusteringIndexFilter; -import org.apache.cassandra.db.filter.ColumnFilter; -import org.apache.cassandra.db.filter.DataLimits; -import org.apache.cassandra.db.filter.RowFilter; import org.apache.cassandra.db.marshal.CompositeType; -import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator; -import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; -import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.db.marshal.TxnIdUtf8Type; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,10 +82,8 @@ import accord.primitives.ProgressToken; import accord.primitives.Route; import accord.primitives.SaveStatus; -import accord.primitives.Status; import accord.primitives.Timestamp; import accord.primitives.TxnId; -import accord.utils.Invariants; import accord.utils.UnhandledEnum; import accord.utils.async.AsyncChain; import accord.utils.async.AsyncChains; @@ -113,11 +98,11 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.Int32Type; import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.dht.LocalPartitioner; import org.apache.cassandra.dht.NormalizedRanges; import org.apache.cassandra.dht.Token; import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; @@ -129,7 +114,7 @@ import org.apache.cassandra.service.accord.AccordKeyspace; import org.apache.cassandra.service.accord.AccordService; import org.apache.cassandra.service.accord.AccordTracing; -import org.apache.cassandra.service.accord.CommandStoreTxnBlockedGraph; +import org.apache.cassandra.service.accord.DebugBlockedTxns; import org.apache.cassandra.service.accord.IAccordService; import org.apache.cassandra.service.accord.JournalKey; import org.apache.cassandra.service.accord.api.AccordAgent; @@ -156,8 +141,12 @@ import static java.lang.String.format; import static java.util.concurrent.TimeUnit.NANOSECONDS; import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest; +import static org.apache.cassandra.db.virtual.AbstractLazyVirtualTable.OnTimeout.BEST_EFFORT; +import static org.apache.cassandra.db.virtual.AbstractLazyVirtualTable.OnTimeout.FAIL; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.ASC; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.SORTED; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.UNSORTED; import static org.apache.cassandra.schema.SchemaConstants.VIRTUAL_ACCORD_DEBUG; -import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; import static org.apache.cassandra.utils.MonotonicClock.Global.approxTime; public class AccordDebugKeyspace extends VirtualKeyspace @@ -181,6 +170,8 @@ public class AccordDebugKeyspace extends VirtualKeyspace public static final String TXN_TRACES = "txn_traces"; public static final String TXN_OPS = "txn_ops"; + private static final Function TO_STRING = AccordDebugKeyspace::toStringOrNull; + public static final AccordDebugKeyspace instance = new AccordDebugKeyspace(); private AccordDebugKeyspace() @@ -227,7 +218,7 @@ private ExecutorsTable() " keys_loading text,\n" + " keys_loading_for text,\n" + " PRIMARY KEY (executor_id, status, position, unique_position)" + - ')', Int32Type.instance), Sorted.ASC); + ')', Int32Type.instance), FAIL, ASC); } @Override @@ -237,27 +228,28 @@ public void collect(PartitionsCollector collector) // TODO (desired): we can easily also support sorted collection for DESC queries for (AccordExecutor executor : commandStores.executors()) { - int uniquePos = 0; int executorId = executor.executorId(); - AccordExecutor.TaskInfo prev = null; - PartitionCollector partition = collector.partition(executorId); - for (AccordExecutor.TaskInfo info : executor.taskSnapshot()) - { - if (prev != null && info.status() == prev.status() && info.position() == prev.position()) ++uniquePos; - else uniquePos = 0; - prev = info; - PreLoadContext preLoadContext = info.preLoadContext(); - partition.row(info.status().name(), info.position(), uniquePos) - .lazyAdd(columns -> { - columns.add("description", info.describe()) - .add("command_store_id", info.commandStoreId()) - .add("txn_id", preLoadContext == null ? null : toStringOrNull(preLoadContext.primaryTxnId())) - .add("txn_id_additional", preLoadContext == null ? null : toStringOrNull(preLoadContext.additionalTxnId())) - .add("keys", preLoadContext == null ? null : toStringOrNull(preLoadContext.keys())) - .add("keys_loading", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeys())) - .add("keys_loading_for", preLoadContext == null ? null : toStringOrNull(preLoadContext.loadKeysFor())); - }); - } + collector.partition(executorId).collect(rows -> { + int uniquePos = 0; + AccordExecutor.TaskInfo prev = null; + for (AccordExecutor.TaskInfo info : executor.taskSnapshot()) + { + if (prev != null && info.status() == prev.status() && info.position() == prev.position()) ++uniquePos; + else uniquePos = 0; + prev = info; + PreLoadContext preLoadContext = info.preLoadContext(); + rows.add(info.status().name(), info.position(), uniquePos) + .lazyCollect(columns -> { + columns.add("description", info.describe()) + .add("command_store_id", info.commandStoreId()) + .add("txn_id", preLoadContext, PreLoadContext::primaryTxnId, TO_STRING) + .add("txn_id_additional", preLoadContext, PreLoadContext::additionalTxnId, TO_STRING) + .add("keys", preLoadContext, PreLoadContext::keys, TO_STRING) + .add("keys_loading", preLoadContext, PreLoadContext::loadKeys, TO_STRING) + .add("keys_loading_for", preLoadContext, PreLoadContext::loadKeysFor, TO_STRING); + }); + } + }); } } } @@ -281,7 +273,7 @@ private CoordinationsTable() " replies text,\n" + " tracker text,\n" + " PRIMARY KEY (txn_id, kind, coordination_id)" + - ')', UTF8Type.instance), Sorted.UNSORTED); + ')', TxnIdUtf8Type.instance), FAIL, UNSORTED); } @Override @@ -291,37 +283,27 @@ public void collect(PartitionsCollector collector) for (Coordination c : coordinations) { collector.row(toStringOrNull(c.txnId()), c.kind().toString(), c.coordinationId()) - .lazyAdd(columns -> { - columns.add("nodes", toStringOrNull(c.nodes())) - .add("nodes_inflight", toStringOrNull(c.inflight())) - .add("nodes_contacted", toStringOrNull(c.contacted())) - .add("description", c.describe()) - .add("participants", toStringOrNull(c.scope())) - .add("replies", summarise(c.replies())) - .add("tracker", summarise(c.tracker())); + .lazyCollect(columns -> { + columns.add("nodes", c, Coordination::nodes, TO_STRING) + .add("nodes_inflight", c, Coordination::inflight, TO_STRING) + .add("nodes_contacted", c, Coordination::contacted, TO_STRING) + .add("description", c, Coordination::describe, TO_STRING) + .add("participants", c, Coordination::scope, TO_STRING) + .add("replies", c, Coordination::replies, CoordinationsTable::summarise) + .add("tracker", c, Coordination::tracker, AbstractTracker::summariseTracker); }); } } - private static String summarise(@Nullable SortedListMap replies) + private static String summarise(SortedListMap replies) { - if (replies == null) - return null; return AbstractCoordination.summariseReplies(replies, 60); } - - private static String summarise(@Nullable AbstractTracker tracker) - { - if (tracker == null) - return null; - return tracker.summariseTracker(); - } } - - public static final class CommandsForKeyTable extends AbstractLazyVirtualTable + private static abstract class AbstractCommandsForKeyTable extends AbstractLazyVirtualTable { - static class Entry + static class Entry implements Comparable { final int commandStoreId; final CommandsForKey cfk; @@ -331,35 +313,29 @@ static class Entry this.commandStoreId = commandStoreId; this.cfk = cfk; } + + @Override + public int compareTo(Entry that) + { + return Integer.compare(this.commandStoreId, that.commandStoreId); + } } - private CommandsForKeyTable() + + AbstractCommandsForKeyTable(TableMetadata metadata) { - super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY, - "Accord per-CommandStore CommandsForKey Managed Transaction State", - "CREATE TABLE %s (\n" + - " key text,\n" + - " command_store_id int,\n" + - " txn_id 'TxnIdUtf8Type',\n" + - " ballot text,\n" + - " deps_known_before text,\n" + - " execute_at text,\n" + - " flags text,\n" + - " missing text,\n" + - " status text,\n" + - " status_overrides text,\n" + - " PRIMARY KEY (key, command_store_id, txn_id)" + - ')', UTF8Type.instance), Sorted.ASC); + super(metadata, BEST_EFFORT, SORTED); } + abstract void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk); + @Override public void collect(PartitionsCollector collector) { - DecoratedKey partitionKey = collector.singleKey(); + Object[] partitionKey = collector.singlePartitionKey(); if (partitionKey == null) - throw new InvalidRequestException("This table currently only supports querying single partitions"); + throw new InvalidRequestException(metadata + " currently only supports querying single partitions"); - String keyStr = UTF8Type.instance.compose(partitionKey.getKey()); - TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); + TokenKey key = TokenKey.parse((String) partitionKey[0], DatabaseDescriptor.getPartitioner()); List cfks = new CopyOnWriteArrayList<>(); CommandStores commandStores = AccordService.instance().node().commandStores(); @@ -375,28 +351,55 @@ public void collect(PartitionsCollector collector) if (cfks.isEmpty()) return; - for (Entry e : cfks) - { - CommandsForKey cfk = e.cfk; - PartitionCollector partition = collector.partition(keyStr); + cfks.sort(collector.dataRange().isReversed() ? Comparator.reverseOrder() : Comparator.naturalOrder()); + for (Entry entry : cfks) + collect(collector.partition(partitionKey[0]), entry.commandStoreId, entry.cfk); + } + } + + public static final class CommandsForKeyTable extends AbstractCommandsForKeyTable + { + private CommandsForKeyTable() + { + super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY, + "Accord per-CommandStore CommandsForKey Managed Transaction State", + "CREATE TABLE %s (\n" + + " key text,\n" + + " command_store_id int,\n" + + " txn_id 'TxnIdUtf8Type',\n" + + " ballot text,\n" + + " deps_known_before text,\n" + + " execute_at text,\n" + + " flags text,\n" + + " missing text,\n" + + " status text,\n" + + " status_overrides text,\n" + + " PRIMARY KEY (key, command_store_id, txn_id)" + + ')', UTF8Type.instance)); + } + + @Override + void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk) + { + partition.collect(rows -> { for (int i = 0 ; i < cfk.size() ; ++i) { - CommandsForKey.TxnInfo txn = cfk.get(i); - partition.row(e.commandStoreId, txn.plainTxnId().toString()) - .lazyAdd(columns -> { - columns.add("ballot", toStringOrNull(txn.ballot())) - .add("deps_known_before", toStringOrNull(txn.depsKnownUntilExecuteAt())) - .add("flags", flags(txn)) - .add("execute_at", toStringOrNull(txn.plainExecuteAt())) - .add("missing", Arrays.toString(txn.missing())) - .add("status", toStringOrNull(txn.status())) - .add("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); - }); + TxnInfo txn = cfk.get(i); + rows.add(commandStoreId, txn.plainTxnId().toString()) + .lazyCollect(columns -> { + columns.add("ballot", txn.ballot(), AccordDebugKeyspace::toStringOrNull) + .add("deps_known_before", txn, TxnInfo::depsKnownUntilExecuteAt, TO_STRING) + .add("flags", txn, CommandsForKeyTable::flags) + .add("execute_at", txn, TxnInfo::plainExecuteAt, TO_STRING) + .add("missing", txn, TxnInfo::missing, Arrays::toString) + .add("status", txn, TxnInfo::status, TO_STRING) + .add("status_overrides", txn.statusOverrides() == 0 ? null : ("0x" + Integer.toHexString(txn.statusOverrides()))); + }); } - } + }); } - private static String flags(CommandsForKey.TxnInfo txn) + private static String flags(TxnInfo txn) { StringBuilder sb = new StringBuilder(); if (!txn.mayExecute()) @@ -417,19 +420,8 @@ private static String flags(CommandsForKey.TxnInfo txn) } } - public static final class CommandsForKeyUnmanagedTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + public static final class CommandsForKeyUnmanagedTable extends AbstractCommandsForKeyTable { - static class Entry - { - final int commandStoreId; - final CommandsForKey cfk; - - Entry(int commandStoreId, CommandsForKey cfk) - { - this.commandStoreId = commandStoreId; - this.cfk = cfk; - } - } private CommandsForKeyUnmanagedTable() { super(parse(VIRTUAL_ACCORD_DEBUG, COMMANDS_FOR_KEY_UNMANAGED, @@ -445,57 +437,19 @@ private CommandsForKeyUnmanagedTable() } @Override - public DataSet data() - { - return this; - } - - @Override - public boolean isEmpty() - { - return false; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) + void collect(PartitionCollector partition, int commandStoreId, CommandsForKey cfk) { - String keyStr = UTF8Type.instance.compose(partitionKey.getKey()); - TokenKey key = TokenKey.parse(keyStr, DatabaseDescriptor.getPartitioner()); - - List cfks = new CopyOnWriteArrayList<>(); - CommandStores commandStores = AccordService.instance().node().commandStores(); - AccordService.getBlocking(commandStores.forEach("commands_for_key_unmanaged table query", RoutingKeys.of(key), Long.MIN_VALUE, Long.MAX_VALUE, safeStore -> { - SafeCommandsForKey safeCfk = safeStore.get(key); - CommandsForKey cfk = safeCfk.current(); - if (cfk == null) - return; - - cfks.add(new Entry(safeStore.commandStore().id(), cfk)); - })); - - if (cfks.isEmpty()) - return null; - - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : cfks) - { - CommandsForKey cfk = e.cfk; + partition.collect(rows -> { for (int i = 0 ; i < cfk.unmanagedCount() ; ++i) { CommandsForKey.Unmanaged txn = cfk.getUnmanaged(i); - ds.row(keyStr, e.commandStoreId, toStringOrNull(txn.txnId)) - .column("waiting_until", toStringOrNull(txn.waitingUntil)) - .column("waiting_until_status", toStringOrNull(txn.pending)); + rows.add(commandStoreId, toStringOrNull(txn.txnId)) + .lazyCollect(columns -> { + columns.add("waiting_until", txn.waitingUntil, TO_STRING) + .add("waiting_until_status", txn.pending, TO_STRING); + }); } - } - - return ds.getPartition(partitionKey); - } - - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); + }); } } @@ -526,7 +480,7 @@ private DurabilityServiceTable() " stopping boolean,\n" + " stopped boolean,\n" + " PRIMARY KEY (table_id, token_start)" + - ')', UTF8Type.instance), Sorted.UNSORTED); + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override @@ -538,7 +492,7 @@ public void collect(PartitionsCollector collector) { TableId tableId = (TableId) view.shard().range.start().prefix(); collector.row(tableId.toString(), printToken(view.shard().range.start())) - .eagerAdd(columns -> { + .eagerCollect(columns -> { columns.add("token_end", printToken(view.shard().range.end())) .add("last_started_at", approxTime.translate().toMillisSinceEpoch(view.lastStartedAtMicros() * 1000)) .add("cycle_started_at", approxTime.translate().toMillisSinceEpoch(view.cycleStartedAtMicros() * 1000)) @@ -574,7 +528,7 @@ private DurableBeforeTable() " quorum 'TxnIdUtf8Type',\n" + " universal 'TxnIdUtf8Type',\n" + " PRIMARY KEY (table_id, token_start)" + - ')', UTF8Type.instance), Sorted.UNSORTED); + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override @@ -585,17 +539,17 @@ public void collect(PartitionsCollector collector) (entry, ignore, start, end) -> { TableId tableId = (TableId) start.prefix(); collector.row(tableId.toString(), printToken(start)) - .lazyAdd(columns -> { - columns.add("token_end", printToken(end)) - .add("quorum", entry.quorumBefore.toString()) - .add("universal", entry.universalBefore.toString()); + .lazyCollect(columns -> { + columns.add("token_end", end, AccordDebugKeyspace::printToken) + .add("quorum", entry.quorumBefore, TO_STRING) + .add("universal", entry.universalBefore, TO_STRING); }); return null; }, null, ignore -> false); } } - public static final class ExecutorCacheTable extends AbstractVirtualTable + public static final class ExecutorCacheTable extends AbstractLazyVirtualTable { private ExecutorCacheTable() { @@ -608,35 +562,34 @@ private ExecutorCacheTable() " hits bigint,\n" + " misses bigint,\n" + " PRIMARY KEY (executor_id, scope)" + - ')', Int32Type.instance)); + ')', Int32Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { AccordCommandStores stores = (AccordCommandStores) AccordService.instance().node().commandStores(); - SimpleDataSet ds = new SimpleDataSet(metadata()); for (AccordExecutor executor : stores.executors()) { try (AccordExecutor.ExclusiveGlobalCaches cache = executor.lockCaches()) { - addRow(ds, executor.executorId(), "commands", cache.commands.statsSnapshot()); - addRow(ds, executor.executorId(), AccordKeyspace.COMMANDS_FOR_KEY, cache.commandsForKey.statsSnapshot()); + addRow(collector, executor.executorId(), "commands", cache.commands.statsSnapshot()); + addRow(collector, executor.executorId(), AccordKeyspace.COMMANDS_FOR_KEY, cache.commandsForKey.statsSnapshot()); } } - return ds; } - private static void addRow(SimpleDataSet ds, int executorId, String scope, AccordCache.ImmutableStats stats) + private static void addRow(PartitionsCollector collector, int executorId, String scope, AccordCache.ImmutableStats stats) { - ds.row(executorId, scope) - .column("queries", stats.hits + stats.misses) - .column("hits", stats.hits) - .column("misses", stats.misses); + collector.row(executorId, scope) + .eagerCollect(columns -> { + columns.add("queries", stats.hits + stats.misses) + .add("hits", stats.hits) + .add("misses", stats.misses); + }); } } - public static final class MaxConflictsTable extends AbstractLazyVirtualTable { private MaxConflictsTable() @@ -650,7 +603,7 @@ private MaxConflictsTable() " token_end 'TokenUtf8Type',\n" + " timestamp text,\n" + " PRIMARY KEY (command_store_id, token_start)" + - ')', Int32Type.instance), Sorted.ASC); + ')', Int32Type.instance), FAIL, ASC); } @Override @@ -665,18 +618,19 @@ public void collect(PartitionsCollector collector) TableId tableId = ((AccordCommandStore) commandStore).tableId(); String tableIdStr = tableId.toString(); - PartitionCollector partition = collector.partition(commandStoreId); - maxConflicts.foldlWithBounds( - (timestamp, p, start, end) -> { - p.row(printToken(start)) - .lazyAdd(columns -> { - columns.add("token_end", printToken(end)) - .add("table_id", tableIdStr) - .add("timestamp", timestamp.toString()); - }); - return p; - }, partition, ignore -> false - ); + collector.partition(commandStoreId).collect(rows -> { + maxConflicts.foldlWithBounds( + (timestamp, rs, start, end) -> { + rows.add(printToken(start)) + .lazyCollect(columns -> { + columns.add("token_end", end, AccordDebugKeyspace::printToken) + .add("table_id", tableIdStr) + .add("timestamp", timestamp, TO_STRING); + }); + return rows; + }, rows, ignore -> false + ); + }); } } } @@ -796,7 +750,7 @@ private ProgressLogTable() " home_retry_counter int,\n" + " home_scheduled_at timestamp,\n" + " PRIMARY KEY (command_store_id, txn_id)" + - ')', Int32Type.instance), Sorted.ASC); + ')', Int32Type.instance), FAIL, ASC); } @Override @@ -808,27 +762,29 @@ public void collect(PartitionsCollector collector) DefaultProgressLog.ImmutableView view = ((DefaultProgressLog) commandStore.unsafeProgressLog()).immutableView(); TableId tableId = ((AccordCommandStore)commandStore).tableId(); String tableIdStr = tableId.toString(); - PartitionCollector partition = collector.partition(commandStore.id()); - while (view.advance()) - { - // TODO (required): view should return an immutable per-row view so that we can call lazyAdd - partition.row(view.txnId().toString()) - .eagerAdd(columns -> { - columns.add("table_id", tableIdStr) - .add("contact_everyone", view.contactEveryone()) - .add("waiting_is_uninitialised", view.isWaitingUninitialised()) - .add("waiting_blocked_until", view.waitingIsBlockedUntil().name()) - .add("waiting_home_satisfies", view.waitingHomeSatisfies().name()) - .add("waiting_progress", view.waitingProgress().name()) - .add("waiting_retry_counter", view.waitingRetryCounter()) - .add("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) - .add("waiting_scheduled_at", view.timerScheduledAt(TxnStateKind.Waiting), ProgressLogTable::toTimestamp) - .add("home_phase", view.homePhase().name()) - .add("home_progress", view.homeProgress().name()) - .add("home_retry_counter", view.homeRetryCounter()) - .add("home_scheduled_at", view.timerScheduledAt(TxnStateKind.Home), ProgressLogTable::toTimestamp); - }); - } + collector.partition(commandStore.id()).collect(collect -> { + while (view.advance()) + { + // TODO (required): view should return an immutable per-row view so that we can call lazyAdd + collect.add(view.txnId().toString()) + .eagerCollect(columns -> { + columns.add("table_id", tableIdStr) + .add("contact_everyone", view.contactEveryone()) + .add("waiting_is_uninitialised", view.isWaitingUninitialised()) + .add("waiting_blocked_until", view.waitingIsBlockedUntil().name()) + .add("waiting_home_satisfies", view.waitingHomeSatisfies().name()) + .add("waiting_progress", view.waitingProgress().name()) + .add("waiting_retry_counter", view.waitingRetryCounter()) + .add("waiting_packed_key_tracker_bits", Long.toBinaryString(view.waitingPackedKeyTrackerBits())) + .add("waiting_scheduled_at", view.timerScheduledAt(TxnStateKind.Waiting), ProgressLogTable::toTimestamp) + .add("home_phase", view.homePhase().name()) + .add("home_progress", view.homeProgress().name()) + .add("home_retry_counter", view.homeRetryCounter()) + .add("home_scheduled_at", view.timerScheduledAt(TxnStateKind.Home), ProgressLogTable::toTimestamp); + }); + } + + }); } } @@ -868,7 +824,7 @@ private RedundantBeforeTable() " unready 'TxnIdUtf8Type',\n" + " stale_until_at_least 'TxnIdUtf8Type',\n" + " PRIMARY KEY (command_store_id, token_start)" + - ')', Int32Type.instance), Sorted.ASC); + ')', Int32Type.instance), FAIL, ASC); } @Override @@ -879,80 +835,75 @@ public void collect(PartitionsCollector collector) for (CommandStore commandStore : commandStores.all()) { int commandStoreId = commandStore.id(); - PartitionCollector partition = collector.partition(commandStoreId); - TableId tableId = ((AccordCommandStore)commandStore).tableId(); - String tableIdStr = tableId.toString(); - commandStore.unsafeGetRedundantBefore().foldl( - (entry, p) -> { - p.row(printToken(entry.range.start())).lazyAdd(columns -> { - columns.add("table_id", tableIdStr) - .add("token_end", printToken(entry.range.end())) - .add("start_epoch", entry.startEpoch) - .add("end_epoch", entry.endEpoch) - .add("gc_before", entry.maxBound(GC_BEFORE).toString()) - .add("shard_applied", entry.maxBound(SHARD_APPLIED).toString()) - .add("quorum_applied", entry.maxBound(QUORUM_APPLIED).toString()) - .add("locally_applied", entry.maxBound(LOCALLY_APPLIED).toString()) - .add("locally_durable_to_command_store", entry.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE).toString()) - .add("locally_durable_to_data_store", entry.maxBound(LOCALLY_DURABLE_TO_DATA_STORE).toString()) - .add("locally_redundant", entry.maxBound(LOCALLY_REDUNDANT).toString()) - .add("locally_synced", entry.maxBound(LOCALLY_SYNCED).toString()) - .add("locally_witnessed", entry.maxBound(LOCALLY_WITNESSED).toString()) - .add("unready", entry.maxBound(UNREADY).toString()) - .add("log_unavailable", entry.maxBound(LOG_UNAVAILABLE).toString()) - .add("stale_until_at_least", entry.staleUntilAtLeast != null ? entry.staleUntilAtLeast.toString() : null); - }); - return p; - }, partition, ignore -> false - ); + collector.partition(commandStoreId).collect(rows -> { + TableId tableId = ((AccordCommandStore)commandStore).tableId(); + String tableIdStr = tableId.toString(); + commandStore.unsafeGetRedundantBefore().foldl( + (entry, rs) -> { + rs.add(printToken(entry.range.start())).lazyCollect(columns -> { + columns.add("table_id", tableIdStr) + .add("token_end", entry.range.end(), AccordDebugKeyspace::printToken) + .add("start_epoch", entry.startEpoch) + .add("end_epoch", entry.endEpoch) + .add("gc_before", entry, e -> e.maxBound(GC_BEFORE), TO_STRING) + .add("shard_applied", entry, e -> e.maxBound(SHARD_APPLIED), TO_STRING) + .add("quorum_applied", entry, e -> e.maxBound(QUORUM_APPLIED), TO_STRING) + .add("locally_applied", entry, e -> e.maxBound(LOCALLY_APPLIED), TO_STRING) + .add("locally_durable_to_command_store", entry, e -> e.maxBound(LOCALLY_DURABLE_TO_COMMAND_STORE), TO_STRING) + .add("locally_durable_to_data_store", entry, e -> e.maxBound(LOCALLY_DURABLE_TO_DATA_STORE), TO_STRING) + .add("locally_redundant", entry, e -> e.maxBound(LOCALLY_REDUNDANT), TO_STRING) + .add("locally_synced", entry, e -> e.maxBound(LOCALLY_SYNCED), TO_STRING) + .add("locally_witnessed", entry, e -> e.maxBound(LOCALLY_WITNESSED), TO_STRING) + .add("log_unavailable", entry, e -> e.maxBound(LOG_UNAVAILABLE), TO_STRING) + .add("unready", entry, e -> e.maxBound(UNREADY), TO_STRING) + .add("stale_until_at_least", entry.staleUntilAtLeast, TO_STRING); + }); + return rs; + }, rows, ignore -> false + ); + }); } } } - public static final class RejectBeforeTable extends AbstractVirtualTable + public static final class RejectBeforeTable extends AbstractLazyVirtualTable { private RejectBeforeTable() { super(parse(VIRTUAL_ACCORD_DEBUG, REJECT_BEFORE, "Accord per-CommandStore RejectBefore State", "CREATE TABLE %s (\n" + - " keyspace_name text,\n" + - " table_name text,\n" + - " table_id text,\n" + " command_store_id int,\n" + " token_start 'TokenUtf8Type',\n" + + " table_id text,\n" + " token_end 'TokenUtf8Type',\n" + " timestamp text,\n" + - " PRIMARY KEY (keyspace_name, table_name, table_id, command_store_id, token_start)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (command_store_id, token_start)" + + ')', UTF8Type.instance), FAIL, ASC); } @Override - public DataSet data() + protected void collect(PartitionsCollector collector) { CommandStores commandStores = AccordService.instance().node().commandStores(); - SimpleDataSet dataSet = new SimpleDataSet(metadata()); for (CommandStore commandStore : commandStores.all()) { RejectBefore rejectBefore = commandStore.unsafeGetRejectBefore(); if (rejectBefore == null) continue; - TableId tableId = ((AccordCommandStore)commandStore).tableId(); - String tableIdStr = tableId.toString(); - TableMetadata tableMetadata = tableMetadata(tableId); - String keyspace = keyspace(tableMetadata); - String table = table(tableId, tableMetadata); - rejectBefore.foldlWithBounds( - (timestamp, ds, start, end) -> ds.row(keyspace, table, tableIdStr, commandStore.id(), printToken(start)) - .column("token_end", printToken(end)) - .column("timestamp", timestamp.toString()) - , - dataSet, - ignore -> false - ); + collector.partition(commandStore.id()).collect(rows -> { + TableId tableId = ((AccordCommandStore)commandStore).tableId(); + String tableIdStr = tableId.toString(); + rejectBefore.foldlWithBounds((timestamp, rs, start, end) -> { + rs.add(printToken(start)) + .lazyCollect(columns -> columns.add("table_id", tableIdStr) + .add("token_end", end, AccordDebugKeyspace::printToken) + .add("timestamp", timestamp, AccordDebugKeyspace::toStringOrNull)); + return rs; + }, rows, ignore -> false); + }); } - return dataSet; } } @@ -969,11 +920,11 @@ private TxnTraceTable() super(parse(VIRTUAL_ACCORD_DEBUG, TXN_TRACE, "Accord Transaction Trace Configuration", "CREATE TABLE %s (\n" + - " txn_id text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " event_type text,\n" + " permits int,\n" + " PRIMARY KEY (txn_id, event_type)" + - ')', UTF8Type.instance)); + ')', TxnIdUtf8Type.instance)); } @Override @@ -1030,21 +981,21 @@ public void truncate() } } - public static final class TxnTracesTable extends AbstractMutableVirtualTable + public static final class TxnTracesTable extends AbstractMutableLazyVirtualTable { private TxnTracesTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN_TRACES, "Accord Transaction Traces", "CREATE TABLE %s (\n" + - " txn_id text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " event_type text,\n" + " id_micros bigint,\n" + " at_micros bigint,\n" + " command_store_id int,\n" + " message text,\n" + " PRIMARY KEY (txn_id, event_type, id_micros, at_micros)" + - ')', UTF8Type.instance)); + ')', TxnIdUtf8Type.instance), FAIL, UNSORTED, UNSORTED); } private AccordTracing tracing() @@ -1053,29 +1004,29 @@ private AccordTracing tracing() } @Override - protected void applyPartitionDeletion(ColumnValues partitionKey) + protected void applyPartitionDeletion(Object[] partitionKeys) { - TxnId txnId = TxnId.parse(partitionKey.value(0)); + TxnId txnId = TxnId.parse((String)partitionKeys[0]); tracing().eraseEvents(txnId); } @Override - protected void applyRangeTombstone(ColumnValues partitionKey, Range range) - { - TxnId txnId = TxnId.parse(partitionKey.value(0)); - if (!range.hasLowerBound() || range.lowerBoundType() != BoundType.CLOSED) throw invalidRequest("May restrict deletion by at most one event_type"); - if (range.lowerEndpoint().size() != 1) throw invalidRequest("Deletion restricted by lower bound on id_micros or at_micros is unsupported"); - if (!range.hasUpperBound() || (range.upperBoundType() != BoundType.CLOSED && range.upperEndpoint().size() == 1)) throw invalidRequest("Range deletion must specify one event_type"); - if (!range.upperEndpoint().value(0).equals(range.lowerEndpoint().value(0))) throw invalidRequest("May restrict deletion by at most one event_type"); - if (range.upperEndpoint().size() > 2) throw invalidRequest("Deletion restricted by upper bound on at_micros is unsupported"); - TraceEventType eventType = parseEventType(range.lowerEndpoint().value(0)); - if (range.upperEndpoint().size() == 1) + protected void applyRangeTombstone(Object[] partitionKeys, Object[] starts, boolean startInclusive, Object[] ends, boolean endInclusive) + { + TxnId txnId = TxnId.parse((String) partitionKeys[0]); + if (!startInclusive) throw invalidRequest("May restrict deletion by at most one event_type"); + if (starts.length != 1) throw invalidRequest("Deletion restricted by lower bound on id_micros or at_micros is unsupported"); + if (ends.length == 0 || (ends.length == 1 && !endInclusive)) throw invalidRequest("Range deletion must specify one event_type"); + if (!ends[0].equals(starts[0])) throw invalidRequest("May restrict deletion by at most one event_type"); + if (ends.length > 2) throw invalidRequest("Deletion restricted by upper bound on at_micros is unsupported"); + TraceEventType eventType = parseEventType((String) starts[0]); + if (ends.length == 1) { tracing().eraseEvents(txnId, eventType); } else { - long before = range.upperEndpoint().value(1); + long before = (Long)ends[1]; tracing().eraseEventsBefore(txnId, eventType, before); } } @@ -1087,60 +1038,40 @@ public void truncate() } @Override - public DataSet data() + public void collect(PartitionsCollector collector) { - SimpleDataSet dataSet = new SimpleDataSet(metadata()); tracing().forEach(id -> true, (txnId, eventType, permits, events) -> { events.forEach(e -> { if (e.messages().isEmpty()) { - dataSet.row(txnId.toString(), eventType.name(), e.idMicros, 0L) - .column("message", ""); - + collector.row(txnId.toString(), eventType.name(), e.idMicros, 0L) + .eagerCollect(columns -> { + columns.add("message", ""); + }); } else { e.messages().forEach(m -> { - dataSet.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) - .column("command_store_id", m.commandStoreId) - .column("message", m.message); + collector.row(txnId.toString(), eventType.name(), e.idMicros, NANOSECONDS.toMicros(m.atNanos - e.atNanos)) + .eagerCollect(columns -> { + columns.add("command_store_id", m.commandStoreId) + .add("message", m.message); + }); }); } }); }); - return dataSet; } } // TODO (desired): don't report null as "null" - public static final class TxnTable extends AbstractLazyVirtualTable + abstract static class AbstractJournalTable extends AbstractLazyVirtualTable { - private static final CompositeType PK = CompositeType.getInstance(Int32Type.instance, UTF8Type.instance); + static final CompositeType PK = CompositeType.getInstance(Int32Type.instance, UTF8Type.instance); - private TxnTable() + AbstractJournalTable(TableMetadata metadata) { - super(parse(VIRTUAL_ACCORD_DEBUG, TXN, - "Accord per-CommandStore Transaction State", - "CREATE TABLE %s (\n" + - " command_store_id int,\n" + - " txn_id text,\n" + - " save_status text,\n" + - " route text,\n" + - " durability text,\n" + - " execute_at text,\n" + - " executes_at_least text,\n" + - " txn text,\n" + - " deps text,\n" + - " waiting_on text,\n" + - " writes text,\n" + - " result text,\n" + - " participants_owns text,\n" + - " participants_touches text,\n" + - " participants_has_touched text,\n" + - " participants_executes text,\n" + - " participants_waits_on text,\n" + - " PRIMARY KEY ((command_store_id, txn_id))" + - ')', PK), Sorted.ASC); + super(metadata, FAIL, ASC); } @Override @@ -1176,7 +1107,7 @@ public void collect(PartitionsCollector collector) FilterRange filterTxnId = collector.filters("txn_id", TxnId::parse, UnaryOperator.identity(), UnaryOperator.identity()); FilterRange filterCommandStoreId = collector.filters("command_store_id", UnaryOperator.identity(), i -> i + 1, i -> i - 1); - int minCommandStoreId = filterCommandStoreId.min == null ? 0 : filterCommandStoreId.min; + int minCommandStoreId = filterCommandStoreId.min == null ? -1 : filterCommandStoreId.min; int maxCommandStoreId = filterCommandStoreId.max == null ? Integer.MAX_VALUE : filterCommandStoreId.max; if (filterTxnId.min != null && filterTxnId.max != null && filterTxnId.min.equals(filterTxnId.max)) @@ -1186,58 +1117,22 @@ public void collect(PartitionsCollector collector) if (commandStore.id() < minCommandStoreId || commandStore.id() > maxCommandStoreId) return; - Command command = ((AccordCommandStore)commandStore).loadCommand(txnId); - if (command == null) - return; - - collector.row(commandStore.id(), txnId.toString()) - .lazyAdd(columns -> addColumns(command, columns)); + collect(collector, accord, new JournalKey(txnId, JournalKey.Type.COMMAND_DIFF, commandStore.id())); }); - return; } - if (filterTxnId.min != null || minCommandStoreId > 0) - min = new JournalKey(filterTxnId.min == null ? TxnId.NONE : filterTxnId.min, JournalKey.Type.COMMAND_DIFF, minCommandStoreId); - if (filterTxnId.max != null || maxCommandStoreId < Integer.MAX_VALUE) + if (filterTxnId.min != null || filterTxnId.max != null || minCommandStoreId >= 0 || maxCommandStoreId < Integer.MAX_VALUE) + { + min = new JournalKey(filterTxnId.min == null ? TxnId.NONE : filterTxnId.min, JournalKey.Type.COMMAND_DIFF, Math.max(0, minCommandStoreId)); max = new JournalKey(filterTxnId.max == null ? TxnId.MAX.withoutNonIdentityFlags() : filterTxnId.max, JournalKey.Type.COMMAND_DIFF, maxCommandStoreId); + } } - accord.journal().forEach(key -> { - if (key.type != JournalKey.Type.COMMAND_DIFF) - return; - - AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); - if (commandStore == null) - return; - - Command command = commandStore.loadCommand(key.id); - if (command == null) - return; - - collector.row(key.commandStoreId, key.id.toString()) - .lazyAdd(columns -> addColumns(command, columns)); - }, min, max, true); + accord.journal().forEach(key -> collect(collector, accord, key), min, max, true); } - private static void addColumns(Command command, ColumnsCollector columns) - { - columns.add("save_status", toStringOrNull(command.saveStatus())) - .add("route", toStringOrNull(command.route())) - .add("participants_owns", toStr(command, StoreParticipants::owns, StoreParticipants::stillOwns)) - .add("participants_touches", toStr(command, StoreParticipants::touches, StoreParticipants::stillTouches)) - .add("participants_has_touched", toStringOrNull(command.participants().hasTouched())) - .add("participants_executes", toStr(command, StoreParticipants::executes, StoreParticipants::stillExecutes)) - .add("participants_waits_on", toStr(command, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) - .add("durability", toStringOrNull(command.durability())) - .add("execute_at", toStringOrNull(command.executeAt())) - .add("executes_at_least", toStringOrNull(command.executesAtLeast())) - .add("txn", toStringOrNull(command.partialTxn())) - .add("deps", toStringOrNull(command.partialDeps())) - .add("waiting_on", toStringOrNull(command.waitingOn())) - .add("writes", toStringOrNull(command.writes())) - .add("result", toStringOrNull(command.result())); - } + abstract void collect(PartitionsCollector collector, AccordService accord, JournalKey key); private static JournalKey toJournalKey(PartitionPosition position) { @@ -1252,24 +1147,76 @@ private static JournalKey toJournalKey(PartitionPosition position) } } - public static final class JournalTable extends AbstractVirtualTable implements AbstractVirtualTable.DataSet + // TODO (desired): don't report null as "null" + public static final class TxnTable extends AbstractJournalTable { - static class Entry + private TxnTable() { - final int commandStoreId; - final long segment; - final int position; - final CommandChange.Builder builder; + super(parse(VIRTUAL_ACCORD_DEBUG, TXN, + "Accord per-CommandStore Transaction State", + "CREATE TABLE %s (\n" + + " command_store_id int,\n" + + " txn_id text,\n" + + " save_status text,\n" + + " route text,\n" + + " durability text,\n" + + " execute_at text,\n" + + " executes_at_least text,\n" + + " txn text,\n" + + " deps text,\n" + + " waiting_on text,\n" + + " writes text,\n" + + " result text,\n" + + " participants_owns text,\n" + + " participants_touches text,\n" + + " participants_has_touched text,\n" + + " participants_executes text,\n" + + " participants_waits_on text,\n" + + " PRIMARY KEY ((command_store_id, txn_id))" + + ')', PK)); + } - Entry(int commandStoreId, long segment, int position, CommandChange.Builder builder) - { - this.commandStoreId = commandStoreId; - this.segment = segment; - this.position = position; - this.builder = builder; - } + @Override + void collect(PartitionsCollector collector, AccordService accord, JournalKey key) + { + if (key.type != JournalKey.Type.COMMAND_DIFF) + return; + + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); + if (commandStore == null) + return; + + Command command = commandStore.loadCommand(key.id); + if (command == null) + return; + + collector.row(key.commandStoreId, key.id.toString()) + .lazyCollect(columns -> addColumns(command, columns)); + } + + private static void addColumns(Command command, ColumnsCollector columns) + { + StoreParticipants participants = command.participants(); + columns.add("save_status", command.saveStatus(), TO_STRING) + .add("route", participants, StoreParticipants::route, TO_STRING) + .add("participants_owns", participants, p -> toStr(p, StoreParticipants::owns, StoreParticipants::stillOwns)) + .add("participants_touches", participants, p -> toStr(p, StoreParticipants::touches, StoreParticipants::stillTouches)) + .add("participants_has_touched", participants, StoreParticipants::hasTouched, TO_STRING) + .add("participants_executes", participants, p -> toStr(p, StoreParticipants::executes, StoreParticipants::stillExecutes)) + .add("participants_waits_on", participants, p -> toStr(p, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) + .add("durability", command, Command::durability, TO_STRING) + .add("execute_at", command, Command::executeAt, TO_STRING) + .add("executes_at_least", command, Command::executesAtLeast, TO_STRING) + .add("txn", command, Command::partialTxn, TO_STRING) + .add("deps", command, Command::partialDeps, TO_STRING) + .add("waiting_on", command, Command::waitingOn, TO_STRING) + .add("writes", command, Command::writes, TO_STRING) + .add("result", command, Command::result, TO_STRING); } + } + public static final class JournalTable extends AbstractJournalTable + { private JournalTable() { super(parse(VIRTUAL_ACCORD_DEBUG, JOURNAL, @@ -1293,68 +1240,40 @@ private JournalTable() " participants_has_touched text,\n" + " participants_executes text,\n" + " participants_waits_on text,\n" + - " PRIMARY KEY (txn_id, command_store_id, segment, segment_position)" + - ')', UTF8Type.instance)); - } - - @Override - public DataSet data() - { - return this; - } - - @Override - public boolean isEmpty() - { - return false; + " PRIMARY KEY ((command_store_id, txn_id), segment, segment_position)" + + ')', PK)); } @Override - public Partition getPartition(DecoratedKey partitionKey) + void collect(PartitionsCollector collector, AccordService accord, JournalKey key) { - String txnIdStr = UTF8Type.instance.compose(partitionKey.getKey()); - TxnId txnId = TxnId.parse(txnIdStr); - - List entries = new ArrayList<>(); - AccordService.instance().node().commandStores().forAllUnsafe(store -> { - for (AccordJournal.DebugEntry e : ((AccordCommandStore)store).debugCommand(txnId)) - entries.add(new Entry(store.id(), e.segment, e.position, e.builder)); + AccordCommandStore commandStore = (AccordCommandStore) accord.node().commandStores().forId(key.commandStoreId); + collector.partition(key.commandStoreId, key.id.toString()).collect(rows -> { + for (AccordJournal.DebugEntry e : commandStore.debugCommand(key.id)) + { + CommandChange.Builder b = e.builder; + StoreParticipants participants = b.participants() != null ? b.participants() : StoreParticipants.empty(key.id); + rows.add(e.segment, e.position) + .lazyCollect(columns -> { + columns.add("save_status", b.saveStatus(), TO_STRING) + .add("route", participants, StoreParticipants::route, TO_STRING) + .add("participants_owns", participants, p -> toStr(p, StoreParticipants::owns, StoreParticipants::stillOwns)) + .add("participants_touches", participants, p -> toStr(p, StoreParticipants::touches, StoreParticipants::stillTouches)) + .add("participants_has_touched", participants, StoreParticipants::hasTouched, TO_STRING) + .add("participants_executes", participants, p -> toStr(p, StoreParticipants::executes, StoreParticipants::stillExecutes)) + .add("participants_waits_on", participants, p -> toStr(p, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) + .add("durability", b.durability(), TO_STRING) + .add("execute_at", b.executeAt(), TO_STRING) + .add("executes_at_least", b.executesAtLeast(), TO_STRING) + .add("txn", b.partialTxn(), TO_STRING) + .add("deps", b.partialDeps(), TO_STRING) + .add("writes", b.writes(), TO_STRING) + .add("result", b.result(), TO_STRING); + }); + } }); - - if (entries.isEmpty()) - return null; - - SimpleDataSet ds = new SimpleDataSet(metadata); - for (Entry e : entries) - { - CommandChange.Builder b = e.builder; - StoreParticipants participants = b.participants(); - if (participants == null) participants = StoreParticipants.empty(txnId); - ds.row(txnIdStr, e.commandStoreId, e.segment, e.position) - .column("save_status", toStringOrNull(b.saveStatus())) - .column("route", toStringOrNull(participants.route())) - .column("participants_owns", toStr(participants, StoreParticipants::owns, StoreParticipants::stillOwns)) - .column("participants_touches", toStr(participants, StoreParticipants::touches, StoreParticipants::stillTouches)) - .column("participants_has_touched", toStringOrNull(participants.hasTouched())) - .column("participants_executes", toStr(participants, StoreParticipants::executes, StoreParticipants::stillExecutes)) - .column("participants_waits_on", toStr(participants, StoreParticipants::waitsOn, StoreParticipants::stillWaitsOn)) - .column("durability", toStringOrNull(b.durability())) - .column("execute_at", toStringOrNull(b.executeAt())) - .column("executes_at_least", toStringOrNull(b.executesAtLeast())) - .column("txn", toStringOrNull(b.partialTxn())) - .column("deps", toStringOrNull(b.partialDeps())) - .column("writes", toStringOrNull(b.writes())) - .column("result", toStringOrNull(b.result())); - } - - return ds.getPartition(partitionKey); } - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(); - } } /** @@ -1369,7 +1288,7 @@ public Iterator getPartitions(DataRange range) */ // Had to be separate from the "regular" journal table since it does not have segment and position, and command store id is inferred // TODO (required): add access control - public static final class TxnOpsTable extends AbstractMutableVirtualTable implements AbstractVirtualTable.DataSet + public static final class TxnOpsTable extends AbstractMutableLazyVirtualTable { // TODO (expected): test each of these operations enum Op { ERASE_VESTIGIAL, INVALIDATE, TRY_EXECUTE, FORCE_APPLY, FORCE_UPDATE, RECOVER, FETCH, RESET_PROGRESS_LOG } @@ -1382,41 +1301,21 @@ private TxnOpsTable() " command_store_id int,\n" + " op text," + " PRIMARY KEY (txn_id, command_store_id)" + - ')', UTF8Type.instance)); + ')', UTF8Type.instance), FAIL, UNSORTED); } @Override - public DataSet data() + protected void collect(PartitionsCollector collector) { throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); } @Override - public boolean isEmpty() - { - return true; - } - - @Override - public Partition getPartition(DecoratedKey partitionKey) + protected void applyRowUpdate(Object[] partitionKeys, Object[] clusteringKeys, ColumnMetadata[] columns, Object[] values) { - throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); - } - - @Override - public Iterator getPartitions(DataRange range) - { - throw new UnsupportedOperationException(TXN_OPS + " is a write-only table"); - } - - - @Override - protected void applyColumnUpdate(ColumnValues partitionKey, ColumnValues clusteringColumns, Optional columnValue) - { - TxnId txnId = TxnId.parse(partitionKey.value(0)); - int commandStoreId = clusteringColumns.value(0); - Invariants.require(columnValue.isPresent()); - Op op = Op.valueOf(columnValue.get().value()); + TxnId txnId = TxnId.parse((String) partitionKeys[0]); + int commandStoreId = (Integer) clusteringKeys[0]; + Op op = Op.valueOf((String)values[0]); switch (op) { default: throw new UnhandledEnum(op); @@ -1544,117 +1443,57 @@ private void cleanup(TxnId txnId, int commandStoreId, Cleanup cleanup) } } - public static class TxnBlockedByTable extends AbstractVirtualTable + public static class TxnBlockedByTable extends AbstractLazyVirtualTable { - enum Reason { Self, Txn, Key } + enum Reason + {Self, Txn, Key} protected TxnBlockedByTable() { super(parse(VIRTUAL_ACCORD_DEBUG, TXN_BLOCKED_BY, - "Accord Transactions Blocked By Table" , + "Accord Transactions Blocked By Table", "CREATE TABLE %s (\n" + - " txn_id text,\n" + - " keyspace_name text,\n" + - " table_name text,\n" + + " txn_id 'TxnIdUtf8Type',\n" + " command_store_id int,\n" + " depth int,\n" + - " blocked_by text,\n" + - " reason text,\n" + + " blocked_by_key text,\n" + + " blocked_by_txn_id 'TxnIdUtf8Type',\n" + " save_status text,\n" + " execute_at text,\n" + - " key text,\n" + - " PRIMARY KEY (txn_id, keyspace_name, table_name, command_store_id, depth, blocked_by, reason)" + - ')', UTF8Type.instance)); + " PRIMARY KEY (txn_id, command_store_id, depth, blocked_by_key, blocked_by_txn_id)" + + ')', TxnIdUtf8Type.instance), BEST_EFFORT, ASC); } @Override - public UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits) + protected void collect(PartitionsCollector collector) { - Partition partition = data(partitionKey, rowFilter).getPartition(partitionKey); - - if (null == partition) - return EmptyIterators.unfilteredPartition(metadata); + Object[] pks = collector.singlePartitionKey(); + if (pks == null) + throw new InvalidRequestException(metadata + " only supports single partition key queries"); - long now = currentTimeMillis(); - UnfilteredRowIterator rowIterator = partition.toRowIterator(metadata(), clusteringIndexFilter, columnFilter, now); - return new SingletonUnfilteredPartitionIterator(rowIterator); - } + FilterRange depthRange = collector.filters("depth", Function.identity(), i -> i + 1, i -> i - 1); + int maxDepth = depthRange.max == null ? Integer.MAX_VALUE : depthRange.max; - public DataSet data(DecoratedKey partitionKey, RowFilter rowFilter) - { - int maxDepth = Integer.MAX_VALUE; - if (rowFilter != null && rowFilter.getExpressions().size() > 0) - { - Invariants.require(rowFilter.getExpressions().size() == 1, "Only depth filter is supported"); - RowFilter.Expression expression = rowFilter.getExpressions().get(0); - Invariants.require(expression.column().name.toString().equals("depth"), "Only depth filter is supported, but got: %s", expression.column().name); - Invariants.require(expression.operator() == Operator.LT || expression.operator() == Operator.LTE, "Only < and <= queries are supported"); - if (expression.operator() == Operator.LT) - maxDepth = expression.getIndexValue().getInt(0); - else - maxDepth = expression.getIndexValue().getInt(0) + 1; - } - - TxnId id = TxnId.parse(UTF8Type.instance.compose(partitionKey.getKey())); - List shards = AccordService.instance().debugTxnBlockedGraph(id); - - SimpleDataSet ds = new SimpleDataSet(metadata()); - CommandStores commandStores = AccordService.instance().node().commandStores(); - for (CommandStoreTxnBlockedGraph shard : shards) - { - Set processed = new HashSet<>(); - process(ds, commandStores, shard, processed, id, 0, maxDepth, id, Reason.Self, null); - // everything was processed right? - if (!shard.txns.isEmpty() && !shard.txns.keySet().containsAll(processed)) - Invariants.expect(false, "Skipped txns: " + Sets.difference(shard.txns.keySet(), processed)); - } - - return ds; - } - - private void process(SimpleDataSet ds, CommandStores commandStores, CommandStoreTxnBlockedGraph shard, Set processed, TxnId userTxn, int depth, int maxDepth, TxnId txnId, Reason reason, Runnable onDone) - { - if (!processed.add(txnId)) - throw new IllegalStateException("Double processed " + txnId); - CommandStoreTxnBlockedGraph.TxnState txn = shard.txns.get(txnId); - if (txn == null) - { - Invariants.require(reason == Reason.Self, "Txn %s unknown for reason %s", txnId, reason); - return; - } - // was it applied? If so ignore it - if (reason != Reason.Self && txn.saveStatus.hasBeen(Status.Applied)) - return; - TableId tableId = tableId(shard.commandStoreId, commandStores); - TableMetadata tableMetadata = tableMetadata(tableId); - ds.row(userTxn.toString(), keyspace(tableMetadata), table(tableId, tableMetadata), - shard.commandStoreId, depth, reason == Reason.Self ? "" : txn.txnId.toString(), reason.name()); - ds.column("save_status", txn.saveStatus.name()); - if (txn.executeAt != null) - ds.column("execute_at", txn.executeAt.toString()); - if (onDone != null) - onDone.run(); - if (txn.isBlocked()) - { - for (TxnId blockedBy : txn.blockedBy) + TxnId txnId = TxnId.parse((String) pks[0]); + PartitionCollector partition = collector.partition(pks[0]); + partition.collect(rows -> { + try { - if (!processed.contains(blockedBy) && depth < maxDepth) - process(ds, commandStores, shard, processed, userTxn, depth + 1, maxDepth, blockedBy, Reason.Txn, null); + DebugBlockedTxns.visit(AccordService.instance(), txnId, maxDepth, collector.deadlineNanos(), txn -> { + String keyStr = txn.blockedViaKey == null ? "" : txn.blockedViaKey.toString(); + String txnIdStr = txn.txnId == null || txn.txnId.equals(txnId) ? "" : txn.txnId.toString(); + rows.add(txn.commandStoreId, txn.depth, keyStr, txnIdStr) + .eagerCollect(columns -> { + columns.add("save_status", txn.saveStatus, TO_STRING) + .add("execute_at", txn.executeAt, TO_STRING); + }); + }); } - - for (TokenKey blockedBy : txn.blockedByKey) + catch (TimeoutException e) { - TxnId blocking = shard.keys.get(blockedBy); - if (!processed.contains(blocking) && depth < maxDepth) - process(ds, commandStores, shard, processed, userTxn, depth + 1, maxDepth, blocking, Reason.Key, () -> ds.column("key", printToken(blockedBy))); + throw new InternalTimeoutException(); } - } - } - - @Override - public DataSet data() - { - throw new InvalidRequestException("Must select a single txn_id"); + }); } } @@ -1673,33 +1512,12 @@ private static TableMetadata tableMetadata(TableId tableId) return Schema.instance.getTableMetadata(tableId); } - private static String keyspace(TableMetadata metadata) - { - return metadata == null ? "Unknown" : metadata.keyspace; - } - - private static String table(TableId tableId, TableMetadata metadata) - { - return metadata == null ? tableId.toString() : metadata.name; - } - private static String printToken(RoutingKey routingKey) { TokenKey key = (TokenKey) routingKey; return key.token().getPartitioner().getTokenFactory().toString(key.token()); } - private static ByteBuffer sortToken(RoutingKey routingKey) - { - TokenKey key = (TokenKey) routingKey; - Token token = key.token(); - IPartitioner partitioner = token.getPartitioner(); - ByteBuffer out = ByteBuffer.allocate(partitioner.accordSerializedSize(token)); - partitioner.accordSerialize(token, out); - out.flip(); - return out; - } - private static TableMetadata parse(String keyspace, String table, String comment, String schema, AbstractType partitionKeyType) { return CreateTableStatement.parse(format(schema, table), keyspace) @@ -1709,13 +1527,11 @@ private static TableMetadata parse(String keyspace, String table, String comment .build(); } - private static String toStr(Command command, Function> a, Function> b) - { - return toStr(command.participants(), a, b); - } - private static String toStr(StoreParticipants participants, Function> a, Function> b) { + if (participants == null) + return null; + Participants av = a.apply(participants); Participants bv = b.apply(participants); if (av == bv || av.equals(bv)) diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java index 51aa5140de03..3f05ed3a690c 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java @@ -32,6 +32,8 @@ */ public interface VirtualTable { + enum Sorted { UNSORTED, ASC, DESC, SORTED } + /** * Returns the view name. * @@ -62,7 +64,7 @@ default String name() * @param clusteringIndexFilter the clustering columns to selected * @param columnFilter the selected columns * @param rowFilter filter on which rows a given query should include or exclude - * @param limits + * @param limits result limits to apply * @return the rows corresponding to the requested data. */ UnfilteredPartitionIterator select(DecoratedKey partitionKey, ClusteringIndexFilter clusteringIndexFilter, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits); diff --git a/src/java/org/apache/cassandra/service/accord/AccordService.java b/src/java/org/apache/cassandra/service/accord/AccordService.java index 34b7dd28c4b0..6742f2484369 100644 --- a/src/java/org/apache/cassandra/service/accord/AccordService.java +++ b/src/java/org/apache/cassandra/service/accord/AccordService.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -40,6 +39,7 @@ import com.google.common.primitives.Ints; import accord.api.ConfigurationService.EpochReady; +import accord.primitives.Txn; import org.apache.cassandra.metrics.AccordReplicaMetrics; import org.apache.cassandra.service.accord.api.AccordViolationHandler; import org.apache.cassandra.utils.Clock; @@ -58,17 +58,11 @@ import accord.impl.RequestCallbacks; import accord.impl.SizeOfIntersectionSorter; import accord.impl.progresslog.DefaultProgressLogs; -import accord.local.Command; -import accord.local.CommandStore; -import accord.local.CommandStores; import accord.local.Node; import accord.local.Node.Id; -import accord.local.PreLoadContext; -import accord.local.SafeCommand; import accord.local.ShardDistributor.EvenSplit; import accord.local.UniqueTimeService.AtomicUniqueTimeWithStaleReservation; import accord.local.cfk.CommandsForKey; -import accord.local.cfk.SafeCommandsForKey; import accord.local.durability.DurabilityService; import accord.local.durability.ShardDurability; import accord.messages.Reply; @@ -76,13 +70,9 @@ import accord.primitives.FullRoute; import accord.primitives.Keys; import accord.primitives.Ranges; -import accord.primitives.RoutingKeys; -import accord.primitives.SaveStatus; import accord.primitives.Seekable; import accord.primitives.Seekables; -import accord.primitives.Status; import accord.primitives.Timestamp; -import accord.primitives.Txn; import accord.primitives.TxnId; import accord.topology.Shard; import accord.topology.Topology; @@ -90,7 +80,6 @@ import accord.utils.DefaultRandom; import accord.utils.Invariants; import accord.utils.async.AsyncChain; -import accord.utils.async.AsyncChains; import accord.utils.async.AsyncResult; import accord.utils.async.AsyncResults; import org.apache.cassandra.concurrent.Shutdownable; @@ -115,7 +104,6 @@ import org.apache.cassandra.service.accord.api.AccordTimeService; import org.apache.cassandra.service.accord.api.AccordTopologySorter; import org.apache.cassandra.service.accord.api.CompositeTopologySorter; -import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.service.accord.api.TokenKey.KeyspaceSplitter; import org.apache.cassandra.service.accord.interop.AccordInteropAdapter.AccordInteropFactory; import org.apache.cassandra.service.accord.serializers.TableMetadatas; @@ -140,8 +128,6 @@ import static accord.api.Journal.TopologyUpdate; import static accord.api.ProtocolModifiers.Toggles.FastExec.MAY_BYPASS_SAFESTORE; -import static accord.local.LoadKeys.SYNC; -import static accord.local.LoadKeysFor.READ_WRITE; import static accord.local.durability.DurabilityService.SyncLocal.Self; import static accord.local.durability.DurabilityService.SyncRemote.All; import static accord.messages.SimpleReply.Ok; @@ -875,139 +861,6 @@ public Id nodeId() return node.id(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return getBlocking(loadDebug(txnId)); - } - - public AsyncChain> loadDebug(TxnId original) - { - CommandStores commandStores = node.commandStores(); - if (commandStores.count() == 0) - return AsyncChains.success(Collections.emptyList()); - int[] ids = commandStores.ids(); - List> chains = new ArrayList<>(ids.length); - for (int id : ids) - chains.add(loadDebug(original, commandStores.forId(id)).chain()); - return AsyncChains.allOf(chains); - } - - private AsyncResult loadDebug(TxnId txnId, CommandStore store) - { - CommandStoreTxnBlockedGraph.Builder state = new CommandStoreTxnBlockedGraph.Builder(store.id()); - populateAsync(state, store, txnId); - return state; - } - - private static void populate(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TxnId blockedBy) - { - if (safeStore.ifLoadedAndInitialised(blockedBy) != null) populateSync(state, safeStore, blockedBy); - else populateAsync(state, safeStore.commandStore(), blockedBy); - } - - private static void populateAsync(CommandStoreTxnBlockedGraph.Builder state, CommandStore store, TxnId txnId) - { - state.asyncTxns.incrementAndGet(); - store.execute(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), in -> { - populateSync(state, (AccordSafeCommandStore) in, txnId); - if (0 == state.asyncTxns.decrementAndGet() && 0 == state.asyncKeys.get()) - state.complete(); - }); - } - - @Nullable - private static void populateSync(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TxnId txnId) - { - try - { - if (state.txns.containsKey(txnId)) - return; // could plausibly request same txn twice - - SafeCommand safeCommand = safeStore.unsafeGet(txnId); - Invariants.nonNull(safeCommand, "Txn %s is not in the cache", txnId); - if (safeCommand.current() == null || safeCommand.current().saveStatus() == SaveStatus.Uninitialised) - return; - - CommandStoreTxnBlockedGraph.TxnState cmdTxnState = populateSync(state, safeCommand.current()); - if (cmdTxnState.notBlocked()) - return; - - for (TxnId blockedBy : cmdTxnState.blockedBy) - { - if (!state.knows(blockedBy)) - populate(state, safeStore, blockedBy); - } - for (TokenKey blockedBy : cmdTxnState.blockedByKey) - { - if (!state.keys.containsKey(blockedBy)) - populate(state, safeStore, blockedBy, txnId, safeCommand.current().executeAt()); - } - } - catch (Throwable t) - { - state.tryFailure(t); - } - } - - private static void populate(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TokenKey blockedBy, TxnId txnId, Timestamp executeAt) - { - if (safeStore.ifLoadedAndInitialised(txnId) != null && safeStore.ifLoadedAndInitialised(blockedBy) != null) populateSync(state, safeStore, blockedBy, txnId, executeAt); - else populateAsync(state, safeStore.commandStore(), blockedBy, txnId, executeAt); - } - - private static void populateAsync(CommandStoreTxnBlockedGraph.Builder state, CommandStore commandStore, TokenKey blockedBy, TxnId txnId, Timestamp executeAt) - { - state.asyncKeys.incrementAndGet(); - commandStore.execute(PreLoadContext.contextFor(txnId, RoutingKeys.of(blockedBy.toUnseekable()), SYNC, READ_WRITE, "Populate txn_blocked_by"), in -> { - populateSync(state, (AccordSafeCommandStore) in, blockedBy, txnId, executeAt); - if (0 == state.asyncKeys.decrementAndGet() && 0 == state.asyncTxns.get()) - state.complete(); - }); - } - - private static void populateSync(CommandStoreTxnBlockedGraph.Builder state, AccordSafeCommandStore safeStore, TokenKey pk, TxnId txnId, Timestamp executeAt) - { - try - { - SafeCommandsForKey commandsForKey = safeStore.ifLoadedAndInitialised(pk); - TxnId blocking = commandsForKey.current().blockedOnTxnId(txnId, executeAt); - if (blocking instanceof CommandsForKey.TxnInfo) - blocking = ((CommandsForKey.TxnInfo) blocking).plainTxnId(); - state.keys.put(pk, blocking); - if (state.txns.containsKey(blocking)) - return; - populate(state, safeStore, blocking); - } - catch (Throwable t) - { - state.tryFailure(t); - } - } - - private static CommandStoreTxnBlockedGraph.TxnState populateSync(CommandStoreTxnBlockedGraph.Builder state, Command cmd) - { - CommandStoreTxnBlockedGraph.Builder.TxnBuilder cmdTxnState = state.txn(cmd.txnId(), cmd.executeAt(), cmd.saveStatus()); - if (!cmd.hasBeen(Status.Applied) && cmd.hasBeen(Status.Stable)) - { - // check blocking state - Command.WaitingOn waitingOn = cmd.asCommitted().waitingOn(); - waitingOn.waitingOn.reverseForEach(null, null, null, null, (i1, i2, i3, i4, i) -> { - if (i < waitingOn.txnIdCount()) - { - // blocked on txn - cmdTxnState.blockedBy.add(waitingOn.txnId(i)); - } - else - { - // blocked on key - cmdTxnState.blockedByKey.add((TokenKey) waitingOn.keys.get(i - waitingOn.txnIdCount())); - } - }); - } - return cmdTxnState.build(); - } - @Override public long minEpoch() { diff --git a/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java b/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java deleted file mode 100644 index 7d553dda7c59..000000000000 --- a/src/java/org/apache/cassandra/service/accord/CommandStoreTxnBlockedGraph.java +++ /dev/null @@ -1,135 +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.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - -import accord.primitives.SaveStatus; -import accord.primitives.Timestamp; -import accord.primitives.TxnId; -import accord.utils.async.AsyncResults; -import org.apache.cassandra.service.accord.api.TokenKey; - -public class CommandStoreTxnBlockedGraph -{ - public final int commandStoreId; - public final Map txns; - public final Map keys; - - public CommandStoreTxnBlockedGraph(Builder builder) - { - commandStoreId = builder.storeId; - txns = ImmutableMap.copyOf(builder.txns); - keys = ImmutableMap.copyOf(builder.keys); - } - - public static class TxnState - { - public final TxnId txnId; - public final Timestamp executeAt; - public final SaveStatus saveStatus; - public final List blockedBy; - public final Set blockedByKey; - - public TxnState(Builder.TxnBuilder builder) - { - txnId = builder.txnId; - executeAt = builder.executeAt; - saveStatus = builder.saveStatus; - blockedBy = ImmutableList.copyOf(builder.blockedBy); - blockedByKey = ImmutableSet.copyOf(builder.blockedByKey); - } - - public boolean isBlocked() - { - return !notBlocked(); - } - - public boolean notBlocked() - { - return blockedBy.isEmpty() && blockedByKey.isEmpty(); - } - } - - public static class Builder extends AsyncResults.SettableResult - { - final AtomicInteger asyncTxns = new AtomicInteger(), asyncKeys = new AtomicInteger(); - final int storeId; - final Map txns = new LinkedHashMap<>(); - final Map keys = new LinkedHashMap<>(); - - public Builder(int storeId) - { - this.storeId = storeId; - } - - boolean knows(TxnId id) - { - return txns.containsKey(id); - } - - public void complete() - { - trySuccess(build()); - } - - public CommandStoreTxnBlockedGraph build() - { - return new CommandStoreTxnBlockedGraph(this); - } - - public TxnBuilder txn(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus) - { - return new TxnBuilder(txnId, executeAt, saveStatus); - } - - public class TxnBuilder - { - final TxnId txnId; - final Timestamp executeAt; - final SaveStatus saveStatus; - List blockedBy = new ArrayList<>(); - Set blockedByKey = new LinkedHashSet<>(); - - public TxnBuilder(TxnId txnId, Timestamp executeAt, SaveStatus saveStatus) - { - this.txnId = txnId; - this.executeAt = executeAt; - this.saveStatus = saveStatus; - } - - public TxnState build() - { - TxnState state = new TxnState(this); - txns.put(txnId, state); - return state; - } - } - } -} diff --git a/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java b/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java new file mode 100644 index 000000000000..a4ca0af5e1dc --- /dev/null +++ b/src/java/org/apache/cassandra/service/accord/DebugBlockedTxns.java @@ -0,0 +1,249 @@ +/* + * 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.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import javax.annotation.Nullable; + +import accord.api.RoutingKey; +import accord.local.Command; +import accord.local.CommandStore; +import accord.local.CommandStores; +import accord.local.PreLoadContext; +import accord.local.SafeCommandStore; +import accord.local.cfk.CommandsForKey; +import accord.local.cfk.SafeCommandsForKey; +import accord.primitives.RoutingKeys; +import accord.primitives.SaveStatus; +import accord.primitives.Status; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.async.AsyncChain; +import accord.utils.async.AsyncChains; +import org.apache.cassandra.service.accord.api.TokenKey; +import org.apache.cassandra.utils.concurrent.Future; + +import static accord.local.LoadKeys.SYNC; +import static accord.local.LoadKeysFor.READ_WRITE; +import static java.util.Collections.emptyList; + +public class DebugBlockedTxns +{ + public static class Txn implements Comparable + { + public final int commandStoreId; + public final int depth; + public final TxnId txnId; + public final Timestamp executeAt; + public final SaveStatus saveStatus; + public final RoutingKey blockedViaKey; + public final List blockedBy; + public final List blockedByKey; + + public Txn(int commandStoreId, int depth, TxnId txnId, Timestamp executeAt, SaveStatus saveStatus, RoutingKey blockedViaKey, List blockedBy, List blockedByKey) + { + this.commandStoreId = commandStoreId; + this.depth = depth; + this.txnId = txnId; + this.executeAt = executeAt; + this.saveStatus = saveStatus; + this.blockedViaKey = blockedViaKey; + this.blockedBy = blockedBy; + this.blockedByKey = blockedByKey; + } + + public boolean isBlocked() + { + return !notBlocked(); + } + + public boolean notBlocked() + { + return blockedBy.isEmpty() && blockedByKey.isEmpty(); + } + + @Override + public int compareTo(Txn that) + { + int c = Integer.compare(this.commandStoreId, that.commandStoreId); + if (c == 0) c = Integer.compare(this.depth, that.depth); + if (c == 0) c = this.txnId.compareTo(that.txnId); + return c; + } + } + + final IAccordService service; + final Consumer visit; + final TxnId root; + final int maxDepth; + final Set visited = Collections.newSetFromMap(new ConcurrentHashMap<>()); + final ConcurrentLinkedQueue> queuedKeys = new ConcurrentLinkedQueue<>(); + final ConcurrentLinkedQueue> queuedTxn = new ConcurrentLinkedQueue<>(); + + public DebugBlockedTxns(IAccordService service, TxnId root, int maxDepth, Consumer visit) + { + this.service = service; + this.visit = visit; + this.root = root; + this.maxDepth = maxDepth; + } + + public static void visit(IAccordService accord, TxnId txnId, int maxDepth, long deadlineNanos, Consumer visit) throws TimeoutException + { + new DebugBlockedTxns(accord, txnId, maxDepth, visit).visit(deadlineNanos); + } + + private void visit(long deadlineNanos) throws TimeoutException + { + CommandStores commandStores = service.node().commandStores(); + if (commandStores.count() == 0) + return; + + int[] ids = commandStores.ids(); + List> chains = new ArrayList<>(ids.length); + for (int id : ids) + chains.add(visitRootTxnAsync(commandStores.forId(id), root)); + + List tmp = new ArrayList<>(); + Future> next = AccordService.toFuture(AsyncChains.allOf(chains)); + while (next != null) + { + if (!next.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos)) + throw new TimeoutException(); + + next.rethrowIfFailed(); + List process = next.getNow().stream() + .filter(Objects::nonNull) + .sorted(Comparator.naturalOrder()) + .collect(Collectors.toList()); + + for (Txn txn : process) + visit.accept(txn); + + Future> awaitKeys = drainToFuture(queuedKeys, (List>)(List)tmp); + if (awaitKeys != null && !awaitKeys.awaitUntilThrowUncheckedOnInterrupt(deadlineNanos)) + throw new TimeoutException(); + + next = drainToFuture(queuedTxn, (List>)(List)tmp); + } + } + + private Future> drainToFuture(Queue> drain, List> tmp) + { + AsyncChain next; + while (null != (next = drain.poll())) + tmp.add(next); + if (tmp.isEmpty()) + return null; + Future> result = AccordService.toFuture(AsyncChains.allOf(List.copyOf(tmp))); + tmp.clear(); + return result; + } + + private AsyncChain visitRootTxnAsync(CommandStore commandStore, TxnId txnId) + { + return commandStore.chain(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), safeStore -> { + Command command = safeStore.unsafeGetNoCleanup(txnId).current(); + if (command == null || command.saveStatus() == SaveStatus.Uninitialised) + return null; + return visitTxnSync(safeStore, command, command.executeAt(), null, 0); + }); + } + + private AsyncChain visitTxnAsync(CommandStore commandStore, TxnId txnId, Timestamp rootExecuteAt, @Nullable TokenKey byKey, int depth, boolean recurse) + { + return commandStore.chain(PreLoadContext.contextFor(txnId, "Populate txn_blocked_by"), safeStore -> { + Command command = safeStore.unsafeGetNoCleanup(txnId).current(); + if (command == null || command.saveStatus() == SaveStatus.Uninitialised) + return null; + return visitTxnSync(safeStore, command, rootExecuteAt, byKey, depth); + }); + } + + private Txn visitTxnSync(SafeCommandStore safeStore, Command command, Timestamp rootExecuteAt, @Nullable TokenKey byKey, int depth) + { + List waitingOnTxnId = new ArrayList<>(); + List waitingOnKey = new ArrayList<>(); + if (!command.hasBeen(Status.Applied) && command.hasBeen(Status.Stable)) + { + // check blocking state + Command.WaitingOn waitingOn = command.asCommitted().waitingOn(); + waitingOn.waitingOn.reverseForEach(null, null, null, null, (i1, i2, i3, i4, i) -> { + if (i < waitingOn.txnIdCount()) waitingOnTxnId.add(waitingOn.txnId(i)); + else waitingOnKey.add((TokenKey) waitingOn.keys.get(i - waitingOn.txnIdCount())); + }); + } + + CommandStore commandStore = safeStore.commandStore(); + if (depth < maxDepth) + { + for (TxnId waitingOn : waitingOnTxnId) + { + if (visited.add(waitingOn)) + queuedTxn.add(visitTxnAsync(commandStore, waitingOn, rootExecuteAt, null, depth + 1, true)); + } + for (TokenKey key : waitingOnKey) + { + if (visited.add(key)) + queuedKeys.add(visitKeysAsync(commandStore, key, rootExecuteAt, depth + 1)); + } + } + + return new Txn(commandStore.id(), depth, command.txnId(), command.executeAt(), command.saveStatus(), byKey, waitingOnTxnId, waitingOnKey); + } + + + private AsyncChain visitKeysAsync(CommandStore commandStore, TokenKey key, Timestamp rootExecuteAt, int depth) + { + return commandStore.chain(PreLoadContext.contextFor(RoutingKeys.of(key.toUnseekable()), SYNC, READ_WRITE, "Populate txn_blocked_by"), safeStore -> { + visitKeysSync(safeStore, key, rootExecuteAt, depth); + }); + } + + private void visitKeysSync(SafeCommandStore safeStore, TokenKey key, Timestamp rootExecuteAt, int depth) + { + SafeCommandsForKey commandsForKey = safeStore.ifLoadedAndInitialised(key); + TxnId blocking = commandsForKey.current().blockedOnTxnId(root, rootExecuteAt); + CommandStore commandStore = safeStore.commandStore(); + if (blocking == null) + { + queuedTxn.add(AsyncChains.success(new Txn(commandStore.id(), depth, null, null, null, key, emptyList(), emptyList()))); + } + else + { + // TODO (required): this type check should not be needed; release accord version that fixes it at origin + if (blocking instanceof CommandsForKey.TxnInfo) + blocking = ((CommandsForKey.TxnInfo) blocking).plainTxnId(); + boolean recurse = visited.add(blocking); + queuedTxn.add(visitTxnAsync(commandStore, blocking, rootExecuteAt, key, depth, recurse)); + } + } +} diff --git a/src/java/org/apache/cassandra/service/accord/IAccordService.java b/src/java/org/apache/cassandra/service/accord/IAccordService.java index 422155449bd9..5915ad281daa 100644 --- a/src/java/org/apache/cassandra/service/accord/IAccordService.java +++ b/src/java/org/apache/cassandra/service/accord/IAccordService.java @@ -19,7 +19,6 @@ package org.apache.cassandra.service.accord; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.List; import java.util.concurrent.TimeUnit; @@ -178,8 +177,6 @@ public AccordCompactionInfos(DurableBefore durableBefore, long minEpoch, AccordC Id nodeId(); - List debugTxnBlockedGraph(TxnId txnId); - long minEpoch(); void awaitDone(TableId id, long epoch); @@ -341,12 +338,6 @@ public Id nodeId() throw new UnsupportedOperationException(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return Collections.emptyList(); - } - @Override public long minEpoch() { @@ -551,12 +542,6 @@ public Id nodeId() return delegate.nodeId(); } - @Override - public List debugTxnBlockedGraph(TxnId txnId) - { - return delegate.debugTxnBlockedGraph(txnId); - } - @Override public long minEpoch() { diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java index 23ff085c778f..413b1553fca3 100644 --- a/test/unit/org/apache/cassandra/cql3/CQLTester.java +++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java @@ -3185,7 +3185,7 @@ public String describe() private static String formatValue(ByteBuffer bb, AbstractType type) { - if (bb == null) + if (bb == null || (!bb.hasRemaining() && type.isEmptyValueMeaningless())) return "null"; if (type instanceof CollectionType) diff --git a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java index c072e6403c06..6cac2f25fcb4 100644 --- a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java @@ -67,7 +67,6 @@ import org.apache.cassandra.service.accord.api.TokenKey; import org.apache.cassandra.tcm.ClusterMetadata; import org.apache.cassandra.transport.Dispatcher; -import org.apache.cassandra.utils.ByteBufferUtil; import org.apache.cassandra.utils.Clock; import org.apache.cassandra.utils.concurrent.Condition; import org.assertj.core.api.Assertions; @@ -248,8 +247,8 @@ public void completedTxn() throws ExecutionException, InterruptedException getBlocking(accord.node().coordinate(id, txn)); spinUntilSuccess(() -> assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.ReadyToExecute.name(), SaveStatus.Applying.name(), SaveStatus.Applied.name())))); - spinUntilSuccess(() -> assertRows(execute(QUERY_TXN, id.toString()), row(id.toString(), "Applied"))); + row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.ReadyToExecute.name(), SaveStatus.Applying.name(), SaveStatus.Applied.name())))); + assertRows(execute(QUERY_TXN, id.toString()), row(id.toString(), "Applied")); assertRows(execute(QUERY_JOURNAL, id.toString()), row(id.toString(), "PreAccepted"), row(id.toString(), "Applying"), row(id.toString(), "Applied"), row(id.toString(), null)); assertRows(execute(QUERY_COMMANDS_FOR_KEY, keyStr), row(id.toString(), "APPLIED_DURABLE")); } @@ -307,11 +306,10 @@ public void inflight() throws ExecutionException, InterruptedException filter.preAccept.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); - + row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), - row(id.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, SaveStatus.ReadyToExecute.name())); + row(id.toString(), anyInt(), 0, "", "", any(), SaveStatus.ReadyToExecute.name())); } finally { @@ -343,12 +341,13 @@ public void blocked() throws ExecutionException, InterruptedException accord.node().coordinate(first, createTxn(insertTxn, 0, 0, 0, 0, 0)).beginAsResult(); filter.preAccept.awaitThrowUncheckedOnInterrupt(); - spinUntilSuccess(() ->assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), - row(first.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", any(), null, anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name())))); - + assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); - spinUntilSuccess(() -> assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), - row(first.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + + filter.reset(); TxnId second = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); filter.reset(); @@ -363,12 +362,10 @@ public void blocked() throws ExecutionException, InterruptedException return rs.size() == 2; }); assertRows(execute(QUERY_TXN_BLOCKED_BY, second.toString()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.Stable.name()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 1, first.toString(), "Key", anyNonNull(), anyNonNull(), SaveStatus.ReadyToExecute.name())); - + row(second.toString(), anyInt(), 0, "", "", anyNonNull(), SaveStatus.Stable.name()), + row(second.toString(), anyInt(), 1, any(), first.toString(), anyNonNull(), SaveStatus.ReadyToExecute.name())); assertRows(execute(QUERY_TXN_BLOCKED_BY + " AND depth < 1", second.toString()), - row(second.toString(), KEYSPACE, tableName, anyInt(), 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, "Self", anyNonNull(), null, SaveStatus.Stable.name())); - + row(second.toString(), anyInt(), 0, any(), "", anyNonNull(), SaveStatus.Stable.name())); } finally { From 03d5e794a1a2162c2b351e4c9cae1dcd492c91b3 Mon Sep 17 00:00:00 2001 From: Benedict Elliott Smith Date: Sat, 13 Sep 2025 10:50:32 +0100 Subject: [PATCH 4/6] RemoteToLocalVirtualKeyspace: supporting access to all nodes' local virtual tables from any node in the cluster patch by Benedict; reviewed by Alex Petrov for CASSANDRA-20900 --- .../cassandra/db/filter/ColumnFilter.java | 46 +- .../apache/cassandra/db/filter/RowFilter.java | 50 +- .../db/virtual/AccordDebugRemoteKeyspace.java | 31 + .../virtual/RemoteToLocalVirtualKeyspace.java | 29 + .../db/virtual/RemoteToLocalVirtualTable.java | 609 ++++++++++++++++++ .../db/virtual/VirtualKeyspaceRegistry.java | 3 + .../cassandra/db/virtual/VirtualMutation.java | 44 ++ .../cassandra/db/virtual/VirtualTable.java | 2 + .../exceptions/ExceptionSerializer.java | 2 +- .../cassandra/exceptions/RequestFailure.java | 3 +- .../cassandra/locator/RemoteStrategy.java | 29 + src/java/org/apache/cassandra/net/Verb.java | 3 + .../cassandra/schema/SchemaConstants.java | 3 +- .../cassandra/service/CassandraDaemon.java | 4 + .../db/virtual/AccordDebugKeyspaceTest.java | 186 +++++- 15 files changed, 1033 insertions(+), 11 deletions(-) create mode 100644 src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java create mode 100644 src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java create mode 100644 src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java create mode 100644 src/java/org/apache/cassandra/locator/RemoteStrategy.java diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java index bc17dd0158f1..83f14a10736a 100644 --- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java +++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java @@ -33,6 +33,7 @@ import org.apache.cassandra.io.util.DataOutputPlus; import org.apache.cassandra.schema.ColumnMetadata; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.utils.btree.BTree; /** * Represents which (non-PK) columns (and optionally which sub-part of a column for complex columns) are selected @@ -64,7 +65,6 @@ */ public abstract class ColumnFilter { - public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE); public static final Serializer serializer = new Serializer(); @@ -305,6 +305,11 @@ public boolean isWildcard() return false; } + /** + * Rebinds matching columns into a new filter; ignores any missing but fails if any are a different type + */ + public abstract ColumnFilter rebind(TableMetadata newTable); + /** * Returns the CQL string corresponding to this {@code ColumnFilter}. * @@ -630,6 +635,12 @@ public boolean isWildcard() return true; } + @Override + public ColumnFilter rebind(TableMetadata newTable) + { + return new WildCardColumnFilter(ColumnFilter.rebind(newTable, fetchedAndQueried)); + } + @Override protected SortedSetMultimap subSelections() { @@ -779,6 +790,17 @@ public Tester newTester(ColumnMetadata column) return new Tester(fetchingStrategy.fetchesAllColumns(column.isStatic()), s.iterator()); } + @Override + public ColumnFilter rebind(TableMetadata newTable) + { + RegularAndStaticColumns queried = ColumnFilter.rebind(newTable, this.queried); + RegularAndStaticColumns fetched = this.queried == this.fetched ? queried : ColumnFilter.rebind(newTable, this.fetched); + SortedSetMultimap subSelections = this.subSelections; + if (subSelections != null) + subSelections = TreeMultimap.create(subSelections); + return new SelectionColumnFilter(fetchingStrategy, queried, fetched, subSelections); + } + @Override protected SortedSetMultimap subSelections() { @@ -1003,4 +1025,26 @@ private long subSelectionsSerializedSize(SortedSetMultimap builder = BTree.fastBuilder()) + { + for (ColumnMetadata in : columns) + { + ColumnMetadata out = newTable.getColumn(in.name); + if (out != null) + builder.add(out); + } + return Columns.from(builder); + } + } } diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java index 12c98c72782a..16ba694e1424 100644 --- a/src/java/org/apache/cassandra/db/filter/RowFilter.java +++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java @@ -141,7 +141,7 @@ public void addCustomIndexExpression(TableMetadata metadata, IndexMetadata targe add(new CustomExpression(metadata, targetIndex, value)); } - private void add(Expression expression) + public void add(Expression expression) { expression.validate(); expressions.add(expression); @@ -549,6 +549,28 @@ public void validateForIndexing() "Index expression values may not be larger than 64K"); } + /** + * Rebind this expression to a table metadata that is expected to have equivalent columns. + * If any referenced column is missing, returns null; + * if any referenced column has a different type throws an exception + */ + public Expression rebind(TableMetadata newTable) + { + throw new UnsupportedOperationException("Expression " + toString(true) + " does not support rebinding to another table definition"); + } + + protected static ColumnMetadata rebind(ColumnMetadata in, TableMetadata newTable) + { + ColumnMetadata out = newTable.getColumn(in.name); + if (out == null) + return null; + + if (!out.type.equals(in.type) && !out.type.isCompatibleWith(in.type) || !in.type.isCompatibleWith(out.type)) + throw new IllegalArgumentException("The provided TableMetadata is not compatible with the expression"); + + return out; + } + /** * Returns whether the provided row satisfied this expression or not. * @@ -734,6 +756,16 @@ public static class SimpleExpression extends Expression super(column, operator, value); } + @Override + public Expression rebind(TableMetadata newTable) + { + ColumnMetadata out = rebind(column, newTable); + if (out == null) + return null; + + return new SimpleExpression(out, operator, value); + } + @Override public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, long nowInSec) { @@ -853,6 +885,16 @@ public void validate() throws InvalidRequestException checkBindValueSet(value, "Unsupported unset map value for column %s", column.name); } + @Override + public Expression rebind(TableMetadata newTable) + { + ColumnMetadata out = rebind(column, newTable); + if (out == null) + return null; + + return new MapElementExpression(out, key, operator, value); + } + @Override public ByteBuffer getIndexValue() { @@ -978,6 +1020,12 @@ protected Kind kind() return Kind.CUSTOM; } + @Override + public Expression rebind(TableMetadata newTable) + { + return new CustomExpression(table, targetIndex, value); + } + // Filtering by custom expressions isn't supported yet, so just accept any row @Override public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, long nowInSec) diff --git a/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java b/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java new file mode 100644 index 000000000000..8698b3ccd821 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/AccordDebugRemoteKeyspace.java @@ -0,0 +1,31 @@ +/* + * 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.db.virtual; + +import org.apache.cassandra.schema.SchemaConstants; + +public class AccordDebugRemoteKeyspace extends RemoteToLocalVirtualKeyspace +{ + public static final AccordDebugRemoteKeyspace instance = new AccordDebugRemoteKeyspace(SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.instance); + + public AccordDebugRemoteKeyspace(String name, VirtualKeyspace wrap) + { + super(name, wrap); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java new file mode 100644 index 000000000000..1f395045bb9c --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualKeyspace.java @@ -0,0 +1,29 @@ +/* + * 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.db.virtual; + +import java.util.stream.Collectors; + +public class RemoteToLocalVirtualKeyspace extends VirtualKeyspace +{ + public RemoteToLocalVirtualKeyspace(String name, VirtualKeyspace wrap) + { + super(name, wrap.tables().stream().map(vt -> new RemoteToLocalVirtualTable(name, vt)).collect(Collectors.toList())); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java new file mode 100644 index 000000000000..c2a1c6a9ad20 --- /dev/null +++ b/src/java/org/apache/cassandra/db/virtual/RemoteToLocalVirtualTable.java @@ -0,0 +1,609 @@ +/* + * 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.db.virtual; + +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NavigableSet; +import java.util.function.Function; + +import accord.utils.Invariants; +import org.apache.cassandra.cql3.Operator; +import org.apache.cassandra.db.BufferClusteringBound; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.ClusteringPrefix; +import org.apache.cassandra.db.DataRange; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionInfo; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.db.PartitionRangeReadCommand; +import org.apache.cassandra.db.RangeTombstone; +import org.apache.cassandra.db.ReadCommand; +import org.apache.cassandra.db.ReadResponse; +import org.apache.cassandra.db.SinglePartitionReadCommand; +import org.apache.cassandra.db.Slice; +import org.apache.cassandra.db.Slices; +import org.apache.cassandra.db.TruncateRequest; +import org.apache.cassandra.db.filter.ClusteringIndexFilter; +import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.filter.DataLimits; +import org.apache.cassandra.db.filter.IndexHints; +import org.apache.cassandra.db.filter.RowFilter; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator; +import org.apache.cassandra.db.rows.BTreeRow; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.db.rows.ColumnData; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.exceptions.InvalidRequestException; +import org.apache.cassandra.exceptions.RequestFailure; +import org.apache.cassandra.locator.InetAddressAndPort; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.RequestCallback; +import org.apache.cassandra.net.Verb; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.membership.NodeId; +import org.apache.cassandra.utils.btree.BTree; +import org.apache.cassandra.utils.concurrent.AsyncPromise; +import org.apache.cassandra.utils.concurrent.Promise; +import org.apache.cassandra.utils.concurrent.SyncPromise; + +import static org.apache.cassandra.db.ClusteringBound.BOTTOM; +import static org.apache.cassandra.db.ClusteringBound.TOP; +import static org.apache.cassandra.db.ClusteringBound.boundKind; +import static org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts.ALLOW; +import static org.apache.cassandra.db.virtual.VirtualTable.Sorted.SORTED; + +public class RemoteToLocalVirtualTable extends AbstractLazyVirtualTable +{ + private static final int MAX_CONCURRENCY = 8; + final TableMetadata local; + final boolean allowFilteringImplicitly; + final boolean allowFilteringLocalPartitionKeysImplicitly; + + public RemoteToLocalVirtualTable(String keyspace, VirtualTable virtualTable) + { + super(wrap(keyspace, virtualTable.name(), virtualTable.metadata()), virtualTable instanceof AbstractLazyVirtualTable ? ((AbstractLazyVirtualTable) virtualTable).onTimeout() : OnTimeout.FAIL, virtualTable.sorted(), SORTED); + this.local = virtualTable.metadata(); + this.allowFilteringImplicitly = virtualTable.allowFilteringImplicitly(); + this.allowFilteringLocalPartitionKeysImplicitly = virtualTable.allowFilteringPrimaryKeysImplicitly(); + } + + @Override + public boolean allowFilteringImplicitly() + { + return allowFilteringImplicitly; + } + + @Override + public boolean allowFilteringPrimaryKeysImplicitly() + { + return true; + } + + private static TableMetadata wrap(String keyspace, String name, TableMetadata local) + { + if (local.partitionKeyColumns().size() != 1 && !(local.partitionKeyType instanceof CompositeType)) + throw new IllegalArgumentException("Underlying table must have a single partition key, else use CompositeType for its partitioner"); + TableMetadata.Builder builder = TableMetadata.builder(keyspace, name); + builder.partitioner(new LocalPartitioner(Int32Type.instance)); + builder.addPartitionKeyColumn("node_id", Int32Type.instance); + for (ColumnMetadata cm : local.partitionKeyColumns()) + builder.addClusteringColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + for (ColumnMetadata cm : local.clusteringColumns()) + builder.addClusteringColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + // we don't add static columns as they can't be modelled correctly with the insertion of a prefix partition column + for (ColumnMetadata cm : local.regularColumns()) + { + if (!cm.isComplex()) + builder.addRegularColumn(cm.name, cm.type, cm.getMask(), cm.getColumnConstraints()); + } + builder.kind(TableMetadata.Kind.VIRTUAL); + return builder.build(); + } + + @Override + protected void collect(PartitionsCollector collector) + { + ClusterMetadata cm = ClusterMetadata.current(); + NavigableSet matchingIds = cm.directory.states.keySet(); + DataRange dataRange = collector.dataRange(); + AbstractBounds bounds = dataRange.keyRange(); + { + NodeId start = null; + if (!bounds.left.isMinimum()) + { + if (!(bounds.left instanceof DecoratedKey)) + throw new InvalidRequestException(metadata + " does not support filtering by token or incomplete partition keys"); + start = new NodeId(Int32Type.instance.compose(((DecoratedKey) bounds.left).getKey())); + } + NodeId end = null; + if (!bounds.right.isMaximum()) + { + if (!(bounds.right instanceof DecoratedKey)) + throw new InvalidRequestException(metadata + " does not support filtering by token or incomplete partition keys"); + end = new NodeId(Int32Type.instance.compose(((DecoratedKey) bounds.right).getKey())); + } + if (start != null && end != null) matchingIds = matchingIds.subSet(start, bounds.isStartInclusive(), end, bounds.isEndInclusive()); + else if (start != null) matchingIds = matchingIds.tailSet(start, bounds.isStartInclusive()); + else if (end != null) matchingIds = matchingIds.headSet(end, bounds.isEndInclusive()); + } + if (dataRange.isReversed()) + matchingIds = matchingIds.descendingSet(); + + RowFilter rowFilter = rebind(local, collector.rowFilter()); + ColumnFilter columnFilter = collector.columnFilter().rebind(local); + // TODO (expected): count this down as we progress where possible (or have AbstractLazyVirtualTable do it for us) + DataLimits limits = collector.limits(); + + Function pksToCks = partitionKeyToClusterings(metadata, local); + ArrayDeque pending = new ArrayDeque<>(); + matchingIds.forEach(id -> { + InetAddressAndPort endpoint = cm.directory.endpoint(id); + DecoratedKey remoteKey = metadata.partitioner.decorateKey(Int32Type.instance.decompose(id.id())); + ClusteringIndexFilter filter = dataRange.clusteringIndexFilter(remoteKey); + Slices slices = filter.getSlices(metadata); + + int i = 0, advance = 1, end = slices.size(); + if (dataRange.isReversed()) + { + i = slices.size() - 1; + end = -1; + advance = -1; + } + + PartitionCollector partition = collector.partition(id.id()); + while (i != end) + { + List request = rebind(local, slices.get(i), dataRange.isReversed(), rowFilter, columnFilter); + for (Request send : request) + { + ReadCommand readCommand; + if (send.dataRange.startKey().equals(send.dataRange.stopKey()) && !send.dataRange.startKey().isMinimum()) + readCommand = SinglePartitionReadCommand.create(local, collector.nowInSeconds(), send.columnFilter, send.rowFilter, limits, (DecoratedKey) send.dataRange.startKey(), send.dataRange.clusteringIndexFilter(remoteKey), ALLOW); + else + readCommand = PartitionRangeReadCommand.create(local, collector.nowInSeconds(), send.columnFilter, send.rowFilter, limits, send.dataRange); + + RequestAndResponse rr = new RequestAndResponse(partition, readCommand); + send(rr, endpoint); + pending.addLast(rr); + + boolean selectsOneRow = selectsOneRow(local, send.dataRange, remoteKey); + while (pending.size() >= (selectsOneRow ? 1 : MAX_CONCURRENCY)) + collect(collector, pending.pollFirst(), pksToCks); + } + i += advance; + } + }); + while (!pending.isEmpty()) + collect(collector, pending.pollFirst(), pksToCks); + } + + private static class RequestAndResponse extends SyncPromise + { + final PartitionCollector partition; + final ReadCommand readCommand; + private RequestAndResponse(PartitionCollector partition, ReadCommand readCommand) + { + this.partition = partition; + this.readCommand = readCommand; + } + } + + private static class Request + { + final DataRange dataRange; + final RowFilter rowFilter; + final ColumnFilter columnFilter; + + private Request(DataRange dataRange, RowFilter rowFilter, ColumnFilter columnFilter) + { + this.dataRange = dataRange; + this.rowFilter = rowFilter; + this.columnFilter = columnFilter; + } + } + + private void send(RequestAndResponse rr, InetAddressAndPort endpoint) + { + send(Verb.READ_REQ, rr.readCommand, rr, endpoint); + } + + private Promise send(Verb verb, Object payload, InetAddressAndPort endpoint) + { + Promise promise = new AsyncPromise<>(); + send(verb, payload, promise, endpoint); + return promise; + } + + private void send(Verb verb, Object payload, Promise promise, InetAddressAndPort endpoint) + { + // we have to send inline some of the MessagingService logic to circumvent the requirement to use AbstractWriteResponseHandler + Message message = Message.out(verb, payload); + RequestCallback callback = new RequestCallback() + { + @Override public void onResponse(Message msg) { promise.trySuccess(msg.payload); } + @Override public boolean invokeOnFailure() { return true; } + @Override public void onFailure(InetAddressAndPort from, RequestFailure failure) + { + if (failure.failure == null) promise.tryFailure(new RuntimeException(failure.reason.toString())); + else promise.tryFailure(failure.failure); + } + }; + + MessagingService.instance().sendWithCallback(message, endpoint, callback); + } + + private void collect(PartitionsCollector collector, RequestAndResponse rr, Function pksToCks) + { + if (!rr.awaitUntilThrowUncheckedOnInterrupt(collector.deadlineNanos())) + throw new InternalTimeoutException(); + + rr.rethrowIfFailed(); + int pkCount = local.partitionKeyColumns().size(); + ReadResponse response = Invariants.nonNull(rr.getNow()); + try (UnfilteredPartitionIterator partitions = response.makeIterator(rr.readCommand)) + { + while (partitions.hasNext()) + { + try (UnfilteredRowIterator iter = partitions.next()) + { + ByteBuffer[] clusterings = pksToCks.apply(iter.partitionKey()); + while (iter.hasNext()) + { + Unfiltered next = iter.next(); + if (!next.isRow()) + throw new UnsupportedOperationException("Range tombstones not supported"); + + Row row = (Row)next; + { + Clustering clustering = row.clustering(); + for (int j = 0 ; j < clustering.size(); ++j) + clusterings[pkCount + j] = clustering.bufferAt(j); + } + rr.partition.collect(rows -> { + rows.add((Object[])clusterings) + .lazyCollect(columns -> { + row.forEach(cd -> { + Invariants.require(cd instanceof Cell); + columns.add(cd.column().name.toString(), ((Cell) cd).buffer()); + }); + }); + }); + } + } + } + } + } + + private static boolean selectsOneRow(TableMetadata metadata, DataRange dataRange, DecoratedKey key) + { + if (dataRange.startKey().isMinimum() || !dataRange.startKey().equals(dataRange.stopKey())) + return false; + + if (metadata.clusteringColumns().isEmpty()) + return true; + + Slices slices = dataRange.clusteringIndexFilter(key).getSlices(metadata); + if (slices.size() != 1) + return false; + + Slice slice = slices.get(0); + return slice.start().equals(slice.end()); + } + + private static Function partitionKeyToClusterings(TableMetadata distributed, TableMetadata local) + { + ByteBuffer[] cks = new ByteBuffer[distributed.clusteringColumns().size()]; + if (local.partitionKeyColumns().size() == 1) + { + return pk -> { + cks[0] = pk.getKey(); + return cks.clone(); + }; + } + + CompositeType type = (CompositeType) local.partitionKeyType; + int pkCount = type.types.size(); + return (pk) -> { + System.arraycopy(type.split(pk.getKey()), 0, cks, 0, pkCount); + return cks.clone(); + }; + } + + private static RowFilter rebind(TableMetadata local, RowFilter rowFilter) + { + if (rowFilter.isEmpty()) + return rowFilter; + + RowFilter result = RowFilter.create(false, IndexHints.NONE); + for (RowFilter.Expression in : rowFilter.getExpressions()) + { + RowFilter.Expression out = in.rebind(local); + if (out != null) + result.add(out); + } + return result; + } + + private List rebind(TableMetadata local, Slice slice, boolean reversed, RowFilter rowFilter, ColumnFilter columnFilter) + { + ClusteringBound start = slice.start(); + ClusteringBound end = slice.end(); + int pkCount = local.partitionKeyColumns().size(); + // TODO (expected): we can filter by partition key by inserting a new row filter, but need to impose ALLOW FILTERING restrictions + if (((start.size() > 0 && start.size() < pkCount) || (end.size() > 0 && end.size() < pkCount))) + { + if (!allowFilteringLocalPartitionKeysImplicitly) + throw new InvalidRequestException("Must specify full partition key bounds for the underlying table"); + + List pks = local.partitionKeyColumns(); + ByteBuffer[] starts = start.getBufferArray(); + ByteBuffer[] ends = end.getBufferArray(); + + int minCount = Math.min(start.size(), end.size()); + int maxCount = Math.max(start.size(), end.size()); + int commonPrefixLength = 0; + while (commonPrefixLength < minCount && equalPart(start, end, commonPrefixLength)) + ++commonPrefixLength; + + RowFilter commonRowFilter = rowFilter; + if (commonPrefixLength > 0) + { + commonRowFilter = copy(commonRowFilter); + for (int i = 0 ; i < commonPrefixLength ; ++i) + commonRowFilter.add(pks.get(i), Operator.EQ, starts[i]); + } + + Operator lastStartOp = start.isInclusive() ? Operator.GTE : Operator.GT; + Operator lastEndOp = end.isInclusive() ? Operator.LTE : Operator.LT; + if (commonPrefixLength == Math.max(minCount, maxCount - 1)) + { + // can simply add our remaining filters and continue on our way + addExpressions(commonRowFilter, pks, commonPrefixLength, starts, Operator.GTE, lastStartOp); + addExpressions(commonRowFilter, pks, commonPrefixLength, ends, Operator.LTE, lastEndOp); + return List.of(new Request(DataRange.allData(local.partitioner), commonRowFilter, columnFilter)); + } + + throw new InvalidRequestException("This table currently does not support the complex partial partition key filters implied for the underlying table"); + } + + ByteBuffer[] startBuffers = start.getBufferArray(); + PartitionPosition startBound; + if (start.size() == 0) startBound = local.partitioner.getMinimumToken().minKeyBound(); + else if (pkCount == 1) startBound = local.partitioner.decorateKey(startBuffers[0]); + else startBound = local.partitioner.decorateKey(CompositeType.build(ByteBufferAccessor.instance, Arrays.copyOf(startBuffers, pkCount))); + + ByteBuffer[] endBuffers = end.getBufferArray(); + PartitionPosition endBound; + if (end.size() == 0) endBound = local.partitioner.getMinimumToken().maxKeyBound(); + else if (pkCount == 1) endBound = local.partitioner.decorateKey(endBuffers[0]); + else endBound = local.partitioner.decorateKey(CompositeType.build(ByteBufferAccessor.instance, Arrays.copyOf(endBuffers, pkCount))); + + AbstractBounds bounds = AbstractBounds.bounds(startBound, start.isEmpty() || start.size() > pkCount || start.isInclusive(), + endBound, end.isEmpty() || end.size() > pkCount || end.isInclusive()); + boolean hasSlices = start.size() > pkCount || end.size() > pkCount; + if (!hasSlices) + return List.of(new Request(new DataRange(bounds, new ClusteringIndexSliceFilter(Slices.ALL, reversed)), rowFilter, columnFilter)); + + ClusteringBound startSlice = ClusteringBound.BOTTOM; + if (start.size() > pkCount) + startSlice = BufferClusteringBound.create(boundKind(true, start.isInclusive()), Arrays.copyOfRange(startBuffers, pkCount, startBuffers.length)); + + ClusteringBound endSlice = ClusteringBound.TOP; + if (end.size() > pkCount) + endSlice = BufferClusteringBound.create(boundKind(false, end.isInclusive()), Arrays.copyOfRange(startBuffers, pkCount, startBuffers.length)); + + if (startBound.equals(endBound)) + return List.of(new Request(new DataRange(bounds, filter(local, startSlice, endSlice, reversed)), rowFilter, columnFilter)); + + List result = new ArrayList<>(3); + if (startSlice != BOTTOM) + { + AbstractBounds startBoundOnly = AbstractBounds.bounds(startBound, true, startBound, true); + result.add(new Request(new DataRange(startBoundOnly, filter(local, startSlice, TOP, reversed)), rowFilter, columnFilter)); + } + result.add(new Request(new DataRange(AbstractBounds.bounds(bounds.left, bounds.inclusiveLeft() && startSlice == BOTTOM, + bounds.right, bounds.inclusiveRight() && endSlice == TOP), + new ClusteringIndexSliceFilter(Slices.ALL, reversed)), rowFilter, columnFilter) + ); + if (endSlice != TOP) + { + AbstractBounds endBoundOnly = AbstractBounds.bounds(endBound, true, endBound, true); + result.add(new Request(new DataRange(endBoundOnly, filter(local, BOTTOM, endSlice, reversed)), rowFilter, columnFilter)); + } + if (reversed) + Collections.reverse(result); + return result; + } + + @SuppressWarnings({ "rawtypes", "unchecked" }) + private static boolean equalPart(ClusteringBound start, ClusteringBound end, int i) + { + return 0 == start.accessor().compare(start.get(i), end.get(i), end.accessor()); + } + + private static RowFilter copy(RowFilter copy) + { + RowFilter newRowFilter = RowFilter.create(false, IndexHints.NONE); + for (RowFilter.Expression expression : copy) + newRowFilter.add(expression); + return newRowFilter; + } + + private static void addExpressions(RowFilter rowFilter, List cms, int start, ByteBuffer[] values, Operator op, Operator lastOp) + { + for (int i = start ; i < values.length ; ++i) + rowFilter.add(cms.get(i), i + 1 == values.length ? lastOp : op, values[i]); + } + + private static ClusteringIndexSliceFilter filter(TableMetadata metadata, ClusteringBound start, ClusteringBound end, boolean reversed) + { + return new ClusteringIndexSliceFilter(Slices.with(metadata.comparator, Slice.make(start, end)), reversed); + } + + @Override + public void apply(PartitionUpdate update) + { + int nodeId = Int32Type.instance.compose(update.partitionKey().getKey()); + InetAddressAndPort endpoint = ClusterMetadata.current().directory.endpoint(new NodeId(nodeId)); + if (endpoint == null) + throw new InvalidRequestException("Unknown node " + nodeId); + + DeletionInfo deletionInfo = update.deletionInfo(); + if (!deletionInfo.getPartitionDeletion().isLive()) + { + truncate(endpoint).syncThrowUncheckedOnInterrupt(); + return; + } + + int pkCount = local.partitionKeyColumns().size(); + ByteBuffer[] pkBuffer, ckBuffer; + { + int ckCount = local.clusteringColumns().size(); + pkBuffer = pkCount == 1 ? null : new ByteBuffer[pkCount]; + ckBuffer = new ByteBuffer[ckCount]; + } + + PartitionUpdate.Builder builder = null; + ArrayDeque> results = new ArrayDeque<>(); + + if (deletionInfo.hasRanges()) + { + Iterator iterator = deletionInfo.rangeIterator(false); + while (iterator.hasNext()) + { + RangeTombstone rt = iterator.next(); + ClusteringBound start = rt.deletedSlice().start(); + ClusteringBound end = rt.deletedSlice().end(); + if (start.size() < pkCount || end.size() < pkCount) + throw new InvalidRequestException("Range deletions must specify a complete partition key in the underlying table " + metadata); + + for (int i = 0 ; i < pkCount ; ++i) + { + if (0 != start.accessor().compare(start.get(i), end.get(i), end.accessor())) + throw new InvalidRequestException("Range deletions must specify a single partition key in the underlying table " + metadata); + } + + DecoratedKey key = remoteClusteringToLocalPartitionKey(local, start, pkCount, pkBuffer); + builder = maybeRolloverAndWait(key, builder, results, endpoint); + if (start.size() == pkCount && end.size() == pkCount) + { + builder.addPartitionDeletion(rt.deletionTime()); + } + else + { + start = ClusteringBound.create(start.kind(), Clustering.make(remoteClusteringToLocalClustering(start.clustering(), pkCount, ckBuffer))); + end = ClusteringBound.create(end.kind(), Clustering.make(remoteClusteringToLocalClustering(end.clustering(), pkCount, ckBuffer))); + builder.add(new RangeTombstone(Slice.make(start, end), rt.deletionTime())); + } + } + } + + if (!update.staticRow().isEmpty()) + throw new InvalidRequestException("Static rows are not supported for remote table " + metadata); + + try (BTree.FastBuilder columns = BTree.fastBuilder()) + { + for (Row row : update) + { + Clustering clustering = row.clustering(); + DecoratedKey key = remoteClusteringToLocalPartitionKey(local, clustering, pkCount, pkBuffer); + builder = maybeRolloverAndWait(key, builder, results, endpoint); + Clustering newClustering = Clustering.make(remoteClusteringToLocalClustering(clustering, pkCount, ckBuffer)); + columns.reset(); + for (ColumnData cd : row) + columns.add(rebind(local, cd)); + builder.add(BTreeRow.create(newClustering, row.primaryKeyLivenessInfo(), row.deletion(), columns.build())); + } + } + + if (builder != null) + results.add(send(Verb.VIRTUAL_MUTATION_REQ, new VirtualMutation(builder.build()), endpoint)); + + while (!results.isEmpty()) + results.pollFirst().syncThrowUncheckedOnInterrupt(); + } + + private PartitionUpdate.Builder maybeRolloverAndWait(DecoratedKey key, PartitionUpdate.Builder builder, ArrayDeque> waiting, InetAddressAndPort endpoint) + { + if (builder == null || !builder.partitionKey().equals(key)) + { + if (builder != null) + waiting.add(send(Verb.VIRTUAL_MUTATION_REQ, new VirtualMutation(builder.build()), endpoint)); + builder = new PartitionUpdate.Builder(local, key, local.regularAndStaticColumns(), 8); + while (waiting.size() >= MAX_CONCURRENCY) + waiting.pollFirst().syncThrowUncheckedOnInterrupt(); + } + return builder; + } + + private Promise truncate(InetAddressAndPort endpoint) + { + return send(Verb.TRUNCATE_REQ, new TruncateRequest(local.keyspace, local.name), endpoint); + } + + private static ColumnData rebind(TableMetadata local, ColumnData cd) + { + ColumnMetadata column = local.getColumn(cd.column().name); + + Invariants.require(column != null, cd.column() + " not found in " + local); + Invariants.require(!column.isComplex(), "Complex column " + column + " not supported; should have been removed from metadata"); + + return ((Cell) cd).withUpdatedColumn(column); + } + + private static DecoratedKey remoteClusteringToLocalPartitionKey(TableMetadata local, ClusteringPrefix clustering, int pkCount, ByteBuffer[] pkBuffer) + { + ByteBuffer bytes; + if (pkCount == 1) bytes = clustering.bufferAt(0); + else + { + for (int i = 0 ; i < pkBuffer.length ; ++i) + pkBuffer[i] = clustering.bufferAt(i); + bytes = CompositeType.build(ByteBufferAccessor.instance, pkBuffer); + } + return local.partitioner.decorateKey(bytes); + } + + private static ByteBuffer[] remoteClusteringToLocalClustering(ClusteringPrefix clustering, int pkCount, ByteBuffer[] ckBuffer) + { + for (int i = pkCount ; i < clustering.size(); ++i) + ckBuffer[i - pkCount] = clustering.bufferAt(i); + + return Arrays.copyOf(ckBuffer, clustering.size() - pkCount); + } +} diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java index 23814cdf59a0..653536c4320d 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualKeyspaceRegistry.java @@ -24,8 +24,11 @@ import com.google.common.collect.Iterables; import org.apache.cassandra.schema.KeyspaceMetadata; +import org.apache.cassandra.schema.Schema; import org.apache.cassandra.schema.TableId; import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; public final class VirtualKeyspaceRegistry { diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java index ee98da29c115..ddfe5a843ec4 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualMutation.java @@ -17,7 +17,9 @@ */ package org.apache.cassandra.db.virtual; +import java.io.IOException; import java.util.Collection; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; import java.util.function.Supplier; @@ -26,14 +28,27 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; +import accord.utils.Invariants; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.db.IMutation; import org.apache.cassandra.db.Mutation; import org.apache.cassandra.db.ReadCommand.PotentialTxnConflicts; +import org.apache.cassandra.db.TypeSizes; import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.io.IVersionedSerializer; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.net.IVerbHandler; +import org.apache.cassandra.net.Message; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.net.NoPayload; import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.serializers.CollectionSerializer; import org.apache.cassandra.service.ClientState; +import org.apache.cassandra.utils.CollectionSerializers; /** * A specialised IMutation implementation for virtual keyspaces. @@ -42,6 +57,35 @@ */ public final class VirtualMutation implements IMutation { + public static final IVersionedSerializer serializer = new IVersionedSerializer() + { + @Override + public void serialize(VirtualMutation t, DataOutputPlus out, int version) throws IOException + { + Invariants.require(t.modifications.size() == 1); + PartitionUpdate.serializer.serialize(t.modifications.values().iterator().next(), out, version); + } + + @Override + public VirtualMutation deserialize(DataInputPlus in, int version) throws IOException + { + PartitionUpdate update = PartitionUpdate.serializer.deserialize(in, version, DeserializationHelper.Flag.FROM_REMOTE); + return new VirtualMutation(update); + } + + @Override + public long serializedSize(VirtualMutation t, int version) + { + Invariants.require(t.modifications.size() == 1); + return PartitionUpdate.serializer.serializedSize(t.modifications.values().iterator().next(), version); + } + }; + + public static final IVerbHandler handler = message -> { + message.payload.apply(); + MessagingService.instance().respond(NoPayload.noPayload, message); + }; + private final String keyspaceName; private final DecoratedKey partitionKey; private final ImmutableMap modifications; diff --git a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java index 3f05ed3a690c..0423b12a4d05 100644 --- a/src/java/org/apache/cassandra/db/virtual/VirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/VirtualTable.java @@ -100,4 +100,6 @@ default boolean allowFilteringPrimaryKeysImplicitly() { return allowFilteringImplicitly(); } + + default Sorted sorted() { return Sorted.UNSORTED; } } diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java b/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java index de379739a383..834abca49dd0 100644 --- a/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java +++ b/src/java/org/apache/cassandra/exceptions/ExceptionSerializer.java @@ -46,7 +46,7 @@ public class ExceptionSerializer { public static class RemoteException extends RuntimeException { - private final String originalClass; + public final String originalClass; public RemoteException(String originalClass, String originalMessage, StackTraceElement[] stackTrace) { diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailure.java b/src/java/org/apache/cassandra/exceptions/RequestFailure.java index b9bba7fc7061..03d088a239b6 100644 --- a/src/java/org/apache/cassandra/exceptions/RequestFailure.java +++ b/src/java/org/apache/cassandra/exceptions/RequestFailure.java @@ -31,6 +31,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.cassandra.exceptions.ExceptionSerializer.nullableRemoteExceptionSerializer; +import static org.apache.cassandra.exceptions.RequestFailureReason.UNKNOWN; /** * Allow inclusion of a serialized exception in failure response messages @@ -125,7 +126,7 @@ public static RequestFailure forException(Throwable t) if (t instanceof CoordinatorBehindException) return COORDINATOR_BEHIND; - return UNKNOWN; + return new RequestFailure(t); } public static RequestFailure forReason(RequestFailureReason reason) diff --git a/src/java/org/apache/cassandra/locator/RemoteStrategy.java b/src/java/org/apache/cassandra/locator/RemoteStrategy.java new file mode 100644 index 000000000000..515462ee980b --- /dev/null +++ b/src/java/org/apache/cassandra/locator/RemoteStrategy.java @@ -0,0 +1,29 @@ +/* + * 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.locator; + +import java.util.Map; + +public class RemoteStrategy extends LocalStrategy +{ + public RemoteStrategy(String keyspaceName, Map configOptions) + { + super(keyspaceName, configOptions); + } +} diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index d24c9e64adff..4a0ebb2c2216 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -44,6 +44,7 @@ import org.apache.cassandra.db.TruncateRequest; import org.apache.cassandra.db.TruncateResponse; import org.apache.cassandra.db.TruncateVerbHandler; +import org.apache.cassandra.db.virtual.VirtualMutation; import org.apache.cassandra.exceptions.RequestFailure; import org.apache.cassandra.gms.GossipDigestAck; import org.apache.cassandra.gms.GossipDigestAck2; @@ -195,6 +196,8 @@ public enum Verb { MUTATION_RSP (60, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), MUTATION_REQ (0, P3, writeTimeout, MUTATION, () -> Mutation.serializer, () -> MutationVerbHandler.instance, MUTATION_RSP ), + VIRTUAL_MUTATION_RSP (200, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), + VIRTUAL_MUTATION_REQ (201, P3, writeTimeout, MUTATION, () -> VirtualMutation.serializer, () -> VirtualMutation.handler, VIRTUAL_MUTATION_RSP), HINT_RSP (61, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), HINT_REQ (1, P4, writeTimeout, MUTATION, () -> HintMessage.serializer, () -> HintVerbHandler.instance, HINT_RSP ), READ_REPAIR_RSP (62, P1, writeTimeout, REQUEST_RESPONSE, () -> NoPayload.serializer, RESPONSE_HANDLER ), diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java index 6e3cb7a72eb7..ada413537d15 100644 --- a/src/java/org/apache/cassandra/schema/SchemaConstants.java +++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java @@ -57,6 +57,7 @@ public final class SchemaConstants public static final String VIRTUAL_VIEWS = "system_views"; public static final String VIRTUAL_METRICS = "system_metrics"; public static final String VIRTUAL_ACCORD_DEBUG = "system_accord_debug"; + public static final String VIRTUAL_ACCORD_DEBUG_REMOTE = "system_accord_debug_remote"; public static final String DUMMY_KEYSPACE_OR_TABLE_NAME = "--dummy--"; @@ -66,7 +67,7 @@ public final class SchemaConstants /* virtual table system keyspace names */ public static final Set VIRTUAL_SYSTEM_KEYSPACE_NAMES = - ImmutableSet.of(VIRTUAL_SCHEMA, VIRTUAL_VIEWS, VIRTUAL_METRICS); + ImmutableSet.of(VIRTUAL_SCHEMA, VIRTUAL_VIEWS, VIRTUAL_METRICS, VIRTUAL_ACCORD_DEBUG, VIRTUAL_ACCORD_DEBUG_REMOTE); /* replicate system keyspace names (the ones with a "true" replication strategy) */ public static final Set REPLICATED_SYSTEM_KEYSPACE_NAMES = diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index d8acdbc26666..e9aa4c96c013 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -59,6 +59,7 @@ import org.apache.cassandra.db.SystemKeyspaceMigrator41; import org.apache.cassandra.db.commitlog.CommitLog; import org.apache.cassandra.db.virtual.AccordDebugKeyspace; +import org.apache.cassandra.db.virtual.AccordDebugRemoteKeyspace; import org.apache.cassandra.db.virtual.LogMessagesTable; import org.apache.cassandra.db.virtual.SlowQueriesTable; import org.apache.cassandra.db.virtual.SystemViewsKeyspace; @@ -556,7 +557,10 @@ public void setupVirtualKeyspaces() VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(VIRTUAL_METRICS, createMetricsKeyspaceTables())); if (DatabaseDescriptor.getAccord().enable_virtual_debug_only_keyspace) + { VirtualKeyspaceRegistry.instance.register(AccordDebugKeyspace.instance); + VirtualKeyspaceRegistry.instance.register(AccordDebugRemoteKeyspace.instance); + } // Flush log messages to system_views.system_logs virtual table as there were messages already logged // before that virtual table was instantiated. diff --git a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java index 6cac2f25fcb4..c6ab0b04322b 100644 --- a/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java +++ b/test/unit/org/apache/cassandra/db/virtual/AccordDebugKeyspaceTest.java @@ -52,6 +52,8 @@ import org.apache.cassandra.cql3.UntypedResultSet; import org.apache.cassandra.db.ConsistencyLevel; import org.apache.cassandra.dht.Murmur3Partitioner.LongToken; +import org.apache.cassandra.exceptions.ExceptionSerializer; +import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.locator.InetAddressAndPort; import org.apache.cassandra.net.Message; import org.apache.cassandra.net.MessagingService; @@ -88,54 +90,108 @@ public class AccordDebugKeyspaceTest extends CQLTester private static final String QUERY_TXN_BLOCKED_BY = String.format("SELECT * FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_BLOCKED_BY); + private static final String QUERY_TXN_BLOCKED_BY_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_BLOCKED_BY); + private static final String QUERY_COMMANDS_FOR_KEY = String.format("SELECT txn_id, status FROM %s.%s WHERE key=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.COMMANDS_FOR_KEY); + private static final String QUERY_COMMANDS_FOR_KEY_REMOTE = + String.format("SELECT txn_id, status FROM %s.%s WHERE node_id = ? AND key=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.COMMANDS_FOR_KEY); + private static final String QUERY_TXN = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_TXN_REMOTE = + String.format("SELECT txn_id, save_status FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + private static final String QUERY_TXNS = String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_TXNS_REMOTE = + String.format("SELECT save_status FROM %s.%s WHERE node_id = ? AND command_store_id = ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + private static final String QUERY_TXNS_SEARCH = String.format("SELECT save_status FROM %s.%s WHERE command_store_id = ? AND txn_id > ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN); + private static final String QUERY_TXNS_SEARCH_REMOTE = + String.format("SELECT save_status FROM %s.%s WHERE node_id = ? AND command_store_id = ? AND txn_id > ? LIMIT 5", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN); + private static final String QUERY_JOURNAL = String.format("SELECT txn_id, save_status FROM %s.%s WHERE txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.JOURNAL); + private static final String ERASE_JOURNAL_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND command_store_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.JOURNAL); + + private static final String QUERY_JOURNAL_REMOTE = + String.format("SELECT txn_id, save_status FROM %s.%s WHERE node_id = ? AND txn_id=?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.JOURNAL); + private static final String SET_TRACE = String.format("UPDATE %s.%s SET permits = ? WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String SET_TRACE_REMOTE = + String.format("UPDATE %s.%s SET permits = ? WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACE = String.format("SELECT * FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACE_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE1 = String.format("DELETE FROM %s.%s WHERE txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE1_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE2 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACE); + private static final String UNSET_TRACE2_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACE); + private static final String QUERY_TRACES = String.format("SELECT * FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String QUERY_TRACES_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES1 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ? AND id_micros < ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES1_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ? AND id_micros < ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES2 = String.format("DELETE FROM %s.%s WHERE txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES2_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ? AND event_type = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES3 = String.format("DELETE FROM %s.%s WHERE txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.TXN_TRACES); + private static final String ERASE_TRACES3_REMOTE = + String.format("DELETE FROM %s.%s WHERE node_id = ? AND txn_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.TXN_TRACES); + private static final String QUERY_REDUNDANT_BEFORE = String.format("SELECT * FROM %s.%s", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ = String.format("SELECT * FROM %s.%s WHERE quorum_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND quorum_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ = String.format("SELECT * FROM %s.%s WHERE shard_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG, AccordDebugKeyspace.REDUNDANT_BEFORE); + private static final String QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE = + String.format("SELECT * FROM %s.%s WHERE node_id = ? AND shard_applied >= ?", SchemaConstants.VIRTUAL_ACCORD_DEBUG_REMOTE, AccordDebugKeyspace.REDUNDANT_BEFORE); + @BeforeClass public static void setUpClass() { @@ -170,43 +226,103 @@ public void unknownIsEmpty() public void tracing() { // simple test to confirm basic tracing functionality works, doesn't validate specific behaviours only requesting/querying/erasing + String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); + AccordService accord = accord(); + DatabaseDescriptor.getAccord().fetch_txn = "1s"; + int nodeId = accord.nodeId().id; + AccordMsgFilter filter = new AccordMsgFilter(); MessagingService.instance().outboundSink.add(filter); try { - String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); - AccordService accord = accord(); - DatabaseDescriptor.getAccord().fetch_txn = "1s"; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); + filter.appliesTo(id); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(SET_TRACE, 0, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(UNSET_TRACE1, id.toString()); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); execute(UNSET_TRACE2, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); execute(SET_TRACE, 1, id.toString(), "WAIT_PROGRESS"); assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); - accord.node().coordinate(id, txn).beginAsResult(); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); filter.appliesTo(id); + accord.node().coordinate(id, txn).beginAsResult(); filter.preAccept.awaitThrowUncheckedOnInterrupt(); - filter.apply.awaitThrowUncheckedOnInterrupt(); spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); execute(ERASE_TRACES1, id.toString(), "FETCH", Long.MAX_VALUE); execute(ERASE_TRACES2, id.toString(), "FETCH"); execute(ERASE_TRACES1, id.toString(), "WAIT_PROGRESS", Long.MAX_VALUE); Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); // just check other variants don't fail execute(ERASE_TRACES2, id.toString(), "WAIT_PROGRESS"); execute(ERASE_TRACES3, id.toString()); + + } + finally + { + MessagingService.instance().outboundSink.remove(filter); + } + + filter = new AccordMsgFilter(); + MessagingService.instance().outboundSink.add(filter); + try + { + TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); + Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 1, 1, 1); + filter.appliesTo(id); + + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(SET_TRACE_REMOTE, 0, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(UNSET_TRACE1_REMOTE, nodeId, id.toString()); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + execute(UNSET_TRACE2_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS")); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS")); + execute(SET_TRACE_REMOTE, 1, nodeId, id.toString(), "WAIT_PROGRESS"); + assertRows(execute(QUERY_TRACE, id.toString(), "WAIT_PROGRESS"), row(id.toString(), "WAIT_PROGRESS", 1)); + assertRows(execute(QUERY_TRACE_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"), row(nodeId, id.toString(), "WAIT_PROGRESS", 1)); + accord.node().coordinate(id, txn); + filter.preAccept.awaitThrowUncheckedOnInterrupt(); + filter.apply.awaitThrowUncheckedOnInterrupt(); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + spinUntilSuccess(() -> Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isGreaterThan(0)); + execute(ERASE_TRACES1_REMOTE, nodeId, id.toString(), "FETCH", Long.MAX_VALUE); + execute(ERASE_TRACES2_REMOTE, nodeId, id.toString(), "FETCH"); + execute(ERASE_TRACES1_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS", Long.MAX_VALUE); + Assertions.assertThat(execute(QUERY_TRACES, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_TRACES_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS").size()).isEqualTo(0); + // just check other variants don't fail + execute(ERASE_TRACES2_REMOTE, nodeId, id.toString(), "WAIT_PROGRESS"); + execute(ERASE_TRACES3_REMOTE, nodeId, id.toString()); } finally { @@ -219,6 +335,7 @@ public void redundantBefore() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); var accord = accord(); + int nodeId = accord.nodeId().id; TableId tableId = Schema.instance.getTableMetadata(KEYSPACE, tableName).id; TxnId syncId1 = new TxnId(100, 200, Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, accord.nodeId()); TxnId syncId2 = new TxnId(101, 300, Txn.Kind.ExclusiveSyncPoint, Routable.Domain.Range, accord.nodeId()); @@ -234,6 +351,27 @@ public void redundantBefore() throws ExecutionException, InterruptedException Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ, syncId2.toString()).size()).isEqualTo(1); Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ, syncId1.toString()).size()).isEqualTo(1); Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ, syncId2.toString()).size()).isEqualTo(0); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_REMOTE, nodeId).size()).isGreaterThan(0); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE, nodeId, syncId1.toString()).size()).isEqualTo(2); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_QUORUM_APPLIED_GEQ_REMOTE, nodeId, syncId2.toString()).size()).isEqualTo(1); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE, nodeId, syncId1.toString()).size()).isEqualTo(1); + Assertions.assertThat(execute(QUERY_REDUNDANT_BEFORE_FILTER_SHARD_APPLIED_GEQ_REMOTE, nodeId, syncId2.toString()).size()).isEqualTo(0); + } + + @Test + public void reportInvalidRequestForUnsupportedRemoteToLocal() + { + AccordService accord = accord(); + int nodeId = accord.nodeId().id; + TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); + try + { + execute(ERASE_JOURNAL_REMOTE, nodeId, 1, id.toString()); + } + catch (ExceptionSerializer.RemoteException t) + { + Assertions.assertThat(t.originalClass).isEqualTo(InvalidRequestException.class.getName()); + } } @Test @@ -241,6 +379,7 @@ public void completedTxn() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); String keyStr = txn.keys().get(0).toUnseekable().toString(); @@ -249,15 +388,19 @@ public void completedTxn() throws ExecutionException, InterruptedException spinUntilSuccess(() -> assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.ReadyToExecute.name(), SaveStatus.Applying.name(), SaveStatus.Applied.name())))); assertRows(execute(QUERY_TXN, id.toString()), row(id.toString(), "Applied")); + assertRows(execute(QUERY_TXN_REMOTE, nodeId, id.toString()), row(id.toString(), "Applied")); assertRows(execute(QUERY_JOURNAL, id.toString()), row(id.toString(), "PreAccepted"), row(id.toString(), "Applying"), row(id.toString(), "Applied"), row(id.toString(), null)); + assertRows(execute(QUERY_JOURNAL_REMOTE, nodeId, id.toString()), row(id.toString(), "PreAccepted"), row(id.toString(), "Applying"), row(id.toString(), "Applied"), row(id.toString(), null)); assertRows(execute(QUERY_COMMANDS_FOR_KEY, keyStr), row(id.toString(), "APPLIED_DURABLE")); + assertRows(execute(QUERY_COMMANDS_FOR_KEY_REMOTE, nodeId, keyStr), row(id.toString(), "APPLIED_DURABLE")); } @Test - public void manyTxns() throws ExecutionException, InterruptedException + public void manyTxns() { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; List await = new ArrayList<>(); Txn txn = createTxn(wrapInTxn(String.format("INSERT INTO %s.%s(k, c, v) VALUES (?, ?, ?)", KEYSPACE, tableName)), 0, 0, 0); for (int i = 0 ; i < 100; ++i) @@ -281,6 +424,22 @@ public void manyTxns() throws ExecutionException, InterruptedException row("Applied"), row("Applied") ); + + assertRows(execute(QUERY_TXNS_REMOTE, nodeId, commandStore.id()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); + + assertRows(execute(QUERY_TXNS_SEARCH_REMOTE, nodeId, commandStore.id(), TxnId.NONE.toString()), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied"), + row("Applied") + ); } @Test @@ -293,6 +452,7 @@ public void inflight() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId id = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); String insertTxn = String.format("BEGIN TRANSACTION\n" + " LET r = (SELECT * FROM %s.%s WHERE k = ? AND c = ?);\n" + @@ -307,9 +467,13 @@ public void inflight() throws ExecutionException, InterruptedException filter.preAccept.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), row(id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, id.toString()), + row(nodeId, id.toString(), anyInt(), 0, "", "", any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, id.toString()), row(id.toString(), anyInt(), 0, "", "", any(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, id.toString()), + row(nodeId, id.toString(), anyInt(), 0, "", "", any(), SaveStatus.ReadyToExecute.name())); } finally { @@ -329,6 +493,7 @@ public void blocked() throws ExecutionException, InterruptedException { String tableName = createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c)) WITH transactional_mode = 'full'"); AccordService accord = accord(); + int nodeId = accord.nodeId().id; TxnId first = accord.node().nextTxnIdWithDefaultFlags(Txn.Kind.Write, Routable.Domain.Key); String insertTxn = String.format("BEGIN TRANSACTION\n" + " LET r = (SELECT * FROM %s.%s WHERE k = ? AND c = ?);\n" + @@ -343,9 +508,13 @@ public void blocked() throws ExecutionException, InterruptedException filter.preAccept.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), row(first.toString(), anyInt(), 0, "", any(), any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, first.toString()), + row(first.toString(), anyInt(), 0, "", any(), any(), anyOf(SaveStatus.PreAccepted.name(), SaveStatus.ReadyToExecute.name()))); filter.apply.awaitThrowUncheckedOnInterrupt(); assertRows(execute(QUERY_TXN_BLOCKED_BY, first.toString()), row(first.toString(), anyInt(), 0, "", any(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, first.toString()), + row(nodeId, first.toString(), anyInt(), 0, "", any(), anyNonNull(), SaveStatus.ReadyToExecute.name())); filter.reset(); @@ -366,6 +535,11 @@ public void blocked() throws ExecutionException, InterruptedException row(second.toString(), anyInt(), 1, any(), first.toString(), anyNonNull(), SaveStatus.ReadyToExecute.name())); assertRows(execute(QUERY_TXN_BLOCKED_BY + " AND depth < 1", second.toString()), row(second.toString(), anyInt(), 0, any(), "", anyNonNull(), SaveStatus.Stable.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE, nodeId, second.toString()), + row(nodeId, second.toString(), anyInt(), 0, "", "", anyNonNull(), SaveStatus.Stable.name()), + row(nodeId, second.toString(), anyInt(), 1, any(), first.toString(), anyNonNull(), SaveStatus.ReadyToExecute.name())); + assertRows(execute(QUERY_TXN_BLOCKED_BY_REMOTE + " AND depth < 1", nodeId, second.toString()), + row(nodeId, second.toString(), anyInt(), 0, any(), "", anyNonNull(), SaveStatus.Stable.name())); } finally { From 958ba398a2df77dfcb18ffdc18d1c76dc93a2ca4 Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Thu, 25 Sep 2025 22:42:31 +0200 Subject: [PATCH 5/6] Imports --- .../cassandra/db/virtual/AbstractMutableLazyVirtualTable.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java index cc90fa4a4692..be928c5f52f5 100644 --- a/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java +++ b/src/java/org/apache/cassandra/db/virtual/AbstractMutableLazyVirtualTable.java @@ -21,8 +21,6 @@ import javax.annotation.Nullable; -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringPrefix; import org.apache.cassandra.db.DeletionInfo; import org.apache.cassandra.db.RangeTombstone; import org.apache.cassandra.db.Slice; From 12d3157bb7767325ef3d2e74ea64db90c1d7b61b Mon Sep 17 00:00:00 2001 From: Alex Petrov Date: Thu, 25 Sep 2025 22:42:47 +0200 Subject: [PATCH 6/6] Add distributed test --- .../test/accord/AccordVirtualTableTest.java | 109 ++++++++++++++++++ 1 file changed, 109 insertions(+) create mode 100644 test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java diff --git a/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java new file mode 100644 index 000000000000..f1e9d17ed4b3 --- /dev/null +++ b/test/distributed/org/apache/cassandra/distributed/test/accord/AccordVirtualTableTest.java @@ -0,0 +1,109 @@ +/* + * 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.distributed.test.accord; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.test.log.FuzzTestBase; +import org.apache.cassandra.service.accord.AccordService; +import org.junit.Test; + + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; + +public class AccordVirtualTableTest extends FuzzTestBase +{ + @Test + public void testCoordinationsTableDistributed() throws Exception + { + try (Cluster cluster = init(builder() + .withNodes(3) + .withConfig(config -> config.with(NETWORK, GOSSIP)) + .start())) + { + cluster.schemaChange(withKeyspace("CREATE TABLE %s.test_table (pk int PRIMARY KEY, v int) " + + "WITH transactional_mode = 'full'")); + + // Wait for accord to be fully init + // Perform concurrent transactional operations to generate coordination activity + for (int i = 0; i < 10; i++) { + final int key = i; + cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.test_table (pk, v) VALUES (?, ?)"), + ConsistencyLevel.QUORUM, key, key * 10); + } + + List nodeIds = new ArrayList<>(); + cluster.forEach(i -> { + nodeIds.add(i.callsOnInstance(() -> AccordService.instance().node().id().id).call()); + }); + + for (int callOn = 1; callOn <= 3; callOn++) + { + for (Integer remote : nodeIds) + { + IInvokableInstance node = cluster.get(callOn); + Object[][] remoteRes = cluster.get(remote).executeInternal("SELECT * FROM system_accord_debug_remote.redundant_before where node_id= " + remote); + Object[][] localRes = node.executeInternal("SELECT * FROM system_accord_debug.redundant_before"); + localMatchesRemote(remote, callOn, remoteRes, localRes); + } + } + } + } + + public static void localMatchesRemote(int remote, int callOn, Object[][] remoteRes, Object[][] localRes) + { + if (remoteRes.length == localRes.length) + { + for (int i = 0; i < remoteRes.length; i++) + { + Object[] remoteRow = remoteRes[i]; + Object[] localRow = localRes[i]; + for (int j = 0; j < localRow.length; j++) + { + if (!Objects.equals(remoteRow[i + 1], localRow[i])) + { + throw new AssertionError(String.format("Mismatch while executing query for %d on %d.Row %d column %d. %s != %s\n" + + "Remote: %s\n" + + "Local: %s", + remote, callOn, + i, j, + Objects.toString(remoteRow[i + 1]), Objects.toString(localRow[i]), + toString(remoteRes), + toString(localRes))); + } + } + } + } + } + public static String toString(Object[][] objects) + { + StringBuilder sb = new StringBuilder(); + for (Object[] o : objects) + { + sb.append(Arrays.toString(o)); + } + return sb.toString(); + } +} \ No newline at end of file