Skip to content

CASSANDRA-21133 Make bin/sstableupgrade functionally on par with nodetool upgradesstables#4825

Open
arvindKandpal-ksolves wants to merge 2 commits into
apache:trunkfrom
arvindKandpal-ksolves:CASSANDRA-21133
Open

CASSANDRA-21133 Make bin/sstableupgrade functionally on par with nodetool upgradesstables#4825
arvindKandpal-ksolves wants to merge 2 commits into
apache:trunkfrom
arvindKandpal-ksolves:CASSANDRA-21133

Conversation

@arvindKandpal-ksolves
Copy link
Copy Markdown
Contributor

Make bin/sstableupgrade functionally on par with nodetool upgradesstables

What this PR does / why we need it:

This PR brings functional parity between bin/sstableupgrade and nodetool upgradesstables by introducing the -a / --include-all-sstables flag to the offline tool.

Previously, bin/sstableupgrade would automatically skip SSTables that were already on the latest version. This patch allows users to force a rewrite of all SSTables even if they are on the current format.

Technical Details & Safety:

Adding this flag to an offline tool introduced a known edge case: If the offline tool is run while the Cassandra node is online, it consumes a new SSTable ID on disk. However, the live node's internal sstableIdGenerator remains unaware of this. Upon the next flush, the live node attempts to use the same ID, which previously triggered a java.lang.AssertionError in ColumnFamilyStore.newSSTableDescriptor().

To safely resolve this without introducing regressions:

  1. Replaced the strict assert !newDescriptor.fileFor(Components.DATA).exists(); check in newSSTableDescriptor with a safe while(true) loop. It checks if the generated ID already exists on disk. If a collision is detected, it logs a warning and advances the generator, self-healing the state.
  2. Updated StandaloneUpgrader to parse and apply the -a option consistently.
  3. Added a robust test (testNewSSTableDescriptorCollision) in ColumnFamilyStoreTest that probes the current ID, uses SSTableIdFactory to pre-create files for future IDs (N+1, N+2), and verifies that the loop correctly skips them and yields N+3.
  4. Updated flag assertions in StandaloneUpgraderTest.
  5. Updated documentation in sstableupgrade.adoc.

patch by Arvind Kandpal; reviewed by TBD for CASSANDRA-21133

Comment thread src/java/org/apache/cassandra/tools/StandaloneUpgrader.java Outdated
Comment thread src/java/org/apache/cassandra/db/ColumnFamilyStore.java Outdated
sstableIdGenerator.get());
assert !newDescriptor.fileFor(Components.DATA).exists();
Descriptor newDescriptor;
while (true)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

this might not happen in practice but it is worth saying that there is no limit as in how many times we loop, if there is some buggy ID supplier or we exhausted the namespace of these ids and all were on disk then we would spin for ever. Currently we have sequence based generator on integer and UUID generator on ... uuid, the namespace is pretty much "infinite" in practical terms so this we will not hit it, one would have to have like ~2 billions sstables on disk to exhaust (and then it would start to overflow)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yeah, agreed. With ~2 billion for sequence IDs and practically infinite for UUIDs, we shouldn't ever hit an infinite loop in the real case. Thanks for pointing this out.

}

@Test
public void testNewSSTableDescriptorCollision() throws Exception
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

just saying we are not testing UUID-based generator but I dont think doing that is the absolute must given how it behaves.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

if needed we can try to add , otherwise I also thought there is no absolute required.

sstableIdGenerator.get());
assert !newDescriptor.fileFor(Components.DATA).exists();
Descriptor newDescriptor;
while (true)
Copy link
Copy Markdown
Contributor

@smiklosovic smiklosovic May 19, 2026

Choose a reason for hiding this comment

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

I think that thread safety is just delegated to how sstableIdGenerator.get() returns the new id, right? That is AtomicInteger, hence increased atomically, so two threads calling this stuff will always have different ids to check the existence of a file against.

@arvindKandpal-ksolves could you put a comment on while(true) about this and that this loop is meant to be a defense against offline / online generation for tooling's sake instead of in-process flow?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yes, exactly. The atomic increment takes care of thread safety in-process. I've added a comment above the while(true) loop to explain that this is purely a defense against offline tool generation.

@arvindKandpal-ksolves
Copy link
Copy Markdown
Contributor Author

Hi @smiklosovic , I have addressed most of the points, Please take a look.

@smiklosovic smiklosovic requested a review from netudima May 20, 2026 08:34
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants