This is an automated email from the ASF dual-hosted git repository. bdeggleston pushed a commit to branch cep-45-mutation-tracking in repository https://gitbox.apache.org/repos/asf/cassandra.git
commit 4a85e448fc6a375e4985bdf32ac2b2cddd4ceb35 Author: Blake Eggleston <[email protected]> AuthorDate: Mon Feb 9 15:51:46 2026 -0800 ninja - fix tracked bootstrap validation --- src/java/org/apache/cassandra/db/ColumnFamilyStore.java | 10 ++++++++++ src/java/org/apache/cassandra/db/lifecycle/Tracker.java | 8 ++++++++ .../apache/cassandra/db/streaming/CassandraStreamReceiver.java | 9 ++++++++- 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index f289e881b3..62518be2ab 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -1703,6 +1703,16 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean, Memtable.Owner CompactionManager.instance.submitBackground(this); } + /** + * addSStables variant that allows adding sstables for tracked tables without requiring tracked transfers during bootstrap + */ + public void addSSTableForBootstrap(Collection<SSTableReader> sstables) + { + data.addSSTablesForBootstrap(sstables); + logger.debug("Adding sstables {}", sstables); + CompactionManager.instance.submitBackground(this); + } + /** * Calculate expected file size of SSTable after compaction. * diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java index ca039361e0..a5eedea941 100644 --- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java +++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java @@ -289,6 +289,14 @@ public class Tracker addSSTablesInternal(sstables, false, true, true); } + /** + * addSStables variant for bootstrap. Doesn't validate tracked/untracked or absence of logOffsets data + */ + public void addSSTablesForBootstrap(Collection<SSTableReader> sstables) + { + addSSTablesInternal(sstables, false, true, true); + } + private void addSSTablesInternal(Collection<SSTableReader> sstables, boolean isInitialSSTables, boolean maybeIncrementallyBackup, diff --git a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java index 62f53d7363..111276b20a 100644 --- a/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java +++ b/src/java/org/apache/cassandra/db/streaming/CassandraStreamReceiver.java @@ -268,7 +268,14 @@ public class CassandraStreamReceiver implements StreamReceiver if (cfs.metadata().replicationType().isTracked() && session.streamOperation().isTrackable()) return; - cfs.addSSTables(readers); + if (session.streamOperation() == StreamOperation.BOOTSTRAP) + { + cfs.addSSTableForBootstrap(readers); + } + else + { + cfs.addSSTables(readers); + } Consumer<Integer> onRowCacheInvalidation = invalidatedKeys -> { logger.debug("[Stream #{}] Invalidated {} row cache entries on table {}.{} after stream " + --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
